Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current stateUnder Discussion

Discussion thread

JIRAhere

Motivation

Kafka is designed to allow messages with the same key from the same producer to be consumed in the same order as they are produced. This feature is useful for applications which maintain local states per key. However, as of current design of Kafka, this in-order delivery is not guaranteed if we expand partition of the topic. This KIP proposes a design to support in-order message delivery even if we expand partition of the topic.

Public Interfaces

 

Proposed Changes

1) Changes in the controller for handling partition expansion
Here we describe how topic znode change triggers partition expansion logic in the controller
- User uses kafka-topics.sh to update the topic znode with the new assignment. This triggers the topic znode listener in controller.
- For those partitions of this topic which already have the partition znode, controller increments their leaderEpoch by 1 in the partition znode. Controller sends LeaderAndIsrRequest and wait for LeaderAndIsrResponse. The LeaderAndIsrRequest should include the new leaderEpoch for each partition and the new partition count for the topic.
- For each partition of this topic which does not have the partition znode, controller creates the partition znode, such that the znode data includes the map (priorPartition -> oldLeaderEpoch), where prior partition are partitions of this topic whose partition index is smaller than the given partition.
- Controller continue the existing logic of partition expansion.
Note that this procedure is fault tolerance. If controller fails in any of these step, the new controller can continue creating partition znode following the same procedure.
2) Changes in how broker handle ProduceRequest
- When broker receives LeaderAndIsrRequest, in addition to the existing procedure (e.g. updating the leaderEpochCache for the new leaderEpoch), the broker should record the latest partition count for each topic.
- When broker receives ProduceRequest, for each partition in the request, broker checks whether its partition count is the same as the partition count from the most recent LeaderAndIsrRequest. If yes, broker handles the produce request in the current way. If no, broker rejects this partition with InvalidPartitionCountException. This error extends InvalidMetadaException and should trigger produce to update its metadata and retry.
3) Changes in how produce constructs ProduceRequest
- Producer should include the partition count for each topic in the ProduceRequest.
4) Changes in how consumer handles HeartbeatRequest and HeartbeatResponse
- HeartbeatRequest includes the current position for each partition assigned to the given consumer. It also includes the list of partitions for which it wants to know the position (of the consumer that is consuming this partitoin).
- Group coordinator remembers the positions for those partitions which are interesting to some consumers of the given group.
- HeartbeatResponse includes the position for the requested partitons based on the most recent HeartbeatRequest from consumers of the group.
5) Changes in how consumer consumes partition
- Consumer receives SyncGroupResponse, which contains its assigned partitions
- Consumer gets the startPosition, i.e.the committedOffset, for its assigned partitions.
- For each partition whose startPosition is 0, consumer sends LeaderEpochForPartitionRequest to the coordinator to get the map (priorPartition -> oldLeaderEpoch), which can be read from the corresponding partition znode. Then the consumer sends OffsetForLeaderEpochRequest to convert the (priorPartition -> oldLeaderEpoch) to (priorPartition -> offsetThreshold), where offsetThreshold should be the last offset of messages published under oldLeaderEpoch for the given priorPartition.
- For each partiton whose startPosition is 0, consumer includes its list of priorPartition in the HeartbeatRequest and gets the corresponding position. Consumer only starts to consume this partition if for all its priorPartition, the position >= offsetThreshold of the priorParition.

Compatibility, Deprecation, and Migration Plan

Rejected Alternatives