lucasbru commented on code in PR #15511:
URL: https://github.com/apache/kafka/pull/15511#discussion_r1524575611


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -1497,4 +1502,93 @@ public PollResult poll(final long currentTimeMs) {
     List<MemberStateListener> stateListeners() {
         return unmodifiableList(stateUpdatesListeners);
     }
+
+    private final static class LocalAssignmentImpl implements LocalAssignment {
+
+        private static final long NONE_EPOCH = -1;
+
+        private static final LocalAssignmentImpl NONE = new 
LocalAssignmentImpl(NONE_EPOCH, Collections.emptyMap());
+
+        private final long localEpoch;
+
+        private final Map<Uuid, SortedSet<Integer>> partitions;
+
+        public LocalAssignmentImpl(long localEpoch, Map<Uuid, 
SortedSet<Integer>> partitions) {
+            this.localEpoch = localEpoch;
+            this.partitions = partitions;
+            if (localEpoch == NONE_EPOCH && !partitions.isEmpty()) {
+                throw new IllegalArgumentException("Local epoch must be set if 
there are partitions");
+            }
+        }
+
+        public LocalAssignmentImpl(long localEpoch, 
SortedSet<TopicIdPartition> topicIdPartitions) {
+            this.localEpoch = localEpoch;
+            this.partitions = new HashMap<>();
+            if (localEpoch == NONE_EPOCH && !topicIdPartitions.isEmpty()) {
+                throw new IllegalArgumentException("Local epoch must be set if 
there are partitions");
+            }
+            topicIdPartitions.forEach(topicIdPartition -> {
+                Uuid topicId = topicIdPartition.topicId();
+                partitions.computeIfAbsent(topicId, k -> new 
TreeSet<>()).add(topicIdPartition.partition());
+            });
+        }
+
+        @Override
+        public String toString() {
+            return "{" +
+                "localEpoch=" + localEpoch +
+                ", partitions=" + partitions +
+                '}';
+        }
+
+        @Override
+        public boolean equals(final Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            final LocalAssignmentImpl that = (LocalAssignmentImpl) o;
+            return localEpoch == that.localEpoch && Objects.equals(partitions, 
that.partitions);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(localEpoch, partitions);
+        }
+
+        @Override
+        public Map<Uuid, SortedSet<Integer>> getPartitions() {
+            return partitions;
+        }
+
+        @Override
+        public boolean isNone() {
+            return localEpoch == NONE_EPOCH;
+        }
+
+        Optional<LocalAssignmentImpl> 
updateWith(ConsumerGroupHeartbeatResponseData.Assignment assignment) {
+
+            // Return if we have an assignment, and it is the same as current 
assignment; comparison without creating a new collection
+            if (localEpoch != NONE_EPOCH) {
+                // check if the new assignment is different from the current 
target assignment

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -529,25 +530,18 @@ public ConsumerGroupHeartbeatRequestData 
buildRequestData() {
             // MemberEpoch - always sent
             data.setMemberEpoch(membershipManager.memberEpoch());
 
-            // InstanceId - only sent if has changed since the last heartbeat
-            // Always send when leaving the group as a static member
-            membershipManager.groupInstanceId().ifPresent(groupInstanceId -> {
-                if (!groupInstanceId.equals(sentFields.instanceId) || 
membershipManager.memberEpoch() == 
ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH) {
-                    data.setInstanceId(groupInstanceId);
-                    sentFields.instanceId = groupInstanceId;
-                }
-            });
+            // InstanceId - always send when leaving the group as a static 
member
+            membershipManager.groupInstanceId().ifPresent(data::setInstanceId);
 
-            // RebalanceTimeoutMs - only sent if has changed since the last 
heartbeat
-            if (sentFields.rebalanceTimeoutMs != rebalanceTimeoutMs) {
+            // RebalanceTimeoutMs - only sent when joining
+            if (membershipManager.memberEpoch() == 0) {

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##########
@@ -1952,19 +1942,22 @@ private void 
assertStaleMemberLeavesGroupAndClearsAssignment(MembershipManagerIm
         // Should reset epoch to leave the group and release the assignment 
(right away because
         // there is no onPartitionsLost callback defined)
         verify(subscriptionState).assignFromSubscribed(Collections.emptySet());
-        assertTrue(membershipManager.currentAssignment().isEmpty());
+        assertTrue(membershipManager.currentAssignment().isNone());
         assertTrue(membershipManager.topicsAwaitingReconciliation().isEmpty());
         assertEquals(LEAVE_GROUP_MEMBER_EPOCH, 
membershipManager.memberEpoch());
     }
 
     @Test
     public void 
testMemberJoiningTransitionsToStableWhenReceivingEmptyAssignment() {

Review Comment:
   I added the two tests.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -529,25 +530,18 @@ public ConsumerGroupHeartbeatRequestData 
buildRequestData() {
             // MemberEpoch - always sent
             data.setMemberEpoch(membershipManager.memberEpoch());
 
-            // InstanceId - only sent if has changed since the last heartbeat
-            // Always send when leaving the group as a static member
-            membershipManager.groupInstanceId().ifPresent(groupInstanceId -> {
-                if (!groupInstanceId.equals(sentFields.instanceId) || 
membershipManager.memberEpoch() == 
ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH) {
-                    data.setInstanceId(groupInstanceId);
-                    sentFields.instanceId = groupInstanceId;
-                }
-            });
+            // InstanceId - always send when leaving the group as a static 
member
+            membershipManager.groupInstanceId().ifPresent(data::setInstanceId);
 
-            // RebalanceTimeoutMs - only sent if has changed since the last 
heartbeat
-            if (sentFields.rebalanceTimeoutMs != rebalanceTimeoutMs) {
+            // RebalanceTimeoutMs - only sent when joining
+            if (membershipManager.memberEpoch() == 0) {
                 data.setRebalanceTimeoutMs(rebalanceTimeoutMs);
-                sentFields.rebalanceTimeoutMs = rebalanceTimeoutMs;
             }
 
             if (!this.subscriptions.hasPatternSubscription()) {
-                // SubscribedTopicNames - only sent if has changed since the 
last heartbeat
+                // SubscribedTopicNames - only sent when joining or if has 
changed since the last heartbeat

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -566,18 +560,18 @@ public ConsumerGroupHeartbeatRequestData 
buildRequestData() {
 
             // ClientAssignors - not supported yet
 
-            // TopicPartitions - only sent if it has changed since the last 
heartbeat. Note that
-            // the string consists of just the topic ID and the partitions. 
When an assignment is
-            // received, we might not yet know the topic name, and then it is 
learnt subsequently
-            // by a metadata update.
-            TreeSet<String> assignedPartitions = 
membershipManager.currentAssignment().entrySet().stream()
-                .map(entry -> entry.getKey() + "-" + entry.getValue())
-                .collect(Collectors.toCollection(TreeSet::new));
-            if (!assignedPartitions.equals(sentFields.topicPartitions)) {
+            // TopicPartitions - sent with the first heartbeat after a new 
assignment from the server was
+            // reconciled. This is ensured by resending the topic partitions 
whenever the local assignment,
+            // including its local epoch is changed (although the local epoch 
is not sent in the heartbeat).
+            LocalAssignment local = membershipManager.currentAssignment();
+            if (local == null) {

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java:
##########
@@ -185,4 +185,18 @@ public interface MembershipManager extends RequestManager {
      * releasing its assignment. This is expected to be used when the poll 
timer is reset.
      */
     void maybeRejoinStaleMember();
+
+    /**
+     * A data structure to represent the current assignment, and current 
target assignment of a member in a consumer group.
+     *
+     * Besides the assigned partitions, it contains a local epoch that is 
bumped whenever the assignment changes, to ensure
+     * that two assignments with the same partitions but different local 
epochs are not considered equal.
+     */
+    interface LocalAssignment {
+
+        Map<Uuid, SortedSet<Integer>> getPartitions();

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -1497,4 +1502,93 @@ public PollResult poll(final long currentTimeMs) {
     List<MemberStateListener> stateListeners() {
         return unmodifiableList(stateUpdatesListeners);
     }
+
+    private final static class LocalAssignmentImpl implements LocalAssignment {
+
+        private static final long NONE_EPOCH = -1;
+
+        private static final LocalAssignmentImpl NONE = new 
LocalAssignmentImpl(NONE_EPOCH, Collections.emptyMap());
+
+        private final long localEpoch;
+
+        private final Map<Uuid, SortedSet<Integer>> partitions;
+
+        public LocalAssignmentImpl(long localEpoch, Map<Uuid, 
SortedSet<Integer>> partitions) {
+            this.localEpoch = localEpoch;
+            this.partitions = partitions;
+            if (localEpoch == NONE_EPOCH && !partitions.isEmpty()) {
+                throw new IllegalArgumentException("Local epoch must be set if 
there are partitions");
+            }
+        }
+
+        public LocalAssignmentImpl(long localEpoch, 
SortedSet<TopicIdPartition> topicIdPartitions) {
+            this.localEpoch = localEpoch;
+            this.partitions = new HashMap<>();
+            if (localEpoch == NONE_EPOCH && !topicIdPartitions.isEmpty()) {
+                throw new IllegalArgumentException("Local epoch must be set if 
there are partitions");
+            }
+            topicIdPartitions.forEach(topicIdPartition -> {
+                Uuid topicId = topicIdPartition.topicId();
+                partitions.computeIfAbsent(topicId, k -> new 
TreeSet<>()).add(topicIdPartition.partition());
+            });
+        }
+
+        @Override
+        public String toString() {
+            return "{" +
+                "localEpoch=" + localEpoch +
+                ", partitions=" + partitions +
+                '}';
+        }
+
+        @Override
+        public boolean equals(final Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            final LocalAssignmentImpl that = (LocalAssignmentImpl) o;
+            return localEpoch == that.localEpoch && Objects.equals(partitions, 
that.partitions);
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(localEpoch, partitions);
+        }
+
+        @Override
+        public Map<Uuid, SortedSet<Integer>> getPartitions() {
+            return partitions;
+        }
+
+        @Override
+        public boolean isNone() {
+            return localEpoch == NONE_EPOCH;
+        }
+
+        Optional<LocalAssignmentImpl> 
updateWith(ConsumerGroupHeartbeatResponseData.Assignment assignment) {
+
+            // Return if we have an assignment, and it is the same as current 
assignment; comparison without creating a new collection
+            if (localEpoch != NONE_EPOCH) {
+                // check if the new assignment is different from the current 
target assignment
+                if (partitions.size() == assignment.topicPartitions().size() &&
+                    assignment.topicPartitions().stream().allMatch(
+                        tp -> partitions.containsKey(tp.topicId()) &&
+                            partitions.get(tp.topicId()).size() == 
tp.partitions().size() &&
+                            
partitions.get(tp.topicId()).containsAll(tp.partitions()))) {

Review Comment:
   No, that's comparing a list and a set and wouldn't work



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java:
##########
@@ -27,6 +27,7 @@
 import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
 import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData;
+import 
org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.Assignment;

Review Comment:
   Done



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##########
@@ -683,7 +681,10 @@ public void testDelayedMetadataUsedToCompleteAssignment() {
         receiveAssignment(newAssignment, membershipManager);
         membershipManager.poll(time.milliseconds());
 
-        verifyReconciliationNotTriggered(membershipManager);
+        // We bumped the local epoch, so new reconciliation is triggered

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java:
##########
@@ -185,4 +185,18 @@ public interface MembershipManager extends RequestManager {
      * releasing its assignment. This is expected to be used when the poll 
timer is reset.
      */
     void maybeRejoinStaleMember();
+
+    /**
+     * A data structure to represent the current assignment, and current 
target assignment of a member in a consumer group.
+     *
+     * Besides the assigned partitions, it contains a local epoch that is 
bumped whenever the assignment changes, to ensure
+     * that two assignments with the same partitions but different local 
epochs are not considered equal.

Review Comment:
   Done



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -1497,4 +1502,93 @@ public PollResult poll(final long currentTimeMs) {
     List<MemberStateListener> stateListeners() {
         return unmodifiableList(stateUpdatesListeners);
     }
+
+    private final static class LocalAssignmentImpl implements LocalAssignment {
+
+        private static final long NONE_EPOCH = -1;
+
+        private static final LocalAssignmentImpl NONE = new 
LocalAssignmentImpl(NONE_EPOCH, Collections.emptyMap());
+
+        private final long localEpoch;
+
+        private final Map<Uuid, SortedSet<Integer>> partitions;
+
+        public LocalAssignmentImpl(long localEpoch, Map<Uuid, 
SortedSet<Integer>> partitions) {
+            this.localEpoch = localEpoch;
+            this.partitions = partitions;
+            if (localEpoch == NONE_EPOCH && !partitions.isEmpty()) {
+                throw new IllegalArgumentException("Local epoch must be set if 
there are partitions");
+            }
+        }
+
+        public LocalAssignmentImpl(long localEpoch, 
SortedSet<TopicIdPartition> topicIdPartitions) {
+            this.localEpoch = localEpoch;
+            this.partitions = new HashMap<>();
+            if (localEpoch == NONE_EPOCH && !topicIdPartitions.isEmpty()) {
+                throw new IllegalArgumentException("Local epoch must be set if 
there are partitions");
+            }
+            topicIdPartitions.forEach(topicIdPartition -> {
+                Uuid topicId = topicIdPartition.topicId();
+                partitions.computeIfAbsent(topicId, k -> new 
TreeSet<>()).add(topicIdPartition.partition());
+            });
+        }
+
+        @Override
+        public String toString() {
+            return "{" +

Review Comment:
   Done



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to