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