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


##########
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:
   While we are here, we may be able to do the same for `serverAssignor` field 
as it never changes during the runtime.



##########
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:
   nit: We usually don't prefix getters with `get`.



##########
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:
   nit: We could use `isNone`.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -988,7 +989,8 @@ long getExpirationTimeForTimeout(final long timeoutMs) {
      * then complete the reconciliation by updating the assignment and making 
the appropriate state
      * transition. Note that if any of the 2 callbacks fails, the 
reconciliation should fail.
      */
-    private void revokeAndAssign(SortedSet<TopicIdPartition> 
assignedTopicIdPartitions,
+    private void revokeAndAssign(LocalAssignmentImpl resolvedAssignment,

Review Comment:
   btw, I just noticed the `boolean memberHasRejoined = 
memberEpochOnReconciliationStart != memberEpoch` condition while reading the 
code again. We have it in two places. I think that this is wrong because the 
member epoch could effectively change without leaving. @lianetm I recall that 
we discussed this a while ago. Do we have a jira to address this?



##########
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 wonder if we could also add a test for the case described in KAFKA-16185. 
Is it possible?



##########
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:
   Should we add a unit test here to validate that the same assignment but with 
a different epoch is sent out?



##########
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:
   nit: Should we add `LocalAssignmentImpl` too?



##########
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:
   nit: I wonder if we could use `equals` here. Would it work?



##########
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:
   This comments is a bit weird because the interface does not have an epoch. 
Perhaps, the interface is a bit overkill here.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -889,43 +914,36 @@ private void transitionToStale() {
      */
     void maybeReconcile() {
         if (targetAssignmentReconciled()) {
-            log.debug("Ignoring reconciliation attempt. Target assignment is 
equal to the " +
+            log.trace("Ignoring reconciliation attempt. Target assignment is 
equal to the " +
                     "current assignment.");
             return;
         }
         if (reconciliationInProgress) {
-            log.debug("Ignoring reconciliation attempt. Another reconciliation 
is already in progress. Assignment " +
+            log.trace("Ignoring reconciliation attempt. Another reconciliation 
is already in progress. Assignment " +
                 currentTargetAssignment + " will be handled in the next 
reconciliation loop.");
             return;
         }
 
         // Find the subset of the target assignment that can be resolved to 
topic names, and trigger a metadata update
         // if some topic IDs are not resolvable.
         SortedSet<TopicIdPartition> assignedTopicIdPartitions = 
findResolvableAssignmentAndTriggerMetadataUpdate();
+        final LocalAssignment resolvedAssignment = new 
LocalAssignment(currentTargetAssignment.localEpoch, assignedTopicIdPartitions);
 
-        SortedSet<TopicPartition> ownedPartitions = new 
TreeSet<>(TOPIC_PARTITION_COMPARATOR);
-        ownedPartitions.addAll(subscriptions.assignedPartitions());
-
-        // Keep copy of assigned TopicPartitions created from the 
TopicIdPartitions that are
-        // being reconciled. Needed for interactions with the centralized 
subscription state that
-        // does not support topic IDs yet, and for the callbacks.
-        SortedSet<TopicPartition> assignedTopicPartitions = 
toTopicPartitionSet(assignedTopicIdPartitions);
-
-        // Check same assignment. Based on topic names for now, until topic 
IDs are properly
-        // supported in the centralized subscription state object. Note that 
this check is
-        // required to make sure that reconciliation is not triggered if the 
assignment ready to
-        // be reconciled is the same as the current one (even though the 
member may remain
-        // in RECONCILING state if it has some unresolved assignments).
-        boolean sameAssignmentReceived = 
assignedTopicPartitions.equals(ownedPartitions);
-
-        if (sameAssignmentReceived) {
+        if (resolvedAssignment.equals(currentAssignment)) {

Review Comment:
   So, my understanding is the following:
   1. Start with [A] at local epoch 10;
   2. Go to [A, B] at local epoch 11;
   3. B is not resolvable yet so we end up with [A] at local epoch 11 to 
reconcile;
   4. [A] is effectively the current assignment so we would trigger the 
callback with [];
   5. When B is finally available, we get [A, B] at local epoch 11 and 
reconcile. 
   
   @lianetm I think that you're saying that step 4. is not needed, right? My 
concern with your suggestion is that it defeats a bit the purpose of the patch. 
However, I do agree with you that in this particular case, we don't have to 
reconcile at all. Perhaps, a better way to phrase this would be to no trigger 
the reconciliation if all the newly partitions are waiting on the metadata and 
there is no revocation. Would something like this work?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -530,19 +530,12 @@ 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);

Review Comment:
   This has already be done in a separate PR: 
https://github.com/apache/kafka/commit/d88a97adef684a0f5403c46f7fb2f8d1723eebd5.



-- 
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