Versions Compared

Key

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

...

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 undeleted_partition_count of the topic.

- For each partition of this topic which does not have the partition znode, controller creates the partition znode, such that the leaderEpochAfterCreation field in the znode data maps partition of this topic to the corresponding leaderEpoch (recorded before controller increments the leaderEpoch)

- Controller propagates the UpdateMetadataRequest with the latest undeleted_partition_count per topic.

- Controller continue the existing logic of partition expansion.

Note that this procedure is fault tolerant. If controller fails in any of these step, the new controller can continue creating partition znode following the same procedure.


2) Changes in the controller for handling partition deletion




3) 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 in memory the undeleted_partition_count for each topic.

- When broker receives ProduceRequest, for each partition in the request, broker checks whether its undeleted_partition_count equals the undeleted_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 InvalidPartitionMetadataException. This error extends InvalidMetadaException and should trigger producer to update its metadata and retry.


34) Changes in how producer constructs ProduceRequest

- Producer should include the undeleted_partition_count for each topic in the ProduceRequest.

- Producer will update metadata and retry ProduceRequest if ProduceResponse shows InvalidPartitionMetadataException, which happens if producer's undeleted_partition_count is different (maybe newer or older) than the undeleted_partition_count in the broker.


45) Changes in how consumer handles HeartbeatRequest and HeartbeatResponsethe leader of the consumer group

Leader of the consumer group query the metadata to split the partition list to those partitions that have not been marked for deletion and those partitions that have been marked for deletion. It should apply the user-defined assignment algorithm to these two lists seperately to determine the partition distribution across consumers in the group, so that partitions which have not been marked for deletion can also be evenly distributed across consumers. This is to prevent load imbalance across consumers because there will be no new data to those partitions which have been marked for deletion.


6) Changes in how consumer handles HeartbeatRequest and HeartbeatResponse

- HeartbeatRequest includes the current position for each partition requested by the coordinator from the previous HeartbeatResponse. It also includes the list of partitions for which it - HeartbeatRequest includes the current position for each partition requested by the coordinator from the previous HeartbeatResponse. 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. It also includes the list of partitions which are interesting to some consumers of the given group.


57) Changes in how consumer consumes partition

1. Consumer receives SyncGroupResponse, which contains its assigned partitions

2. Consumer gets the startPosition, i.e.the committedOffset, for its assigned partitions.

3. Consumer sends ListOffsetRequest to get the earliest offset for its assigned partitions.

4. For each partition P1 whose startPosition is not available, or whose startPosition equals the earliest offset, consumer does the following before consuming the partition P1:

    4.1 Consumer sends PartitionLeaderEpochsForPartitionsRequest to the coordinator to get the leaderEpochAfterCreation map for the partition P1, which can be read by broker from the corresponding partition znode. Then the consumer sends OffsetsForLeaderEpochRequest to convert the leaderEpochAfterCreation from (priorPartition -> oldLeaderEpoch) to (priorPartition -> lowerOffsetThreshold), where lowerOffsetThreshold should be the last offset of messages published under the oldLeaderEpoch for the given priorPartition.

    4.2 Consumer includes the keys (i.e. partitions) of the leaderEpochAfterCreation map in the HeartbeatRequest and gets the corresponding position of these partitions of the consumer group in the HeartbeatResponse. Consumer only starts to consume partition P1 if for all its priorPartition, the position >= lowerOffsetThreshold of the priorParition.

5. For each parrtition P1 assigned to this consumer, consumer queries the metadata to see if there any partition of this topic has been marked for deletion. If so, consumer does the following before deliverying a message with offset T from this partition P1:

    5.1 Consumer sends PartitionLeaderEpochsForPartitionsRequest to coordinator to get the leaderEpochBeforeDeletion map for all partitions of this topic. Note that a partition is marked for deletion if and only if the leaderEpochBeforeDeletion map in its partition znode is not empty. For each partition P2 whose leaderEpochBeforeDeletion includes the partition P1, consumer then sends OffsetsForLeaderEpochRequest to convert the leaderEpoch (i.e. P2.leaderEpochBeforeDeletion[P1]) to upperOffsetThreshold. This results in a map from P2 -> upperOffsetThreshold, where P2 represents all those partitions whose leaderEpochBeforeDeletion includes the partition P1.

    5.2 For all partitions whose leaderEpochBeforeDeletion includes the partition P1, consumer includes these partitions in the HeartbeatRequest and gets the corresponding position of these partitions of the consumer group in the HeartbeatResponse. Consumer also sends ListOffsetRequest to get the LogEndOffset for these partitons.

    5.3 Consumer only consumers message with offset T from partition P1, if for every partition P2 whose leaderEpochBeforeDeletion includes the partition P1, either the position of P2 of the consumer group has reached the LogEndOffset of P2, or the offset T <= upperOffsetThreshold.



...