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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -970,7 +973,11 @@ void maybeReconcile() {
                 log.debug("Auto-commit before reconciling new assignment 
completed successfully.");
             }
 
-            revokeAndAssign(assignedTopicIdPartitions, revokedPartitions, 
addedPartitions);
+            revokeAndAssign(resolvedAssignment, assignedTopicIdPartitions, 
revokedPartitions, addedPartitions);
+        }).whenComplete((__, error) -> {

Review Comment:
   nit: Could we use `exceptionally`?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManager.java:
##########
@@ -552,26 +553,23 @@ public ConsumerGroupHeartbeatRequestData 
buildRequestData() {
 
             // ServerAssignor - only sent if has changed since the last 
heartbeat
             this.membershipManager.serverAssignor().ifPresent(serverAssignor 
-> {
-                if (!serverAssignor.equals(sentFields.serverAssignor)) {
+                if (membershipManager.memberEpoch() == 0 || 
!serverAssignor.equals(sentFields.serverAssignor)) {
                     data.setServerAssignor(serverAssignor);
                     sentFields.serverAssignor = serverAssignor;
                 }
             });
 
             // 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.equals(sentFields.localAssignment)) {

Review Comment:
   Do we need `membershipManager.memberEpoch() == 0` here too? I suppose that 
it works because the current assignment is reset in the membership manager but 
it may be better to add it here for consistency.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImplTest.java:
##########
@@ -29,6 +29,8 @@
 import org.apache.kafka.common.TopicPartition;

Review Comment:
   I wonder if we could add the example that we discussed offline as a test:
   ```
   0: [T1, T2] -- T2 unresolved (only T1 is reconciled)
   1: [T1, T2, T3] -- T2 unresolved (skipped, since reconciliation in progress)
   2: [T1, T2] -- T2 unresolved
   ```
   What do you think?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManager.java:
##########
@@ -185,4 +190,89 @@ 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.
+     */
+    final class LocalAssignment {
+
+        public static final long NONE_EPOCH = -1;
+
+        public static final LocalAssignment NONE = new 
LocalAssignment(NONE_EPOCH, Collections.emptyMap());
+
+        public final long localEpoch;
+
+        public final Map<Uuid, SortedSet<Integer>> partitions;
+
+        public LocalAssignment(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 LocalAssignment(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());
+            });
+        }
+
+        public String toString() {
+            return "LocalAssignment{" +
+                "localEpoch=" + localEpoch +
+                ", partitions=" + partitions +
+                '}';
+        }
+
+        public boolean equals(final Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            final LocalAssignment that = (LocalAssignment) o;
+            return localEpoch == that.localEpoch && Objects.equals(partitions, 
that.partitions);
+        }
+
+        public int hashCode() {
+            return Objects.hash(localEpoch, partitions);
+        }
+
+        public boolean isNone() {
+            return localEpoch == NONE_EPOCH;
+        }
+
+        Optional<LocalAssignment> 
updateWith(ConsumerGroupHeartbeatResponseData.Assignment assignment) {
+

Review Comment:
   nit: Should we remove this empty line?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -889,42 +894,39 @@ 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);
+
+        if (!currentAssignment.isNone() &&
+            
resolvedAssignment.partitions.equals(currentAssignment.partitions)) {
+            log.debug("There are unresolved partitions, and the resolvable 
fragment of the  target assignment {} is equal to the current "

Review Comment:
   nit: There is an extra space before `target`.



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java:
##########
@@ -76,15 +83,14 @@
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 public class HeartbeatRequestManagerTest {
-    private long retryBackoffMs = DEFAULT_RETRY_BACKOFF_MS;
-    private int heartbeatIntervalMs = DEFAULT_HEARTBEAT_INTERVAL_MS;
-    private int maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL_MS;
-    private long retryBackoffMaxMs = DEFAULT_RETRY_BACKOFF_MAX_MS;
+    private final long retryBackoffMs = DEFAULT_RETRY_BACKOFF_MS;
+    private final int heartbeatIntervalMs = DEFAULT_HEARTBEAT_INTERVAL_MS;
+    private final int maxPollIntervalMs = DEFAULT_MAX_POLL_INTERVAL_MS;
+    private final long retryBackoffMaxMs = DEFAULT_RETRY_BACKOFF_MAX_MS;

Review Comment:
   nit: If there are final, could we remove them and directly use the constants 
where we need to?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -889,42 +894,39 @@ 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);
+
+        if (!currentAssignment.isNone() &&
+            
resolvedAssignment.partitions.equals(currentAssignment.partitions)) {

Review Comment:
   nit: Should we bring this one on the previous line?



##########
clients/src/test/java/org/apache/kafka/clients/consumer/internals/HeartbeatRequestManagerTest.java:
##########
@@ -311,19 +315,76 @@ public void 
testValidateConsumerGroupHeartbeatRequest(final short version) {
         NetworkClientDelegate.PollResult pollResult = 
heartbeatRequestManager.poll(time.milliseconds());
         assertEquals(1, pollResult.unsentRequests.size());
         NetworkClientDelegate.UnsentRequest request = 
pollResult.unsentRequests.get(0);
-        assertTrue(request.requestBuilder() instanceof 
ConsumerGroupHeartbeatRequest.Builder);
+        assertInstanceOf(Builder.class, request.requestBuilder());
 
         ConsumerGroupHeartbeatRequest heartbeatRequest =
             (ConsumerGroupHeartbeatRequest) 
request.requestBuilder().build(version);
         assertEquals(DEFAULT_GROUP_ID, heartbeatRequest.data().groupId());
         assertEquals(memberId, heartbeatRequest.data().memberId());
         assertEquals(memberEpoch, heartbeatRequest.data().memberEpoch());
-        assertEquals(DEFAULT_MAX_POLL_INTERVAL_MS, 
heartbeatRequest.data().rebalanceTimeoutMs());
+        assertEquals(10000, heartbeatRequest.data().rebalanceTimeoutMs());
         assertEquals(subscribedTopics, 
heartbeatRequest.data().subscribedTopicNames());
         assertEquals(DEFAULT_GROUP_INSTANCE_ID, 
heartbeatRequest.data().instanceId());
         assertEquals(DEFAULT_REMOTE_ASSIGNOR, 
heartbeatRequest.data().serverAssignor());
     }
 
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.CONSUMER_GROUP_HEARTBEAT)
+    public void 
testValidateConsumerGroupHeartbeatRequestAssignmentSentWhenLocalEpochChanges(final
 short version) {
+        CoordinatorRequestManager coordinatorRequestManager = 
mock(CoordinatorRequestManager.class);
+        MembershipManager membershipManager = mock(MembershipManager.class);
+        BackgroundEventHandler backgroundEventHandler = 
mock(BackgroundEventHandler.class);
+        SubscriptionState subscriptionState = mock(SubscriptionState.class);
+        HeartbeatRequestState requestState = mock(HeartbeatRequestState.class);
+        HeartbeatState heartbeatState = new HeartbeatState(subscriptionState, 
membershipManager, maxPollIntervalMs);
+
+        HeartbeatRequestManager heartbeatRequestManager = 
createHeartbeatRequestManager(
+            coordinatorRequestManager,
+            membershipManager,
+            heartbeatState,
+            requestState,
+            backgroundEventHandler
+        );
+
+        when(membershipManager.shouldHeartbeatNow()).thenReturn(true);
+        
when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(new 
Node(1, "localhost", 9999)));
+
+        Uuid topicId = Uuid.randomUuid();
+        ConsumerGroupHeartbeatRequestData.TopicPartitions 
expectedTopicPartitions =
+            new ConsumerGroupHeartbeatRequestData.TopicPartitions();
+        Map<Uuid, SortedSet<Integer>> testAssignment = 
Collections.singletonMap(
+            topicId, mkSortedSet(0)
+        );
+        expectedTopicPartitions.setTopicId(topicId);
+        expectedTopicPartitions.setPartitions(Collections.singletonList(0));
+
+        // First heartbeat, include assignment
+        when(membershipManager.currentAssignment()).thenReturn(new 
LocalAssignment(0, testAssignment));
+
+        ConsumerGroupHeartbeatRequest heartbeatRequest1 = 
getHeartbeatRequest(heartbeatRequestManager, version);
+        assertEquals(Collections.singletonList(expectedTopicPartitions), 
heartbeatRequest1.data().topicPartitions());
+
+        // Assignment did not change, so no assignment should be sent
+        ConsumerGroupHeartbeatRequest heartbeatRequest2 = 
getHeartbeatRequest(heartbeatRequestManager, version);
+        assertNull(heartbeatRequest2.data().topicPartitions());
+
+        // Local epoch bumped, so assignment should be sent
+        when(membershipManager.currentAssignment()).thenReturn(new 
LocalAssignment(1, testAssignment));
+
+        ConsumerGroupHeartbeatRequest heartbeatRequest3 = 
getHeartbeatRequest(heartbeatRequestManager, version);
+        assertEquals(Collections.singletonList(expectedTopicPartitions), 
heartbeatRequest3.data().topicPartitions());
+

Review Comment:
   nit: We could remove this empty line.



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