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