You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/12/05 17:15:22 UTC

[GitHub] [kafka] dajac commented on a diff in pull request #12954: KAFKA-14352: Rack-aware consumer partition assignment protocol changes (KIP-881)

dajac commented on code in PR #12954:
URL: https://github.com/apache/kafka/pull/12954#discussion_r1039862483


##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java:
##########
@@ -56,7 +56,7 @@ public void serializeDeserializeSubscriptionAllVersions() {
             new TopicPartition("foo", 0),
             new TopicPartition("bar", 0));
         Subscription subscription = new Subscription(Arrays.asList("foo", "bar"),
-            ByteBuffer.wrap("hello".getBytes()), ownedPartitions, generationId);
+            ByteBuffer.wrap("hello".getBytes()), ownedPartitions, generationId, Optional.empty());

Review Comment:
   It would be great if we could extend this test and possibly others in the file to cover the protocol changes.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java:
##########
@@ -47,23 +51,37 @@ public abstract class AbstractPartitionAssignor implements ConsumerPartitionAssi
     public abstract Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
                                                              Map<String, Subscription> subscriptions);
 
+    /**
+     * Default implementation of assignPartitions() that does not include racks. This is only
+     * included to avoid breaking any custom implementation that extends AbstractPartitionAssignor.
+     * Note that this class is internal, but to be safe, we are maintaining compatibility.
+     */
+    public Map<String, List<TopicPartition>> assignPartitions(Map<String, List<PartitionInfo>> partitionsPerTopic,

Review Comment:
   Do we need changes in the file for the patch? They do not seem to be related.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -245,7 +248,8 @@ protected JoinGroupRequestData.JoinGroupRequestProtocolCollection metadata() {
             Subscription subscription = new Subscription(topics,
                                                          assignor.subscriptionUserData(joinedSubscription),
                                                          subscriptions.assignedPartitionsList(),
-                                                         generation().generationId);
+                                                         generation().generationId,
+                                                         rackId);

Review Comment:
   Is it possible to add a unit test which verifies that `rackId` is correctly sent out in the `JoinGroup` request? I am not sure how feasible is this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org