From 7d9d82149968ba7bb5e9a967321c12fd29e64389 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Fri, 14 Jul 2023 10:29:37 -0400 Subject: [PATCH 01/11] Implement SyncGroup in new Coordinator --- .../group/GroupCoordinatorService.java | 27 +- .../group/GroupMetadataManager.java | 226 +- .../group/ReplicatedGroupCoordinator.java | 14 + .../group/generic/GenericGroup.java | 24 +- .../group/generic/GenericGroupMember.java | 7 +- .../group/GroupMetadataManagerTest.java | 3058 ++++++++++++++++- 6 files changed, 3314 insertions(+), 42 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 571ea6e2b9a11..0dae52f5e4d72 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -330,9 +330,30 @@ public CompletableFuture syncGroup( return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception()); } - return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception( - "This API is not implemented yet." - )); + if (!isGroupIdNotEmpty(request.groupId())) { + return CompletableFuture.completedFuture(new SyncGroupResponseData() + .setErrorCode(Errors.INVALID_GROUP_ID.code())); + } + + CompletableFuture responseFuture = new CompletableFuture<>(); + + runtime.scheduleWriteOperation("generic-group-sync", + topicPartitionFor(request.groupId()), + coordinator -> coordinator.genericGroupSync(context, request, responseFuture) + ).exceptionally(exception -> { + if (!(exception instanceof KafkaException)) { + log.error("Request {} hit an unexpected exception: {}", + request, exception.getMessage()); + } + + if (!responseFuture.isDone()) { + responseFuture.complete(new SyncGroupResponseData() + .setErrorCode(Errors.forException(exception).code())); + } + return null; + }); + + return responseFuture; } /** diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index da5010eac0313..cd5d159afb11f 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; +import org.apache.kafka.common.message.SyncGroupRequestData; import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.JoinGroupRequest; @@ -75,6 +76,7 @@ import org.slf4j.Logger; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -1491,8 +1493,9 @@ public void replay( /** * Handle a JoinGroupRequest. * - * @param context The request context. - * @param request The actual JoinGroup request. + * @param context The request context. + * @param request The actual JoinGroup request. + * @param responseFuture The join group response future. * * @return The result that contains records to append if the join group phase completes. */ @@ -1619,7 +1622,7 @@ private CoordinatorResult genericGroupJoinNewMember( // finding the correct coordinator and rejoin. responseFuture.complete(new JoinGroupResponseData() .setMemberId(UNKNOWN_MEMBER_ID) - .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setErrorCode(COORDINATOR_NOT_AVAILABLE.code()) ); } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) { responseFuture.complete(new JoinGroupResponseData() @@ -1678,7 +1681,7 @@ private CoordinatorResult genericGroupJoinNewStaticMember( "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.", groupInstanceId, group.groupId(), group.currentState(), existingMemberId); - return updateStaticMemberAndRebalance( + return updateStaticMemberThenRebalanceOrCompleteJoin( context, request, group, @@ -1691,7 +1694,7 @@ private CoordinatorResult genericGroupJoinNewStaticMember( "group {} in {} state. Created a new member id {} for this member and added to the group.", groupInstanceId, group.groupId(), group.currentState(), newMemberId); - return addMemberAndRebalance(context, request, group, newMemberId, responseFuture); + return addMemberThenRebalanceOrCompleteJoin(context, request, group, newMemberId, responseFuture); } } @@ -1744,7 +1747,7 @@ private CoordinatorResult genericGroupJoinNewDynamicMember( "Created a new member id {} and added the member to the group.", group.groupId(), group.currentState(), newMemberId); - return addMemberAndRebalance(context, request, group, newMemberId, responseFuture); + return addMemberThenRebalanceOrCompleteJoin(context, request, group, newMemberId, responseFuture); } return EMPTY_RESULT; @@ -1776,7 +1779,7 @@ private CoordinatorResult genericGroupJoinExistingMember( // finding the correct coordinator and rejoin. responseFuture.complete(new JoinGroupResponseData() .setMemberId(memberId) - .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setErrorCode(COORDINATOR_NOT_AVAILABLE.code()) ); } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) { responseFuture.complete(new JoinGroupResponseData() @@ -1793,7 +1796,7 @@ private CoordinatorResult genericGroupJoinExistingMember( log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.", memberId, group.groupId(), group.currentState()); - return addMemberAndRebalance( + return addMemberThenRebalanceOrCompleteJoin( context, request, group, @@ -1812,6 +1815,8 @@ private CoordinatorResult genericGroupJoinExistingMember( responseFuture.complete(new JoinGroupResponseData() .setMemberId(memberId) .setErrorCode(memberError.get().code()) + .setProtocolType(null) + .setProtocolName(null) ); } else { GenericGroupMember member = group.member(memberId); @@ -2015,6 +2020,7 @@ private CoordinatorResult completeGenericGroupJoin( * @param group The group. */ private void schedulePendingSync(GenericGroup group) { + group.setPendingSyncGenerationId(group.generationId()); timer.schedule(genericGroupSyncKey(group.groupId()), group.rebalanceTimeoutMs(), TimeUnit.MILLISECONDS, @@ -2188,7 +2194,7 @@ private Optional validateExistingMember( } /** - * Add a member and rebalance. + * Add a member then rebalance or complete join. * * @param context The request context. * @param request The join group request. @@ -2198,7 +2204,7 @@ private Optional validateExistingMember( * * @return The coordinator result that will be appended to the log. */ - private CoordinatorResult addMemberAndRebalance( + private CoordinatorResult addMemberThenRebalanceOrCompleteJoin( RequestContext context, JoinGroupRequestData request, GenericGroup group, @@ -2536,7 +2542,7 @@ private boolean acceptJoiningMember(GenericGroup group, String memberId) { } /** - * Update a static member and rebalance. + * Update a static member then rebalance or complete join. * * @param context The request context. * @param request The join group request. @@ -2547,7 +2553,7 @@ private boolean acceptJoiningMember(GenericGroup group, String memberId) { * * @return The coordinator result that will be appended to the log. */ - private CoordinatorResult updateStaticMemberAndRebalance( + private CoordinatorResult updateStaticMemberThenRebalanceOrCompleteJoin( RequestContext context, JoinGroupRequestData request, GenericGroup group, @@ -2660,6 +2666,116 @@ private CoordinatorResult updateStaticMemberAndRebalance( group.stateAsString() + " when the unknown static member " + request.groupInstanceId() + " rejoins."); } + return maybeCompleteJoinPhase(group); + } + + /** + * Handle a SyncGroupRequest. + * + * @param context The request context. + * @param request The actual SyncGroup request. + * @param responseFuture The sync group response future. + * + * @return The result that contains records to append if the group metadata manager received assignments. + */ + public CoordinatorResult genericGroupSync( + RequestContext context, + SyncGroupRequestData request, + CompletableFuture responseFuture + ) throws UnknownMemberIdException, GroupIdNotFoundException { + String groupId = request.groupId(); + String memberId = request.memberId(); + GenericGroup group; + try { + group = getOrMaybeCreateGenericGroup(groupId, false); + } catch (Throwable t) { + responseFuture.complete(new SyncGroupResponseData() + .setErrorCode(Errors.forException(t).code()) + ); + return EMPTY_RESULT; + } + + Optional errorOpt = validateSyncGroup(group, request); + if (errorOpt.isPresent()) { + responseFuture.complete(new SyncGroupResponseData() + .setErrorCode(errorOpt.get().code())); + } else if (group.isInState(EMPTY)) { + responseFuture.complete(new SyncGroupResponseData() + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())); + } else if (group.isInState(PREPARING_REBALANCE)) { + responseFuture.complete(new SyncGroupResponseData() + .setErrorCode(Errors.REBALANCE_IN_PROGRESS.code())); + } else if (group.isInState(COMPLETING_REBALANCE)) { + group.member(memberId).setAwaitingSyncFuture(responseFuture); + removePendingSyncMember(group, request.memberId()); + + // If this is the leader, then we can attempt to persist state and transition to stable + if (group.isLeader(memberId)) { + log.info("Assignment received from leader {} for group {} for generation {}. " + + "The group has {} members, {} of which are static.", + memberId, groupId, group.generationId(), group.size(), group.allStaticMemberIds().size()); + + // Fill all members with corresponding assignment. Reset members not specified in + // the assignment to empty assignments. + Map assignments = new HashMap<>(); + request.assignments().forEach(assignment -> + assignments.put(assignment.memberId(), assignment.assignment()) + ); + + Set membersWithMissingAssignment = new HashSet<>(); + group.allMembers().forEach(member -> { + byte[] assignment = assignments.get(member.memberId()); + if (assignment != null) { + member.setAssignment(assignment); + } else { + membersWithMissingAssignment.add(member.memberId()); + member.setAssignment(new byte[0]); + } + }); + + if (!membersWithMissingAssignment.isEmpty()) { + log.warn("Setting empty assignments for members {} of {} for generation {}.", + membersWithMissingAssignment, groupId, group.generationId()); + } + + CompletableFuture appendFuture = new CompletableFuture<>(); + appendFuture.whenComplete((__, t) -> { + // Another member may have joined the group while we were awaiting this callback, + // so we must ensure we are still in the CompletingRebalance state and the same generation + // when it gets invoked. if we have transitioned to another state, then do nothing + if (group.isInState(COMPLETING_REBALANCE) && request.generationId() == group.generationId()) { + if (t != null) { + Errors error = Errors.forException(t); + resetAndPropagateAssignmentWithError(group, error); + maybePrepareRebalanceOrCompleteJoin(group, "Error " + error + " when storing group assignment" + + "during SyncGroup (member: " + memberId + ")."); + } else { + // Members' assignments were already updated. Propagate and transition to Stable. + propagateAssignment(group, Errors.NONE); + group.transitionTo(STABLE); + } + } + }); + + List records = Collections.singletonList( + RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion()) + ); + return new CoordinatorResult<>(records, appendFuture); + } + } else if (group.isInState(STABLE)) { + removePendingSyncMember(group, memberId); + + // If the group is stable, we just return the current assignment + GenericGroupMember member = group.member(memberId); + responseFuture.complete(new SyncGroupResponseData() + .setProtocolType(group.protocolType().orElse(null)) + .setProtocolName(group.protocolName().orElse(null)) + .setAssignment(member.assignment()) + .setErrorCode(Errors.NONE.code())); + } else if (group.isInState(DEAD)) { + throw new IllegalStateException("Reached unexpected condition for Dead group " + groupId); + } + return EMPTY_RESULT; } @@ -2684,8 +2800,88 @@ static Errors appendGroupMetadataErrorToResponseError(Errors appendError) { } } + private Optional validateSyncGroup( + GenericGroup group, + SyncGroupRequestData request + ) { + if (group.isInState(DEAD)) { + // If the group is marked as dead, it means some other thread has just removed the group + // from the coordinator metadata; this is likely that the group has migrated to some other + // coordinator OR the group is in a transient unstable phase. Let the member retry + // finding the correct coordinator and rejoin. + return Optional.of(COORDINATOR_NOT_AVAILABLE); + } else { + Optional memberError = validateExistingMember( + group, + request.memberId(), + request.groupInstanceId(), + "sync-group" + ); + if (memberError.isPresent()) { + return memberError; + } else { + if (request.generationId() != group.generationId()) { + return Optional.of(Errors.ILLEGAL_GENERATION); + + } else if (isProtocolInconsistent(request.protocolType(), group.protocolType().orElse(null)) || + isProtocolInconsistent(request.protocolName(), group.protocolName().orElse(null))) { + + return Optional.of(Errors.INCONSISTENT_GROUP_PROTOCOL); + } else { + return Optional.empty(); + } + } + } + } + + private void removePendingSyncMember( + GenericGroup group, + String memberId + ) { + group.removePendingSyncMember(memberId); + + String syncKey = genericGroupSyncKey(group.groupId()); + + if (group.generationId() != group.pendingSyncGenerationId()) { + timer.cancel(syncKey); + } else { + switch (group.currentState()) { + case DEAD: + case EMPTY: + case PREPARING_REBALANCE: + timer.cancel(syncKey); + break; + case COMPLETING_REBALANCE: + case STABLE: + if (group.hasReceivedSyncFromAllMembers()) { + timer.cancel(syncKey); + } + break; + default: + throw new IllegalStateException("Unknown group state: " + group.stateAsString()); + } + } + } + + /** + * Checks whether the given protocol type or name in the request is inconsistent with the group's. + * + * @param protocolTypeOrName The request's protocol type or name. + * @param groupProtocolTypeOrName The group's protoocl type or name. + * + * @return True if protocol is inconsistent, false otherwise. + */ + private boolean isProtocolInconsistent( + String protocolTypeOrName, + String groupProtocolTypeOrName + ) { + return protocolTypeOrName != null + && groupProtocolTypeOrName != null + && !groupProtocolTypeOrName.equals(protocolTypeOrName); + } + /** - * Generate a heartbeat key for the timer. + * Generate a generic group heartbeat key for the timer. * * Package private for testing. * @@ -2699,7 +2895,7 @@ static String genericGroupHeartbeatKey(String groupId, String memberId) { } /** - * Generate a join key for the timer. + * Generate a generic group join key for the timer. * * Package private for testing. * @@ -2712,7 +2908,7 @@ static String genericGroupJoinKey(String groupId) { } /** - * Generate a sync key for the timer. + * Generate a generic group sync key for the timer. * * Package private for testing. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java index 854bed1060b2e..7c3a71056971a 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java @@ -21,6 +21,8 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; +import org.apache.kafka.common.message.SyncGroupRequestData; +import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.utils.LogContext; @@ -193,6 +195,18 @@ public CoordinatorResult genericGroupJoin( ); } + public CoordinatorResult genericGroupSync( + RequestContext context, + SyncGroupRequestData request, + CompletableFuture responseFuture + ) { + return groupMetadataManager.genericGroupSync( + context, + request, + responseFuture + ); + } + /** * The coordinator has been loaded. This is used to apply any * post loading operations (e.g. registering timers). diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java index c6036374d78c1..7f76b0694c949 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java @@ -126,6 +126,12 @@ public class GenericGroup implements Group { */ private int generationId; + /** + * The group's generation id when the current + * pending sync operation was scheduled. + */ + private int pendingSyncGenerationId = -1; + /** * The id of the group's leader. */ @@ -251,6 +257,13 @@ public int generationId() { return this.generationId; } + /** + * @return the pending sync generation id. + */ + public int pendingSyncGenerationId() { + return this.pendingSyncGenerationId; + } + /** * @return the protocol name. */ @@ -362,6 +375,15 @@ public void setSubscribedTopics(Optional> subscribedTopics) { this.subscribedTopics = subscribedTopics; } + /** + * Sets pendingSyncGenerationId. + * + * @param generationId the value to set. + */ + public void setPendingSyncGenerationId(int generationId) { + this.pendingSyncGenerationId = generationId; + } + /** * @return whether the group is using the consumer protocol. */ @@ -694,7 +716,7 @@ public Set allStaticMemberIds() { } // For testing only. - Set allDynamicMemberIds() { + public Set allDynamicMemberIds() { Set dynamicMemberSet = new HashSet<>(allMemberIds()); staticMembers.values().forEach(dynamicMemberSet::remove); return dynamicMemberSet; diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java index 9878e2d4afed8..3e0386e71cdac 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java @@ -26,6 +26,7 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; /** * This class encapsulates a generic group member's metadata. @@ -400,13 +401,15 @@ public void setIsNew(boolean value) { public String toString() { return "GenericGroupMember(" + "memberId='" + memberId + '\'' + - ", groupInstanceId='" + groupInstanceId + '\'' + + ", groupInstanceId='" + groupInstanceId.orElse("") + '\'' + ", clientId='" + clientId + '\'' + ", clientHost='" + clientHost + '\'' + ", rebalanceTimeoutMs=" + rebalanceTimeoutMs + ", sessionTimeoutMs=" + sessionTimeoutMs + ", protocolType='" + protocolType + '\'' + - ", supportedProtocols=" + supportedProtocols + + ", supportedProtocols=" + supportedProtocols.stream() + .map(JoinGroupRequestProtocol::name) + .collect(Collectors.toList()) + ')'; } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index af4cb709b2801..7cefc7d3b3279 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -32,6 +32,10 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; +import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember; +import org.apache.kafka.common.message.SyncGroupRequestData; +import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment; +import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.common.metadata.RemoveTopicRecord; import org.apache.kafka.common.metadata.TopicRecord; @@ -44,6 +48,7 @@ import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.coordinator.group.MockCoordinatorTimer.ExpiredTimeout; @@ -73,6 +78,7 @@ import org.apache.kafka.coordinator.group.generated.GroupMetadataValue; import org.apache.kafka.coordinator.group.generic.GenericGroup; import org.apache.kafka.coordinator.group.generic.GenericGroupMember; +import org.apache.kafka.coordinator.group.generic.GenericGroupState; import org.apache.kafka.coordinator.group.runtime.CoordinatorResult; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; @@ -567,7 +573,7 @@ public CoordinatorResult sendGenericGroupJoin( ); } - public JoinGroupResponseData joinGenericGroupAsDynamicMemberAndCompleteJoin( + public JoinGroupResponseData joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin( JoinGroupRequestData request ) throws ExecutionException, InterruptedException { boolean requireKnownMemberId = true; @@ -614,9 +620,23 @@ public JoinGroupResponseData joinGenericGroupAndCompleteJoin( JoinGroupRequestData request, boolean requireKnownMemberId, boolean supportSkippingAssignment + ) throws ExecutionException, InterruptedException { + return joinGenericGroupAndCompleteJoin( + request, + requireKnownMemberId, + supportSkippingAssignment, + genericGroupInitialRebalanceDelayMs + ); + } + + public JoinGroupResponseData joinGenericGroupAndCompleteJoin( + JoinGroupRequestData request, + boolean requireKnownMemberId, + boolean supportSkippingAssignment, + int advanceClockMs ) throws ExecutionException, InterruptedException { if (requireKnownMemberId && request.groupInstanceId().isEmpty()) { - return joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + return joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); } try { @@ -628,7 +648,7 @@ public JoinGroupResponseData joinGenericGroupAndCompleteJoin( supportSkippingAssignment ); - sleep(genericGroupInitialRebalanceDelayMs); + sleep(advanceClockMs); assertTrue(responseFuture.isDone()); assertEquals(Errors.NONE.code(), responseFuture.get().errorCode()); return responseFuture.get(); @@ -638,6 +658,33 @@ public JoinGroupResponseData joinGenericGroupAndCompleteJoin( return null; } + public CoordinatorResult sendGenericGroupSync( + SyncGroupRequestData request, + CompletableFuture responseFuture + ) { + RequestContext context = new RequestContext( + new RequestHeader( + ApiKeys.SYNC_GROUP, + ApiKeys.SYNC_GROUP.latestVersion(), + "client", + 0 + ), + "1", + InetAddress.getLoopbackAddress(), + KafkaPrincipal.ANONYMOUS, + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), + SecurityProtocol.PLAINTEXT, + ClientInformation.EMPTY, + false + ); + + return groupMetadataManager.genericGroupSync( + context, + request, + responseFuture + ); + } + private ApiMessage messageOrNull(ApiMessageAndVersion apiMessageAndVersion) { if (apiMessageAndVersion == null) { return null; @@ -3918,7 +3965,7 @@ public void testGenericGroupJoinInconsistentProtocolType() throws Exception { .withDefaultProtocolTypeAndProtocols() .build(); - context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -4005,7 +4052,7 @@ public void testNewMemberJoinExpiration() throws Exception { .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs) .build(); - JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData firstResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); String firstMemberId = firstResponse.memberId(); assertEquals(firstResponse.leader(), firstMemberId); assertEquals(Errors.NONE.code(), firstResponse.errorCode()); @@ -4184,7 +4231,7 @@ public void testJoinGroupUnknownConsumerExistingGroup() throws Exception { .withRebalanceTimeoutMs(5000) .build(); - JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4518,7 +4565,7 @@ public void testExistingMemberJoinDeadGroup() throws Exception { .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4581,7 +4628,7 @@ public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() throws .withProtocols(protocols) .build(); - JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4630,7 +4677,7 @@ public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4672,7 +4719,7 @@ public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInS .withProtocols(protocols) .build(); - JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData leaderResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), leaderResponse.errorCode()); String leaderId = leaderResponse.leader(); assertEquals(1, group.generationId()); @@ -4744,7 +4791,7 @@ public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() th .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData leaderResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), leaderResponse.errorCode()); String leaderId = leaderResponse.leader(); assertEquals(1, group.generationId()); @@ -4795,7 +4842,7 @@ public void testJoinGroupExistingMemberInEmptyState() throws Exception { .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4828,7 +4875,7 @@ public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exceptio .withRebalanceTimeoutMs(1000) .build(); - JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData leaderResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), leaderResponse.errorCode()); assertEquals(1, group.generationId()); @@ -5153,15 +5200,17 @@ public void testReplaceStaticMemberInStableStateErrors() throws Exception { .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("consumer") .withProtocols(protocols) + .withRebalanceTimeoutMs(4000) + .withSessionTimeoutMs(3000) .build(); JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, false, false); assertEquals(Errors.NONE.code(), response.errorCode()); - GenericGroupMember oldMember = group.member(response.memberId()); assertEquals(1, group.size()); assertEquals(1, group.generationId()); assertTrue(group.isInState(COMPLETING_REBALANCE)); + String oldMemberId = response.memberId(); // Simulate successful sync group phase group.transitionTo(STABLE); @@ -5187,13 +5236,13 @@ public void testReplaceStaticMemberInStableStateErrors() throws Exception { ); assertFalse(responseFuture.isDone()); - // Simulate failed write to log. + // Simulate a failed write to the log. result.appendFuture().completeExceptionally(new UnknownTopicOrPartitionException()); assertTrue(responseFuture.isDone()); JoinGroupResponseData expectedResponse = new JoinGroupResponseData() .setMembers(Collections.emptyList()) - .setLeader(oldMember.memberId()) + .setLeader(oldMemberId) .setMemberId(UNKNOWN_MEMBER_ID) .setGenerationId(1) .setProtocolType("consumer") @@ -5205,11 +5254,102 @@ public void testReplaceStaticMemberInStableStateErrors() throws Exception { GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id")); - assertEquals(oldMember.memberId(), revertedMember.memberId()); - assertEquals(oldMember.groupInstanceId(), revertedMember.groupInstanceId()); - assertEquals(oldMember.rebalanceTimeoutMs(), revertedMember.rebalanceTimeoutMs()); - assertEquals(oldMember.sessionTimeoutMs(), revertedMember.sessionTimeoutMs()); - assertEquals(oldMember.supportedProtocols(), revertedMember.supportedProtocols()); + assertEquals(oldMemberId, revertedMember.memberId()); + assertEquals(Optional.of("group-instance-id"), revertedMember.groupInstanceId()); + assertEquals(4000, revertedMember.rebalanceTimeoutMs()); + assertEquals(3000, revertedMember.sessionTimeoutMs()); + assertEquals(protocols, revertedMember.supportedProtocols()); + assertEquals(1, group.size()); + assertEquals(1, group.generationId()); + assertTrue(group.isInState(STABLE)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testReplaceStaticMemberInStableStateSucceeds( + boolean supportSkippingAssignment + ) throws Exception { + // If the append future succeeds, the soft state is updated with the new member. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); + + protocols.add(new JoinGroupRequestProtocol() + .setName("range") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array()) + ); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("group-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolType("consumer") + .withProtocols(protocols) + .build(); + + JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin( + request, + true, + supportSkippingAssignment + ); + + assertEquals(Errors.NONE.code(), response.errorCode()); + assertEquals(1, group.size()); + assertEquals(1, group.generationId()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + + String oldMemberId = response.memberId(); + // Simulate successful sync group phase + group.transitionTo(STABLE); + + // Static member rejoins with UNKNOWN_MEMBER_ID and the append succeeds. + protocols.add(new JoinGroupRequestProtocol() + .setName("roundrobin") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("bar"))).array())); + + CompletableFuture responseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request + .setProtocols(protocols) + .setRebalanceTimeoutMs(7000) + .setSessionTimeoutMs(6000), + responseFuture, + true, + supportSkippingAssignment); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + assertFalse(responseFuture.isDone()); + + // Simulate a successful write to the log. + result.appendFuture().complete(null); + assertTrue(responseFuture.isDone()); + + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() + .setMembers(supportSkippingAssignment ? toJoinResponseMembers(group) : Collections.emptyList()) + .setLeader(supportSkippingAssignment ? responseFuture.get().memberId() : oldMemberId) + .setMemberId(responseFuture.get().memberId()) + .setGenerationId(1) + .setProtocolType("consumer") + .setProtocolName("range") + .setSkipAssignment(supportSkippingAssignment) + .setErrorCode(Errors.NONE.code()); + + assertEquals(expectedResponse, responseFuture.get()); + + GenericGroupMember newMember = group.member(group.staticMemberId("group-instance-id")); + + assertNotEquals(oldMemberId, newMember.memberId()); + assertEquals(Optional.of("group-instance-id"), newMember.groupInstanceId()); + assertEquals(7000, newMember.rebalanceTimeoutMs()); + assertEquals(6000, newMember.sessionTimeoutMs()); + assertEquals(protocols, newMember.supportedProtocols()); assertEquals(1, group.size()); assertEquals(1, group.generationId()); assertTrue(group.isInState(STABLE)); @@ -5469,6 +5609,21 @@ JoinGroupRequestBuilder withDefaultProtocolTypeAndProtocols() { return this; } + JoinGroupRequestBuilder withProtocolSuperset() { + this.protocols.add(new JoinGroupRequestProtocol() + .setName("range") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array()) + ); + + this.protocols.add(new JoinGroupRequestProtocol() + .setName("roundrobin") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("bar"))).array()) + ); + return this; + } + JoinGroupRequestBuilder withProtocolType(String protocolType) { this.protocolType = protocolType; return this; @@ -5524,5 +5679,2866 @@ private static Record newGroupMetadataRecord( ) ); } + + @Test + public void testNewMemberTimeoutCompletion() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .withSessionTimeoutMs(context.genericGroupNewMemberJoinTimeoutMs + 5000) + .build(); + + CompletableFuture joinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(joinRequest, joinFuture); + assertTrue(result.records().isEmpty()); + assertFalse(joinFuture.isDone()); + + context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs); + + assertTrue(joinFuture.isDone()); + assertEquals(Errors.NONE.code(), joinFuture.get().errorCode()); + + assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count()); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(joinFuture.get().memberId()) + .withGenerationId(joinFuture.get().generationId()) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate a successful write to the log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertEquals(1, group.size()); + + // Make sure the NewMemberTimeout is not still in effect, and the member is not kicked + context.sleepAndAssertEmptyResult(context.genericGroupNewMemberJoinTimeoutMs); + assertEquals(1, group.size()); + + // Member should be removed as heartbeat expires. + List> timeouts = context.sleep(5000); + List expectedRecords = Collections.singletonList(newGroupMetadataRecord("group-id", + new GroupMetadataValue() + .setMembers(Collections.emptyList()) + .setGeneration(2) + .setLeader(null) + .setProtocolType("consumer") + .setProtocol(null) + .setCurrentStateTimestamp(context.time.milliseconds()), + MetadataVersion.latest())); + + assertEquals(1, timeouts.size()); + String memberId = joinFuture.get().memberId(); + timeouts.forEach(timeout -> { + assertEquals(genericGroupHeartbeatKey("group-id", memberId), timeout.key); + assertEquals(expectedRecords, timeout.result.records()); + }); + + assertEquals(0, group.size()); + } + + @Test + public void testNewMemberFailureAfterJoinGroupCompletion() throws Exception { + // For old versions of the JoinGroup protocol, new members were subject + // to expiration if the rebalance took long enough. This test case ensures + // that following completion of the JoinGroup phase, new members follow + // normal heartbeat expiration logic. + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .withSessionTimeoutMs(5000) + .withRebalanceTimeoutMs(10000) + .build(); + + JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, false, false); + assertEquals(Errors.NONE.code(), joinResponse.errorCode()); + + String memberId = joinResponse.memberId(); + assertEquals(memberId, joinResponse.leader()); + assertEquals(1, joinResponse.generationId()); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(memberId) + .withGenerationId(1) + .build(); + + CompletableFuture syncResponseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncResponseFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate a successful write to the log. + result.appendFuture().complete(null); + + assertTrue(syncResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResponseFuture.get().errorCode()); + + assertTrue(group.isInState(STABLE)); + assertEquals(1, group.generationId()); + + CompletableFuture otherJoinFuture = new CompletableFuture<>(); + CoordinatorResult otherJoinResult = context.sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID), + otherJoinFuture); + + CompletableFuture joinFuture = new CompletableFuture<>(); + CoordinatorResult joinResult = context.sendGenericGroupJoin( + joinRequest.setMemberId(memberId), + joinFuture); + + assertTrue(otherJoinResult.records().isEmpty()); + assertTrue(joinResult.records().isEmpty()); + assertTrue(joinFuture.isDone()); + assertTrue(otherJoinFuture.isDone()); + + verifySessionExpiration(context, group, 5000); + } + + @Test + public void testStaticMemberFenceDuplicateRejoinedFollower() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // A third member joins. Trigger a rebalance. + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + context.sendGenericGroupJoin(request, new CompletableFuture<>()); + + assertTrue(group.isInState(PREPARING_REBALANCE)); + + // Old follower rejoins group will be matching current member.id. + CompletableFuture oldFollowerJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request + .setMemberId(rebalanceResult.followerId) + .setGroupInstanceId("follower-instance-id"), + oldFollowerJoinFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(oldFollowerJoinFuture.isDone()); + + // Duplicate follower joins group with unknown member id will trigger member id replacement. + result = context.sendGenericGroupJoin( + request.setMemberId(UNKNOWN_MEMBER_ID).setGroupInstanceId("follower-instance-id"), + new CompletableFuture<>()); + + // Old member shall be fenced immediately upon duplicate follower joins. + assertTrue(result.records().isEmpty()); + assertTrue(oldFollowerJoinFuture.isDone()); + + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() + .setErrorCode(Errors.FENCED_INSTANCE_ID.code()) + .setProtocolName(null) + .setProtocolType(null) + .setLeader(UNKNOWN_MEMBER_ID) + .setMemberId(rebalanceResult.followerId) + .setGenerationId(-1); + + checkJoinGroupResponse( + expectedResponse, + oldFollowerJoinFuture.get(), + group, + PREPARING_REBALANCE, + Collections.emptySet() + ); + } + + @Test + public void testStaticMemberFenceDuplicateSyncingFollowerAfterMemberIdChanged() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // Known leader rejoins will trigger rebalance. + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .build(); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(request, leaderJoinFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + + // Old follower rejoins group will match current member.id. + CompletableFuture oldFollowerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setMemberId(rebalanceResult.followerId).setGroupInstanceId("follower-instance-id"), + oldFollowerJoinFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(oldFollowerJoinFuture.isDone()); + assertTrue(leaderJoinFuture.isDone()); + + JoinGroupResponseData expectedLeaderResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(rebalanceResult.leaderId) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setMembers(toJoinResponseMembers(group)); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + + checkJoinGroupResponse( + expectedLeaderResponse, + leaderJoinFuture.get(), + group, + COMPLETING_REBALANCE, + expectedGroupInstanceIds + ); + + assertEquals(rebalanceResult.leaderId, leaderJoinFuture.get().memberId()); + assertEquals(rebalanceResult.leaderId, leaderJoinFuture.get().leader()); + + // Old follower should get a successful join group response. + assertTrue(oldFollowerJoinFuture.isDone()); + + JoinGroupResponseData expectedFollowerResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(oldFollowerJoinFuture.get().memberId()) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer"); + + checkJoinGroupResponse( + expectedFollowerResponse, + oldFollowerJoinFuture.get(), + group, + COMPLETING_REBALANCE, + Collections.emptySet() + ); + + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertEquals(rebalanceResult.followerId, oldFollowerJoinFuture.get().memberId()); + assertEquals(rebalanceResult.leaderId, oldFollowerJoinFuture.get().leader()); + + // Duplicate follower joins group with unknown member id will trigger member.id replacement, + // and will also trigger a rebalance under CompletingRebalance state; the old follower sync callback + // will return fenced exception while broker replaces the member identity with the duplicate follower joins. + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withGenerationId(oldFollowerJoinFuture.get().generationId()) + .withMemberId(oldFollowerJoinFuture.get().memberId()) + .build(); + + CompletableFuture oldFollowerSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest, oldFollowerSyncFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(oldFollowerSyncFuture.isDone()); + + CompletableFuture duplicateFollowerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setMemberId(UNKNOWN_MEMBER_ID).setGroupInstanceId("follower-instance-id"), + duplicateFollowerJoinFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + assertFalse(duplicateFollowerJoinFuture.isDone()); + assertTrue(oldFollowerSyncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldFollowerSyncFuture.get().errorCode()); + + // Advance clock by rebalance timeout so that the join phase completes with duplicate follower. + // Leader is kicked out. + context.sleepAndAssertEmptyResult(10000); + + assertTrue(duplicateFollowerJoinFuture.isDone()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertEquals(3, group.generationId()); + assertEquals(1, group.size()); + assertTrue(group.hasMemberId(duplicateFollowerJoinFuture.get().memberId())); + assertEquals(duplicateFollowerJoinFuture.get().memberId(), duplicateFollowerJoinFuture.get().leader()); + } + + @Test + public void testStaticMemberFenceDuplicateRejoiningFollowerAfterMemberIdChanged() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // Known leader rejoins will trigger rebalance. + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .build(); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(request, leaderJoinFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + + // Duplicate follower joins group will trigger member id replacement. + CompletableFuture duplicateFollowerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setMemberId(UNKNOWN_MEMBER_ID).setGroupInstanceId("follower-instance-id"), + duplicateFollowerJoinFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(duplicateFollowerJoinFuture.isDone()); + + // Old follower rejoins group will fail because member id is already updated. + CompletableFuture oldFollowerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setMemberId(rebalanceResult.followerId), + oldFollowerJoinFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(oldFollowerJoinFuture.isDone()); + assertTrue(leaderJoinFuture.isDone()); + + JoinGroupResponseData expectedLeaderResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(rebalanceResult.leaderId) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setMembers(toJoinResponseMembers(group)); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + + checkJoinGroupResponse( + expectedLeaderResponse, + leaderJoinFuture.get(), + group, + COMPLETING_REBALANCE, + expectedGroupInstanceIds + ); + + assertTrue(duplicateFollowerJoinFuture.isDone()); + + JoinGroupResponseData expectedDuplicateFollowerResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(duplicateFollowerJoinFuture.get().memberId()) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedDuplicateFollowerResponse, + duplicateFollowerJoinFuture.get(), + group, + COMPLETING_REBALANCE, + Collections.emptySet() + ); + + assertTrue(duplicateFollowerJoinFuture.isDone()); + + JoinGroupResponseData expectedOldFollowerResponse = new JoinGroupResponseData() + .setErrorCode(Errors.FENCED_INSTANCE_ID.code()) + .setGenerationId(-1) + .setMemberId(rebalanceResult.followerId) + .setLeader(UNKNOWN_MEMBER_ID) + .setProtocolName(null) + .setProtocolType(null) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedOldFollowerResponse, + oldFollowerJoinFuture.get(), + group, + COMPLETING_REBALANCE, + Collections.emptySet() + ); + } + + @Test + public void testStaticMemberRejoinWithKnownMemberId() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withGroupInstanceId("group-instance-id") + .withDefaultProtocolTypeAndProtocols() + .build(); + + JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(request, false, false); + assertEquals(Errors.NONE.code(), joinResponse.errorCode()); + + String memberId = joinResponse.memberId(); + + CompletableFuture rejoinResponseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request.setMemberId(memberId), + rejoinResponseFuture); + + // The second join group should return immediately since we are using the same metadata during CompletingRebalance. + assertTrue(result.records().isEmpty()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertTrue(rejoinResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), rejoinResponseFuture.get().errorCode()); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(memberId) + .withGenerationId(joinResponse.generationId()) + .withGroupInstanceId("group-instance-id") + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Successful write to the log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(group.isInState(STABLE)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testStaticMemberRejoinWithLeaderIdAndUnknownMemberId( + boolean supportSkippingAssignment + ) throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // A static leader rejoin with unknown id will not trigger rebalance, and no assignment will be returned. + // As the group was in Stable state and the member id was updated, this will generate records. + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + CompletableFuture joinResponseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + joinRequest, + joinResponseFuture, + true, + supportSkippingAssignment); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate a successful write to the log. + result.appendFuture().complete(null); + assertTrue(joinResponseFuture.isDone()); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + + String leader = supportSkippingAssignment ? + joinResponseFuture.get().memberId() : rebalanceResult.leaderId; + + List members = supportSkippingAssignment ? + toJoinResponseMembers(group) : Collections.emptyList(); + + JoinGroupResponseData expectedJoinResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId) + .setMemberId(joinResponseFuture.get().memberId()) + .setLeader(leader) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(supportSkippingAssignment) + .setMembers(members); + + checkJoinGroupResponse( + expectedJoinResponse, + joinResponseFuture.get(), + group, + STABLE, + supportSkippingAssignment ? expectedGroupInstanceIds : Collections.emptySet() + ); + } + + @Test + public void testStaticMemberRejoinWithLeaderIdAndKnownMemberId() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // Known static leader rejoin will trigger rebalance. + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .build(); + + JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(request, true, true, 10000); + + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(rebalanceResult.leaderId) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setMembers(toJoinResponseMembers(group)); + + checkJoinGroupResponse( + expectedResponse, + joinResponse, + group, + COMPLETING_REBALANCE, + Collections.singleton("leader-instance-id") + ); + } + + @Test + public void testStaticMemberRejoinWithLeaderIdAndUnexpectedDeadGroup() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + group.transitionTo(DEAD); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withDefaultProtocolTypeAndProtocols() + .build(); + + CompletableFuture joinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(request, joinFuture, true, true); + + assertTrue(result.records().isEmpty()); + assertTrue(joinFuture.isDone()); + assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), joinFuture.get().errorCode()); + } + + @Test + public void testStaticMemberRejoinWithLeaderIdAndUnexpectedEmptyGroup() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + group.transitionTo(PREPARING_REBALANCE); + group.transitionTo(EMPTY); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withDefaultProtocolTypeAndProtocols() + .build(); + + CompletableFuture joinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(request, joinFuture, true, true); + + assertTrue(result.records().isEmpty()); + assertTrue(joinFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), joinFuture.get().errorCode()); + } + + @Test + public void testStaticMemberRejoinWithFollowerIdAndChangeOfProtocol() throws Exception { + int rebalanceTimeoutMs = 10000; + int sessionTimeoutMs = 15000; + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id", + rebalanceTimeoutMs, + sessionTimeoutMs + ); + + // A static follower rejoin with changed protocol will trigger rebalance. + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); + protocols.add(new JoinGroupRequestProtocol() + .setName("roundrobin") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array()) + ); + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(rebalanceResult.followerId) + .withProtocols(protocols) + .withRebalanceTimeoutMs(rebalanceTimeoutMs) + .withSessionTimeoutMs(sessionTimeoutMs) + .build(); + + CompletableFuture responseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(request, responseFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(responseFuture.isDone()); + + // Old leader hasn't joined in the meantime, triggering a re-election. + context.sleepAndAssertEmptyResult(rebalanceTimeoutMs); + + assertTrue(responseFuture.isDone()); + assertEquals(Errors.NONE.code(), responseFuture.get().errorCode()); + assertTrue(group.hasStaticMember("leader-instance-id")); + assertTrue(group.isLeader(rebalanceResult.followerId)); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(rebalanceResult.followerId) + .setLeader(rebalanceResult.followerId) + .setProtocolName("roundrobin") + .setProtocolType("consumer") + .setMembers(toJoinResponseMembers(group)); + + checkJoinGroupResponse( + expectedResponse, + responseFuture.get(), + group, + COMPLETING_REBALANCE, + expectedGroupInstanceIds + ); + } + + @Test + public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWithSelectedProtocolChanged() + throws Exception { + + int rebalanceTimeoutMs = 10000; + int sessionTimeoutMs = 15000; + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id", + rebalanceTimeoutMs, + sessionTimeoutMs + ); + + assertNotEquals("roundrobin", group.selectProtocol()); + + // A static follower rejoin with changed protocol will trigger rebalance. + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); + protocols.add(new JoinGroupRequestProtocol() + .setName("roundrobin") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array()) + ); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocols(protocols) + .withRebalanceTimeoutMs(rebalanceTimeoutMs) + .withSessionTimeoutMs(sessionTimeoutMs) + .build(); + + CompletableFuture responseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(request, responseFuture); + assertTrue(result.records().isEmpty()); + assertFalse(responseFuture.isDone()); + assertEquals("roundrobin", group.selectProtocol()); + + // Old leader hasn't joined in the meantime, triggering a re-election. + context.sleepAndAssertEmptyResult(rebalanceTimeoutMs); + assertTrue(responseFuture.isDone()); + assertEquals(Errors.NONE.code(), responseFuture.get().errorCode()); + assertTrue(group.hasStaticMember("leader-instance-id")); + assertTrue(group.isLeader(responseFuture.get().memberId())); + assertNotEquals(rebalanceResult.followerId, responseFuture.get().memberId()); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(responseFuture.get().memberId()) + .setLeader(responseFuture.get().memberId()) + .setProtocolName("roundrobin") + .setProtocolType("consumer") + .setMembers(toJoinResponseMembers(group)); + + checkJoinGroupResponse( + expectedResponse, + responseFuture.get(), + group, + COMPLETING_REBALANCE, + expectedGroupInstanceIds + ); + } + + @Test + public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSelectProtocolUnchangedPersistenceFailure() + throws Exception { + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + String selectedProtocol = group.selectProtocol(); + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); + protocols.add(new JoinGroupRequestProtocol() + .setName(selectedProtocol) + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array()) + ); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocols(protocols) + .build(); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + followerJoinFuture, + true, + true); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate a failed write to the log. + result.appendFuture().completeExceptionally(Errors.MESSAGE_TOO_LARGE.exception()); + assertTrue(followerJoinFuture.isDone()); + + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) + .setGenerationId(rebalanceResult.generationId) + .setMemberId(followerJoinFuture.get().memberId()) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedResponse, + followerJoinFuture.get(), + group, + STABLE, + Collections.emptySet() + ); + + // Join with old member id will not fail because the member id is not updated because of persistence failure + assertNotEquals(rebalanceResult.followerId, followerJoinFuture.get().memberId()); + followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin(request.setMemberId(rebalanceResult.followerId), followerJoinFuture); + + assertTrue(result.records().isEmpty()); + + // Join with leader and complete join phase. + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setGroupInstanceId("leader-instance-id") + .setMemberId(rebalanceResult.leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + + // Sync with leader and receive assignment. + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withGenerationId(rebalanceResult.generationId + 1) + .build(); + + CompletableFuture leaderSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest, leaderSyncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + + // Simulate a successful write to the log. + result.appendFuture().complete(null); + + assertTrue(leaderSyncFuture.isDone()); + assertTrue(group.isInState(STABLE)); + assertEquals(Errors.NONE.code(), leaderSyncFuture.get().errorCode()); + + // Sync with old member id will also not fail because the member id is not updated because of persistence failure + CompletableFuture oldMemberSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest + .setGroupInstanceId("follower-instance-id") + .setMemberId(rebalanceResult.followerId), + oldMemberSyncFuture + ); + assertTrue(result.records().isEmpty()); + assertTrue(oldMemberSyncFuture.isDone()); + assertTrue(group.isInState(STABLE)); + assertEquals(Errors.NONE.code(), oldMemberSyncFuture.get().errorCode()); + } + + @Test + public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSelectProtocolUnchanged() + throws Exception { + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // A static follower rejoin with protocol changing to leader protocol subset won't trigger rebalance if updated + // group's selectProtocol remain unchanged. + String selectedProtocol = group.selectProtocol(); + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); + protocols.add(new JoinGroupRequestProtocol() + .setName(selectedProtocol) + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array()) + ); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocols(protocols) + .build(); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + followerJoinFuture, + true, + true); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + + // Simulate a successful write to the log. + result.appendFuture().complete(null); + assertTrue(followerJoinFuture.isDone()); + + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId) + .setMemberId(followerJoinFuture.get().memberId()) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedResponse, + followerJoinFuture.get(), + group, + STABLE, + Collections.emptySet() + ); + + // Join with old member id will fail because the member id is updated + String newFollowerId = followerJoinFuture.get().memberId(); + assertNotEquals(rebalanceResult.followerId, newFollowerId); + followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin(request.setMemberId(rebalanceResult.followerId), followerJoinFuture); + + assertTrue(result.records().isEmpty()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), followerJoinFuture.get().errorCode()); + + // Sync with old member id will fail because the member id is updated + CompletableFuture syncFuture = new CompletableFuture<>(); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withGenerationId(rebalanceResult.generationId) + .withMemberId(rebalanceResult.followerId) + .withAssignments(Collections.emptyList()) + .build(); + + CoordinatorResult syncResult = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertTrue(syncResult.records().isEmpty()); + assertTrue(syncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), syncFuture.get().errorCode()); + + + // Sync with new member id succeeds + syncFuture = new CompletableFuture<>(); + syncResult = context.sendGenericGroupSync( + syncRequest.setMemberId(newFollowerId), + syncFuture + ); + + assertTrue(syncResult.records().isEmpty()); + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertEquals(rebalanceResult.followerAssignment, syncFuture.get().assignment()); + } + + @Test + public void testStaticMemberRejoinWithKnownLeaderIdToTriggerRebalanceAndFollowerWithChangeofProtocol() + throws Exception { + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // A static leader rejoin with known member id will trigger rebalance. + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withProtocolSuperset() + .withRebalanceTimeoutMs(10000) + .withSessionTimeoutMs(5000) + .build(); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + leaderJoinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinFuture.isDone()); + + // Rebalance completes immediately after follower rejoins. + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setGroupInstanceId("follower-instance-id") + .setMemberId(rebalanceResult.followerId), + followerJoinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertEquals(2, group.generationId()); + + // Leader should get the same assignment as last round. + JoinGroupResponseData expectedLeaderResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) // The group has promoted to the new generation. + .setMemberId(leaderJoinFuture.get().memberId()) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(toJoinResponseMembers(group)); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + + checkJoinGroupResponse( + expectedLeaderResponse, + leaderJoinFuture.get(), + group, + COMPLETING_REBALANCE, + expectedGroupInstanceIds + ); + + JoinGroupResponseData expectedFollowerResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) // The group has promoted to the new generation. + .setMemberId(followerJoinFuture.get().memberId()) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedFollowerResponse, + followerJoinFuture.get(), + group, + COMPLETING_REBALANCE, + Collections.emptySet() + ); + + // The follower protocol changed from protocolSuperset to general protocols. + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(); + protocols.add(new JoinGroupRequestProtocol() + .setName("range") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array())); + + followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setGroupInstanceId("follower-instance-id") + .setMemberId(rebalanceResult.followerId) + .setProtocols(protocols), + followerJoinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + // The group will transition to PreparingRebalance due to protocol change from follower. + assertTrue(group.isInState(PREPARING_REBALANCE)); + + // Advance clock by session timeout to kick leader out and complete join phase. + List> timeouts = context.sleep(5000); + // Both leader and follower heartbeat timers may expire. However, the follower heartbeat expiration + // will not kick the follower out because it is awaiting a join response. + assertTrue(timeouts.size() <= 2); + assertTrue(followerJoinFuture.isDone()); + + String newFollowerId = followerJoinFuture.get().memberId(); + expectedFollowerResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 2) // The group has promoted to the new generation. + .setMemberId(newFollowerId) + .setLeader(newFollowerId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(toJoinResponseMembers(group)); + + checkJoinGroupResponse( + expectedFollowerResponse, + followerJoinFuture.get(), + group, + COMPLETING_REBALANCE, + Collections.singleton("follower-instance-id") + ); + } + + @Test + public void testStaticMemberRejoinAsFollowerWithUnknownMemberId() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // A static follower rejoin with no protocol change will not trigger rebalance. + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolSuperset() + .build(); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + followerJoinFuture, + true, + true); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + assertTrue(followerJoinFuture.isDone()); + + // Old leader shouldn't be timed out. + assertTrue(group.hasStaticMember("leader-instance-id")); + + JoinGroupResponseData expectedFollowerResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId) // The group has not changed. + .setMemberId(followerJoinFuture.get().memberId()) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedFollowerResponse, + followerJoinFuture.get(), + group, + STABLE, + Collections.emptySet() + ); + assertNotEquals(rebalanceResult.followerId, followerJoinFuture.get().memberId()); + + CompletableFuture syncFuture = new CompletableFuture<>(); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withGenerationId(rebalanceResult.generationId) + .withMemberId(followerJoinFuture.get().memberId()) + .build(); + + result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertEquals(rebalanceResult.followerAssignment, syncFuture.get().assignment()); + } + + @Test + public void testStaticMemberRejoinAsFollowerWithKnownMemberIdAndNoProtocolChange() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + // A static follower rejoin with no protocol change will not trigger rebalance. + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(rebalanceResult.followerId) + .withProtocolSuperset() + .build(); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + followerJoinFuture, + true, + true); + + // No records to write because no metadata changed. + assertTrue(result.records().isEmpty()); + assertTrue(followerJoinFuture.isDone()); + + // Old leader shouldn't be timed out. + assertTrue(group.hasStaticMember("leader-instance-id")); + + JoinGroupResponseData expectedFollowerResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId) // The group has not changed. + .setMemberId(rebalanceResult.followerId) + .setLeader(rebalanceResult.leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedFollowerResponse, + followerJoinFuture.get(), + group, + STABLE, + Collections.emptySet() + ); + } + + @Test + public void testStaticMemberRejoinAsFollowerWithMismatchedInstanceId() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(rebalanceResult.followerId) + .withProtocolSuperset() + .build(); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + followerJoinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), followerJoinFuture.get().errorCode()); + } + + @Test + public void testStaticMemberRejoinAsLeaderWithMismatchedInstanceId() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withProtocolSuperset() + .build(); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + leaderJoinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), leaderJoinFuture.get().errorCode()); + } + + @Test + public void testStaticMemberSyncAsLeaderWithInvalidMemberId() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + context.createGenericGroup("group-id"); + + staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + SyncGroupRequestData request = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId("invalid-member-id") + .build(); + + CompletableFuture leaderSyncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync( + request, + leaderSyncFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderSyncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), leaderSyncFuture.get().errorCode()); + } + + @Test + public void testGetDifferentStaticMemberIdAfterEachRejoin() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + String lastMemberId = rebalanceResult.leaderId; + for (int i = 0; i < 5; i++) { + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolSuperset() + .build(); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + leaderJoinFuture, + true, + true); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + assertTrue(leaderJoinFuture.isDone()); + assertEquals(group.staticMemberId("leader-instance-id"), leaderJoinFuture.get().memberId()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertNotEquals(lastMemberId, leaderJoinFuture.get().memberId()); + + lastMemberId = leaderJoinFuture.get().memberId(); + } + } + + @Test + public void testStaticMemberJoinWithUnknownInstanceIdAndKnownMemberId() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + context.createGenericGroup("group-id"); + + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("unknown-instance-id") + .withMemberId(rebalanceResult.leaderId) + .withProtocolSuperset() + .build(); + + CompletableFuture joinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + joinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertTrue(joinFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), joinFuture.get().errorCode()); + } + + @Test + public void testStaticMemberReJoinWithIllegalStateAsUnknownMember() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id" + ); + + group.transitionTo(PREPARING_REBALANCE); + group.transitionTo(EMPTY); + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("follower-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolSuperset() + .build(); + + // Illegal state exception shall trigger since follower id resides in pending member bucket. + IllegalStateException exception = assertThrows(IllegalStateException.class, () -> context.sendGenericGroupJoin( + request, + new CompletableFuture<>(), + true, + true)); + + String message = exception.getMessage(); + assertTrue(message.contains(group.groupId())); + assertTrue(message.contains("follower-instance-id")); + } + + @Test + public void testStaticMemberFollowerFailToRejoinBeforeRebalanceTimeout() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + // Increase session timeout so that the follower won't be evicted when rebalance timeout is reached. + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id", + 10000, + 15000 + ); + + String newMemberInstanceId = "new-member-instance-id"; + String leaderId = rebalanceResult.leaderId; + + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId(newMemberInstanceId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolSuperset() + .build(); + + CompletableFuture newMemberFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + newMemberFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertFalse(newMemberFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setGroupInstanceId("leader-instance-id") + .setMemberId(leaderId), + leaderJoinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinFuture.isDone()); + + // Advance clock by rebalance timeout to complete join phase. + assertNoOrEmptyResult(context.sleep(10000)); + + assertTrue(leaderJoinFuture.isDone()); + assertTrue(newMemberFuture.isDone()); + + JoinGroupResponseData expectedLeaderResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(leaderId) + .setLeader(leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(toJoinResponseMembers(group)); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + expectedGroupInstanceIds.add(newMemberInstanceId); + + checkJoinGroupResponse( + expectedLeaderResponse, + leaderJoinFuture.get(), + group, + COMPLETING_REBALANCE, + expectedGroupInstanceIds + ); + + JoinGroupResponseData expectedNewMemberResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(newMemberFuture.get().memberId()) + .setLeader(leaderId) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedNewMemberResponse, + newMemberFuture.get(), + group, + COMPLETING_REBALANCE, + Collections.emptySet() + ); + } + + @Test + public void testStaticMemberLeaderFailToRejoinBeforeRebalanceTimeout() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + // Increase session timeout so that the leader won't be evicted when rebalance timeout is reached. + RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( + context, + "group-id", + "leader-instance-id", + "follower-instance-id", + 10000, + 15000 + ); + + String newMemberInstanceId = "new-member-instance-id"; + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId(newMemberInstanceId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolSuperset() + .build(); + + CompletableFuture newMemberFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin( + request, + newMemberFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertFalse(newMemberFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + + CompletableFuture oldFollowerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + request.setGroupInstanceId("follower-instance-id") + .setMemberId(rebalanceResult.followerId), + oldFollowerJoinFuture, + true, + true); + + assertTrue(result.records().isEmpty()); + assertFalse(oldFollowerJoinFuture.isDone()); + + // Advance clock by rebalance timeout to complete join phase. + assertNoOrEmptyResult(context.sleep(10000)); + + assertTrue(oldFollowerJoinFuture.isDone()); + assertTrue(newMemberFuture.isDone()); + + JoinGroupResponseData newLeaderResponse = oldFollowerJoinFuture.get().leader() + .equals(oldFollowerJoinFuture.get().memberId()) ? oldFollowerJoinFuture.get() : newMemberFuture.get(); + + JoinGroupResponseData newFollowerResponse = oldFollowerJoinFuture.get().leader() + .equals(oldFollowerJoinFuture.get().memberId()) ? newMemberFuture.get() : oldFollowerJoinFuture.get(); + + JoinGroupResponseData expectedLeaderResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(newLeaderResponse.memberId()) + .setLeader(newLeaderResponse.memberId()) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(toJoinResponseMembers(group)); + + Set expectedGroupInstanceIds = new HashSet<>(); + expectedGroupInstanceIds.add("leader-instance-id"); + expectedGroupInstanceIds.add("follower-instance-id"); + expectedGroupInstanceIds.add(newMemberInstanceId); + + checkJoinGroupResponse( + expectedLeaderResponse, + newLeaderResponse, + group, + COMPLETING_REBALANCE, + expectedGroupInstanceIds + ); + + JoinGroupResponseData expectedNewMemberResponse = new JoinGroupResponseData() + .setErrorCode(Errors.NONE.code()) + .setGenerationId(rebalanceResult.generationId + 1) + .setMemberId(newFollowerResponse.memberId()) + .setLeader(newLeaderResponse.memberId()) + .setProtocolName("range") + .setProtocolType("consumer") + .setSkipAssignment(false) + .setMembers(Collections.emptyList()); + + checkJoinGroupResponse( + expectedNewMemberResponse, + newFollowerResponse, + group, + COMPLETING_REBALANCE, + Collections.emptySet() + ); + } + + @Test + public void testSyncGroupReturnsAnErrorWhenProtocolTypeIsInconsistent() throws Exception { + testSyncGroupProtocolTypeAndNameWith( + Optional.of("protocolType"), + Optional.empty(), + Errors.INCONSISTENT_GROUP_PROTOCOL, + Optional.empty(), + Optional.empty() + ); + } + + @Test + public void testSyncGroupReturnsAnErrorWhenProtocolNameIsInconsistent() throws Exception { + testSyncGroupProtocolTypeAndNameWith( + Optional.empty(), + Optional.of("protocolName"), + Errors.INCONSISTENT_GROUP_PROTOCOL, + Optional.empty(), + Optional.empty() + ); + } + + @Test + public void testSyncGroupSucceedWhenProtocolTypeAndNameAreNotProvided() throws Exception { + testSyncGroupProtocolTypeAndNameWith( + Optional.empty(), + Optional.empty(), + Errors.NONE, + Optional.of("consumer"), + Optional.of("range") + ); + } + + @Test + public void testSyncGroupSucceedWhenProtocolTypeAndNameAreConsistent() throws Exception { + testSyncGroupProtocolTypeAndNameWith( + Optional.of("consumer"), + Optional.of("range"), + Errors.NONE, + Optional.of("consumer"), + Optional.of("range") + ); + } + + private void testSyncGroupProtocolTypeAndNameWith( + Optional protocolType, + Optional protocolName, + Errors expectedError, + Optional expectedProtocolType, + Optional expectedProtocolName + ) throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + // JoinGroup(leader) with the Protocol Type of the group + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolSuperset() + .build(); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(joinRequest, leaderJoinFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinFuture.isDone()); + + // JoinGroup(follower) with the Protocol Type of the group + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin(joinRequest.setGroupInstanceId("follower-instance-id"), followerJoinFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + + // Advance clock by rebalance timeout to complete join phase. + assertNoOrEmptyResult(context.sleep(10000)); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + + String leaderId = leaderJoinFuture.get().memberId(); + String followerId = followerJoinFuture.get().memberId(); + int generationId = leaderJoinFuture.get().generationId(); + + // SyncGroup with the provided Protocol Type and Name + List assignments = new ArrayList<>(); + assignments.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withProtocolType(protocolType.orElse(null)) + .withProtocolName(protocolName.orElse(null)) + .withGenerationId(generationId) + .withAssignments(assignments) + .build(); + + CompletableFuture leaderSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest, leaderSyncFuture); + + if (expectedError == Errors.NONE) { + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + } else { + assertTrue(result.records().isEmpty()); + } + + assertTrue(leaderSyncFuture.isDone()); + assertEquals(expectedError.code(), leaderSyncFuture.get().errorCode()); + assertEquals(expectedProtocolType.orElse(null), leaderSyncFuture.get().protocolType()); + assertEquals(expectedProtocolName.orElse(null), leaderSyncFuture.get().protocolName()); + + CompletableFuture followerSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest.setMemberId(followerId), followerSyncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(followerSyncFuture.isDone()); + assertEquals(expectedError.code(), followerSyncFuture.get().errorCode()); + assertEquals(expectedProtocolType.orElse(null), followerSyncFuture.get().protocolType()); + assertEquals(expectedProtocolName.orElse(null), followerSyncFuture.get().protocolName()); + } + + @Test + public void testSyncGroupFromUnknownGroup() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + + // SyncGroup with the provided Protocol Type and Name + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId("member-id") + .withGenerationId(1) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(syncFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncFuture.get().errorCode()); + } + + @Test + public void testSyncGroupFromUnknownMember() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, true, true); + + String memberId = joinResponse.memberId(); + int generationId = joinResponse.generationId(); + + List assignments = new ArrayList<>(); + assignments.add(new SyncGroupRequestAssignment().setMemberId(memberId)); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(memberId) + .withGenerationId(generationId) + .withAssignments(assignments) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertEquals(assignments.get(0).assignment(), syncFuture.get().assignment()); + + syncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest.setMemberId("unknown-member-id"), syncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(syncFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncFuture.get().errorCode()); + assertEquals(Bytes.EMPTY, syncFuture.get().assignment()); + } + + @Test + public void testSyncGroupFromIllegalGeneration() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, true, true); + + String memberId = joinResponse.memberId(); + int generationId = joinResponse.generationId(); + + // Send the sync group with an invalid generation + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(memberId) + .withGenerationId(generationId + 1) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(syncFuture.isDone()); + assertEquals(Errors.ILLEGAL_GENERATION.code(), syncFuture.get().errorCode()); + } + + @Test + public void testJoinGroupFromUnchangedFollowerDoesNotRebalance() throws Exception { + // To get a group of two members: + // 1. join and sync with a single member (because we can't immediately join with two members) + // 2. join and sync with the first member and a new member + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + JoinGroupResponseData leaderJoinResponse = + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + assertEquals(leaderJoinFuture.get().generationId(), followerJoinFuture.get().generationId()); + assertEquals(leaderId, leaderJoinFuture.get().leader()); + assertEquals(leaderId, followerJoinFuture.get().leader()); + + int nextGenerationId = leaderJoinFuture.get().generationId(); + String followerId = followerJoinFuture.get().memberId(); + + // This shouldn't cause a rebalance since protocol information hasn't changed + followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(followerId), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + assertEquals(nextGenerationId, followerJoinFuture.get().generationId()); + } + + @Test + public void testLeaderFailureInSyncGroup() throws Exception { + // To get a group of two members: + // 1. join and sync with a single member (because we can't immediately join with two members) + // 2. join and sync with the first member and a new member + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .withSessionTimeoutMs(5000) + .build(); + + JoinGroupResponseData leaderJoinResponse = + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + assertEquals(leaderJoinFuture.get().generationId(), followerJoinFuture.get().generationId()); + assertEquals(leaderId, leaderJoinFuture.get().leader()); + assertEquals(leaderId, followerJoinFuture.get().leader()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + + int nextGenerationId = leaderJoinFuture.get().generationId(); + String followerId = followerJoinFuture.get().memberId(); + + // With no leader SyncGroup, the follower's request should fail with an error indicating + // that it should rejoin + CompletableFuture followerSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest.setMemberId(followerId) + .setGenerationId(nextGenerationId), + followerSyncFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(followerSyncFuture.isDone()); + + // Advance clock by session timeout to expire follower heartbeat and prepare rebalance. + // This should complete all pending syncs. + assertNoOrEmptyResult(context.sleep(5000)); + + assertTrue(followerSyncFuture.isDone()); + assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), followerSyncFuture.get().errorCode()); + } + + @Test + public void testSyncGroupFollowerAfterLeader() throws Exception { + // To get a group of two members: + // 1. join and sync with a single member (because we can't immediately join with two members) + // 2. join and sync with the first member and a new member + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .withSessionTimeoutMs(5000) + .build(); + + JoinGroupResponseData leaderJoinResponse = + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + assertEquals(leaderJoinFuture.get().generationId(), followerJoinFuture.get().generationId()); + assertEquals(leaderId, leaderJoinFuture.get().leader()); + assertEquals(leaderId, followerJoinFuture.get().leader()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + + int nextGenerationId = leaderJoinFuture.get().generationId(); + String followerId = followerJoinFuture.get().memberId(); + byte[] leaderAssignment = new byte[]{0}; + byte[] followerAssignment = new byte[]{1}; + + // Sync group with leader to get new assignment. + List assignments = new ArrayList<>(); + assignments.add(new SyncGroupRequestAssignment() + .setMemberId(leaderId) + .setAssignment(leaderAssignment) + ); + assignments.add(new SyncGroupRequestAssignment() + .setMemberId(followerId) + .setAssignment(followerAssignment) + ); + + syncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest.setGenerationId(nextGenerationId) + .setAssignments(assignments), + syncFuture + ); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertEquals(leaderAssignment, syncFuture.get().assignment()); + + // Sync group with follower to get new assignment. + CompletableFuture followerSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest.setMemberId(followerId) + .setGenerationId(nextGenerationId), + followerSyncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(followerSyncFuture.isDone()); + assertEquals(Errors.NONE.code(), followerSyncFuture.get().errorCode()); + assertEquals(followerAssignment, followerSyncFuture.get().assignment()); + assertTrue(group.isInState(STABLE)); + } + + @Test + public void testSyncGroupLeaderAfterFollower() throws Exception { + // To get a group of two members: + // 1. join and sync with a single member (because we can't immediately join with two members) + // 2. join and sync with the first member and a new member + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .withSessionTimeoutMs(5000) + .build(); + + JoinGroupResponseData leaderJoinResponse = + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + assertEquals(leaderJoinFuture.get().generationId(), followerJoinFuture.get().generationId()); + assertEquals(leaderId, leaderJoinFuture.get().leader()); + assertEquals(leaderId, followerJoinFuture.get().leader()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + + int nextGenerationId = leaderJoinFuture.get().generationId(); + String followerId = followerJoinFuture.get().memberId(); + byte[] leaderAssignment = new byte[]{0}; + byte[] followerAssignment = new byte[]{1}; + + // Sync group with follower to get new assignment. + CompletableFuture followerSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest.setMemberId(followerId) + .setGenerationId(nextGenerationId), + followerSyncFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(followerSyncFuture.isDone()); + + // Sync group with leader to get new assignment. + List assignments = new ArrayList<>(); + assignments.add(new SyncGroupRequestAssignment() + .setMemberId(leaderId) + .setAssignment(leaderAssignment) + ); + assignments.add(new SyncGroupRequestAssignment() + .setMemberId(followerId) + .setAssignment(followerAssignment) + ); + + syncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest.setMemberId(leaderId) + .setGenerationId(nextGenerationId) + .setAssignments(assignments), + syncFuture + ); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertEquals(leaderAssignment, syncFuture.get().assignment()); + + // Follower sync group should also be completed. + assertEquals(Errors.NONE.code(), followerSyncFuture.get().errorCode()); + assertEquals(followerAssignment, followerSyncFuture.get().assignment()); + assertTrue(group.isInState(STABLE)); + } + + @Test + public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + JoinGroupResponseData leaderJoinResponse = + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + + // Join group from the leader should force the group to rebalance, which allows the + // leader to push new assignments when local metadata changes + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(leaderJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(generationId + 1, leaderJoinFuture.get().generationId()); + } + + @Test + public void testJoinGroupCompletionWhenPendingMemberJoins() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + // Set up a group in with a pending member. The test checks if the pending member joining + // completes the rebalancing operation + JoinGroupResponseData pendingMemberResponse = setupGroupWithPendingMember(context, group); + + // Compete join group for the pending member + JoinGroupRequestData request = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(pendingMemberResponse.memberId()) + .withDefaultProtocolTypeAndProtocols() + .build(); + + CompletableFuture responseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(request, responseFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(responseFuture.isDone()); + assertEquals(Errors.NONE.code(), responseFuture.get().errorCode()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertEquals(3, group.allMembers().size()); + assertEquals(0, group.numPendingJoinMembers()); + } + + @Test + public void testJoinGroupCompletionWhenPendingMemberTimesOut() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + // Set up a group in with a pending member. The test checks if the timeout of the pending member will + // cause the group to return to a CompletingRebalance state. + setupGroupWithPendingMember(context, group); + + // Advancing clock by > 2500 (session timeout for the third member) + // and < 5000 (for first and second members). This will force the coordinator to attempt join + // completion on heartbeat expiration (since we are in PendingRebalance stage). + assertNoOrEmptyResult(context.sleep(3000)); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertEquals(2, group.allMembers().size()); + assertEquals(0, group.numPendingJoinMembers()); + } + + private JoinGroupResponseData setupGroupWithPendingMember( + GroupMetadataManagerTestContext context, + GenericGroup group + ) throws Exception { + // Add the first member + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .withSessionTimeoutMs(5000) + .build(); + + JoinGroupResponseData leaderJoinResponse = + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + List assignments = new ArrayList<>(); + assignments.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + // Now the group is stable, with the one member that joined above + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + + // Start the join for the second member + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + assertEquals(leaderJoinFuture.get().generationId(), followerJoinFuture.get().generationId()); + assertEquals(leaderId, leaderJoinFuture.get().leader()); + assertEquals(leaderId, followerJoinFuture.get().leader()); + + int nextGenerationId = leaderJoinFuture.get().generationId(); + String followerId = followerJoinFuture.get().memberId(); + + // Stabilize the group + syncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(syncRequest.setGenerationId(nextGenerationId), syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(group.isInState(STABLE)); + + // Re-join an existing member, to transition the group to PreparingRebalance state. + leaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(leaderId), + leaderJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + + // Create a pending member in the group + CompletableFuture pendingMemberJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID).setSessionTimeoutMs(2500), + pendingMemberJoinFuture, + true + ); + + assertTrue(result.records().isEmpty()); + assertTrue(pendingMemberJoinFuture.isDone()); + assertEquals(Errors.MEMBER_ID_REQUIRED.code(), pendingMemberJoinFuture.get().errorCode()); + assertEquals(1, group.numPendingJoinMembers()); + + // Re-join the second existing member + followerJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(followerId).setSessionTimeoutMs(5000), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + assertEquals(1, group.numPendingJoinMembers()); + + return pendingMemberJoinFuture.get(); + } + + @Test + public void testGenerationIdIncrementsOnRebalance() throws Exception { + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .build(); + + JoinGroupResponseData leaderJoinResponse = + context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + assertEquals(1, generationId); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .build(); + + CompletableFuture syncFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + // Simulate successful write to log. + result.appendFuture().complete(null); + + assertTrue(syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + + CompletableFuture joinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin(joinRequest.setMemberId(leaderId), joinFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(joinFuture.isDone()); + assertEquals(2, joinFuture.get().generationId()); + assertEquals(Errors.NONE.code(), joinFuture.get().errorCode()); + } + + private List toJoinResponseMembers(GenericGroup group) { + List members = new ArrayList<>(); + String protocolName = group.protocolName().get(); + group.allMembers().forEach(member -> { + members.add(new JoinGroupResponseMember() + .setMemberId(member.memberId()) + .setGroupInstanceId(member.groupInstanceId().orElse("")) + .setMetadata(member.metadata(protocolName))); + }); + + return members; + } + + private void checkJoinGroupResponse( + JoinGroupResponseData expectedResponse, + JoinGroupResponseData actualResponse, + GenericGroup group, + GenericGroupState expectedState, + Set expectedGroupInstanceIds + ) { + assertEquals(expectedResponse, actualResponse); + assertTrue(group.isInState(expectedState)); + + Set groupInstanceIds = actualResponse.members() + .stream() + .map(JoinGroupResponseData.JoinGroupResponseMember::groupInstanceId) + .collect(Collectors.toSet()); + + assertEquals(expectedGroupInstanceIds, groupInstanceIds); + } + + private void verifySessionExpiration( + GroupMetadataManagerTestContext context, + GenericGroup group, + int timeoutMs + ) { + Set expectedHeartbeatKeys = group.allMembers().stream() + .map(member -> genericGroupHeartbeatKey(group.groupId(), member.memberId())).collect(Collectors.toSet()); + + // Member should be removed as session expires. + List> timeouts = context.sleep(timeoutMs); + List expectedRecords = Collections.singletonList(newGroupMetadataRecord( + "group-id", + new GroupMetadataValue() + .setMembers(Collections.emptyList()) + .setGeneration(group.generationId()) + .setLeader(null) + .setProtocolType("consumer") + .setProtocol(null) + .setCurrentStateTimestamp(context.time.milliseconds()), + MetadataVersion.latest())); + + + Set heartbeatKeys = timeouts.stream().map(timeout -> timeout.key).collect(Collectors.toSet()); + assertEquals(expectedHeartbeatKeys, heartbeatKeys); + + // Only the last member leaving the group should result in the empty group metadata record. + int timeoutsSize = timeouts.size(); + assertEquals(expectedRecords, timeouts.get(timeoutsSize - 1).result.records()); + assertNoOrEmptyResult(timeouts.subList(0, timeoutsSize - 1)); + assertTrue(group.isInState(EMPTY)); + assertEquals(0, group.size()); + } + + private static class SyncGroupRequestBuilder { + String groupId = null; + String groupInstanceId = null; + String memberId = null; + String protocolType = "consumer"; + String protocolName = "range"; + int generationId = 0; + List assignments = Collections.emptyList(); + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); + + SyncGroupRequestBuilder withGroupId(String groupId) { + this.groupId = groupId; + return this; + } + + SyncGroupRequestBuilder withGroupInstanceId(String groupInstanceId) { + this.groupInstanceId = groupInstanceId; + return this; + } + + SyncGroupRequestBuilder withMemberId(String memberId) { + this.memberId = memberId; + return this; + } + + SyncGroupRequestBuilder withGenerationId(int generationId) { + this.generationId = generationId; + return this; + } + + SyncGroupRequestBuilder withProtocolType(String protocolType) { + this.protocolType = protocolType; + return this; + } + + SyncGroupRequestBuilder withProtocolName(String protocolName) { + this.protocolName = protocolName; + return this; + } + + SyncGroupRequestBuilder withAssignments(List assignments) { + this.assignments = assignments; + return this; + } + + + SyncGroupRequestData build() { + return new SyncGroupRequestData() + .setGroupId(groupId) + .setGroupInstanceId(groupInstanceId) + .setMemberId(memberId) + .setGenerationId(generationId) + .setProtocolType(protocolType) + .setProtocolName(protocolName) + .setAssignments(assignments); + } + } + + private RebalanceResult staticMembersJoinAndRebalance( + GroupMetadataManagerTestContext context, + String groupId, + String leaderInstanceId, + String followerInstanceId + ) throws Exception { + return staticMembersJoinAndRebalance( + context, + groupId, + leaderInstanceId, + followerInstanceId, + 10000, + 5000 + ); + } + + private RebalanceResult staticMembersJoinAndRebalance( + GroupMetadataManagerTestContext context, + String groupId, + String leaderInstanceId, + String followerInstanceId, + int rebalanceTimeoutMs, + int sessionTimeoutMs + ) throws Exception { + GenericGroup group = context.createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId(groupId) + .withGroupInstanceId(leaderInstanceId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolType("consumer") + .withProtocolSuperset() + .withRebalanceTimeoutMs(rebalanceTimeoutMs) + .withSessionTimeoutMs(sessionTimeoutMs) + .build(); + + CompletableFuture leaderJoinResponseFuture = new CompletableFuture<>(); + CoordinatorResult result = context.sendGenericGroupJoin(joinRequest, leaderJoinResponseFuture); + + assertTrue(result.records().isEmpty()); + + CompletableFuture followerJoinResponseFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setGroupInstanceId(followerInstanceId), + followerJoinResponseFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinResponseFuture.isDone()); + assertFalse(followerJoinResponseFuture.isDone()); + + // The goal for two timer advance is to let first group initial join complete and set newMemberAdded flag to false. Next advance is + // to trigger the rebalance as needed for follower delayed join. One large time advance won't help because we could only populate one + // delayed join from purgatory and the new delayed op is created at that time and never be triggered. + context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs); + context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs); + + assertTrue(leaderJoinResponseFuture.isDone()); + assertTrue(followerJoinResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinResponseFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinResponseFuture.get().errorCode()); + assertEquals(1, leaderJoinResponseFuture.get().generationId()); + assertEquals(1, followerJoinResponseFuture.get().generationId()); + assertEquals(2, group.size()); + assertEquals(1, group.generationId()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + + String leaderId = leaderJoinResponseFuture.get().memberId(); + List assignments = new ArrayList<>(); + assignments.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId(groupId) + .withGroupInstanceId(leaderInstanceId) + .withMemberId(leaderId) + .withGenerationId(1) + .withAssignments(assignments) + .build(); + + CompletableFuture leaderSyncResponseFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest, + leaderSyncResponseFuture); + + // Simulate a successful write to the log. + assertEquals( + Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + result.records() + ); + result.appendFuture().complete(null); + + assertTrue(leaderSyncResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderSyncResponseFuture.get().errorCode()); + assertTrue(group.isInState(STABLE)); + + String followerId = followerJoinResponseFuture.get().memberId(); + CompletableFuture followerSyncResponseFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync( + syncRequest.setGroupInstanceId(followerInstanceId) + .setMemberId(followerId) + .setAssignments(Collections.emptyList()), + followerSyncResponseFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(followerSyncResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), followerSyncResponseFuture.get().errorCode()); + assertTrue(group.isInState(STABLE)); + + assertEquals(2, group.size()); + assertEquals(1, group.generationId()); + + return new RebalanceResult( + 1, + leaderId, + leaderSyncResponseFuture.get().assignment(), + followerId, + followerSyncResponseFuture.get().assignment() + ); + } + + private static class RebalanceResult { + int generationId; + String leaderId; + byte[] leaderAssignment; + String followerId; + byte[] followerAssignment; + + RebalanceResult( + int generationId, + String leaderId, + byte[] leaderAssignment, + String followerId, + byte[] followerAssignment + ) { + this.generationId = generationId; + this.leaderId = leaderId; + this.leaderAssignment = leaderAssignment; + this.followerId = followerId; + this.followerAssignment = followerAssignment; + } + } } From 9fa62de4dbcdb5b4290b24e11068bf460c6fdf57 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Thu, 20 Jul 2023 10:53:57 -0400 Subject: [PATCH 02/11] fix build --- .../org/apache/kafka/coordinator/group/GroupMetadataManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index cd5d159afb11f..350dc7195eae7 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -76,7 +76,6 @@ import org.slf4j.Logger; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; From 3272673efd3156e7b9cdb317c9a1d2648aac0a36 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Thu, 20 Jul 2023 13:13:23 -0400 Subject: [PATCH 03/11] fix test, add tests and cleanup --- .../group/ReplicatedGroupCoordinator.java | 18 ++++ .../group/GroupCoordinatorServiceTest.java | 96 +++++++++++++++++++ .../group/GroupMetadataManagerTest.java | 14 ++- 3 files changed, 124 insertions(+), 4 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java index 7c3a71056971a..344c6e8fabe03 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java @@ -183,6 +183,15 @@ public CoordinatorResult consumerGro return groupMetadataManager.consumerGroupHeartbeat(context, request); } + /** + * Handles a JoinGroup request. + * + * @param context The request context. + * @param request The actual JoinGroup request. + * + * @return A Result containing the JoinGroup response and + * a list of records to update the state machine. + */ public CoordinatorResult genericGroupJoin( RequestContext context, JoinGroupRequestData request, @@ -195,6 +204,15 @@ public CoordinatorResult genericGroupJoin( ); } + /** + * Handles a SyncGroup request. + * + * @param context The request context. + * @param request The actual SyncGroup request. + * + * @return A Result containing the SyncGroup response and + * a list of records to update the state machine. + */ public CoordinatorResult genericGroupSync( RequestContext context, SyncGroupRequestData request, diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index f2586ab8fcadd..e3b598bf168ac 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -31,6 +31,8 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.JoinGroupRequestData; import org.apache.kafka.common.message.JoinGroupResponseData; +import org.apache.kafka.common.message.SyncGroupRequestData; +import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.network.ClientInformation; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.ApiKeys; @@ -398,4 +400,98 @@ public void testJoinGroupInvalidGroupId() throws Exception { assertEquals(expectedResponse, response.get()); } + + @Test + public void testSyncGroup() { + CoordinatorRuntime runtime = mockRuntime(); + GroupCoordinatorService service = new GroupCoordinatorService( + new LogContext(), + createConfig(), + runtime + ); + + SyncGroupRequestData request = new SyncGroupRequestData() + .setGroupId("foo"); + + service.startup(() -> 1); + + when(runtime.scheduleWriteOperation( + ArgumentMatchers.eq("generic-group-sync"), + ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.any() + )).thenReturn(CompletableFuture.completedFuture( + new SyncGroupResponseData() + )); + + CompletableFuture responseFuture = service.syncGroup( + requestContext(ApiKeys.SYNC_GROUP), + request, + BufferSupplier.NO_CACHING + ); + + assertFalse(responseFuture.isDone()); + } + + @Test + public void testSyncGroupWithException() throws Exception { + CoordinatorRuntime runtime = mockRuntime(); + GroupCoordinatorService service = new GroupCoordinatorService( + new LogContext(), + createConfig(), + runtime + ); + + SyncGroupRequestData request = new SyncGroupRequestData() + .setGroupId("foo"); + + service.startup(() -> 1); + + when(runtime.scheduleWriteOperation( + ArgumentMatchers.eq("generic-group-sync"), + ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), + ArgumentMatchers.any() + )).thenReturn(FutureUtils.failedFuture(new IllegalStateException())); + + CompletableFuture future = service.syncGroup( + requestContext(ApiKeys.SYNC_GROUP), + request, + BufferSupplier.NO_CACHING + ); + + assertTrue(future.isDone()); + assertEquals( + new SyncGroupResponseData() + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()), + future.get() + ); + } + + @Test + public void testSyncGroupInvalidGroupId() throws Exception { + CoordinatorRuntime runtime = mockRuntime(); + GroupCoordinatorService service = new GroupCoordinatorService( + new LogContext(), + createConfig(), + runtime + ); + + service.startup(() -> 1); + + SyncGroupRequestData request = new SyncGroupRequestData() + .setGroupId(null) + .setMemberId(UNKNOWN_MEMBER_ID); + + + CompletableFuture response = service.syncGroup( + requestContext(ApiKeys.SYNC_GROUP), + request, + BufferSupplier.NO_CACHING + ); + + assertTrue(response.isDone()); + SyncGroupResponseData expectedResponse = new SyncGroupResponseData() + .setErrorCode(Errors.INVALID_GROUP_ID.code()); + + assertEquals(expectedResponse, response.get()); + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 7cefc7d3b3279..371cfb5a7d98e 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -7754,7 +7754,7 @@ public void testLeaderFailureInSyncGroup() throws Exception { int nextGenerationId = leaderJoinFuture.get().generationId(); String followerId = followerJoinFuture.get().memberId(); - // With no leader SyncGroup, the follower's request should fail with an error indicating + // With no leader SyncGroup, the follower's sync request should fail with an error indicating // that it should rejoin CompletableFuture followerSyncFuture = new CompletableFuture<>(); result = context.sendGenericGroupSync( @@ -7765,12 +7765,18 @@ public void testLeaderFailureInSyncGroup() throws Exception { assertTrue(result.records().isEmpty()); assertFalse(followerSyncFuture.isDone()); - // Advance clock by session timeout to expire follower heartbeat and prepare rebalance. - // This should complete all pending syncs. - assertNoOrEmptyResult(context.sleep(5000)); + // Advance clock by session timeout to expire leader heartbeat and prepare rebalance. + // This should complete follower's sync response. The follower's heartbeat expiration will not kick + // the follower out because it is awaiting sync. + List> timeouts = context.sleep(10000); + assertTrue(timeouts.size() <= 2); + timeouts.forEach(timeout -> assertTrue(timeout.result.records().isEmpty())); assertTrue(followerSyncFuture.isDone()); assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), followerSyncFuture.get().errorCode()); + assertEquals(1, group.size()); + assertTrue(group.hasMemberId(followerId)); + assertTrue(group.isInState(PREPARING_REBALANCE)); } @Test From 46c73b0bb81cd41c4e52c0e4d7bc349036a3d369 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Thu, 20 Jul 2023 16:18:53 -0400 Subject: [PATCH 04/11] update assignment soft state only after successful commit --- .../group/GroupMetadataManager.java | 46 +++++++++++++------ .../group/generic/GenericGroupMember.java | 3 +- .../group/GroupMetadataManagerTest.java | 3 +- 3 files changed, 34 insertions(+), 18 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 350dc7195eae7..399af79f12bde 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -102,6 +102,7 @@ import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionRecord; import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionTombstoneRecord; import static org.apache.kafka.coordinator.group.RecordHelpers.newTargetAssignmentTombstoneRecord; +import static org.apache.kafka.coordinator.group.generic.GenericGroupMember.EMPTY_ASSIGNMENT; import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE; import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD; import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY; @@ -2381,10 +2382,28 @@ private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors err " state but is in " + group.currentState() + "."); } - group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT)); + group.allMembers().forEach(member -> member.setAssignment(EMPTY_ASSIGNMENT)); propagateAssignment(group, error); } + /** + * Sets assignment for group and propagate assignment and error to all members. + * + * @param group The group. + * @param assignment The assignment for all members. + */ + private void setAndPropagateAssignment(GenericGroup group, Map assignment) { + if (!group.isInState(COMPLETING_REBALANCE)) { + throw new IllegalStateException("The group must be in CompletingRebalance state " + + "to set and propagate assignment."); + } + + group.allMembers().forEach(member -> + member.setAssignment(assignment.getOrDefault(member.memberId(), EMPTY_ASSIGNMENT))); + + propagateAssignment(group, Errors.NONE); + } + /** * Propagate assignment and error to all members. * @@ -2714,23 +2733,20 @@ public CoordinatorResult genericGroupSync( "The group has {} members, {} of which are static.", memberId, groupId, group.generationId(), group.size(), group.allStaticMemberIds().size()); - // Fill all members with corresponding assignment. Reset members not specified in - // the assignment to empty assignments. - Map assignments = new HashMap<>(); - request.assignments().forEach(assignment -> - assignments.put(assignment.memberId(), assignment.assignment()) + // Fill all members with corresponding member assignment. If the member assignment + // does not exist, fill with an empty assignment. + Map assignment = new HashMap<>(); + request.assignments().forEach(memberAssignment -> + assignment.put(memberAssignment.memberId(), memberAssignment.assignment()) ); - Set membersWithMissingAssignment = new HashSet<>(); + Map membersWithMissingAssignment = new HashMap<>(); group.allMembers().forEach(member -> { - byte[] assignment = assignments.get(member.memberId()); - if (assignment != null) { - member.setAssignment(assignment); - } else { - membersWithMissingAssignment.add(member.memberId()); - member.setAssignment(new byte[0]); + if (!assignment.containsKey(member.memberId())) { + membersWithMissingAssignment.put(member.memberId(), EMPTY_ASSIGNMENT); } }); + assignment.putAll(membersWithMissingAssignment); if (!membersWithMissingAssignment.isEmpty()) { log.warn("Setting empty assignments for members {} of {} for generation {}.", @@ -2749,8 +2765,8 @@ public CoordinatorResult genericGroupSync( maybePrepareRebalanceOrCompleteJoin(group, "Error " + error + " when storing group assignment" + "during SyncGroup (member: " + memberId + ")."); } else { - // Members' assignments were already updated. Propagate and transition to Stable. - propagateAssignment(group, Errors.NONE); + // Update group's assignment and propagate to all members. + setAndPropagateAssignment(group, assignment); group.transitionTo(STABLE); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java index 3e0386e71cdac..64c3fbcad5b78 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocolCollection; import org.apache.kafka.common.message.JoinGroupResponseData; import org.apache.kafka.common.message.SyncGroupResponseData; +import org.apache.kafka.common.utils.Bytes; import java.util.HashSet; import java.util.Optional; @@ -55,7 +56,7 @@ public class GenericGroupMember { /** * An empty assignment. */ - public static final byte[] EMPTY_ASSIGNMENT = new byte[0]; + public static final byte[] EMPTY_ASSIGNMENT = Bytes.EMPTY; /** * The member id. diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 371cfb5a7d98e..02d35db2a40f6 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -48,7 +48,6 @@ import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.coordinator.group.MockCoordinatorTimer.ExpiredTimeout; @@ -7566,7 +7565,7 @@ public void testSyncGroupFromUnknownMember() throws Exception { assertTrue(result.records().isEmpty()); assertTrue(syncFuture.isDone()); assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncFuture.get().errorCode()); - assertEquals(Bytes.EMPTY, syncFuture.get().assignment()); + assertEquals(GenericGroupMember.EMPTY_ASSIGNMENT, syncFuture.get().assignment()); } @Test From 9afa66aaed3fb5a20ef1f81e245269ddd2357bfe Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Thu, 20 Jul 2023 16:27:10 -0400 Subject: [PATCH 05/11] remove generation id check in removePendingSyncMember --- .../group/GroupMetadataManager.java | 33 ++++++++----------- .../group/generic/GenericGroup.java | 22 ------------- 2 files changed, 13 insertions(+), 42 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 399af79f12bde..57e9ec2d7712c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -2020,7 +2020,6 @@ private CoordinatorResult completeGenericGroupJoin( * @param group The group. */ private void schedulePendingSync(GenericGroup group) { - group.setPendingSyncGenerationId(group.generationId()); timer.schedule(genericGroupSyncKey(group.groupId()), group.rebalanceTimeoutMs(), TimeUnit.MILLISECONDS, @@ -2854,27 +2853,21 @@ private void removePendingSyncMember( String memberId ) { group.removePendingSyncMember(memberId); - String syncKey = genericGroupSyncKey(group.groupId()); - - if (group.generationId() != group.pendingSyncGenerationId()) { - timer.cancel(syncKey); - } else { - switch (group.currentState()) { - case DEAD: - case EMPTY: - case PREPARING_REBALANCE: + switch (group.currentState()) { + case DEAD: + case EMPTY: + case PREPARING_REBALANCE: + timer.cancel(syncKey); + break; + case COMPLETING_REBALANCE: + case STABLE: + if (group.hasReceivedSyncFromAllMembers()) { timer.cancel(syncKey); - break; - case COMPLETING_REBALANCE: - case STABLE: - if (group.hasReceivedSyncFromAllMembers()) { - timer.cancel(syncKey); - } - break; - default: - throw new IllegalStateException("Unknown group state: " + group.stateAsString()); - } + } + break; + default: + throw new IllegalStateException("Unknown group state: " + group.stateAsString()); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java index 7f76b0694c949..30fd963bef86b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java @@ -126,12 +126,6 @@ public class GenericGroup implements Group { */ private int generationId; - /** - * The group's generation id when the current - * pending sync operation was scheduled. - */ - private int pendingSyncGenerationId = -1; - /** * The id of the group's leader. */ @@ -257,13 +251,6 @@ public int generationId() { return this.generationId; } - /** - * @return the pending sync generation id. - */ - public int pendingSyncGenerationId() { - return this.pendingSyncGenerationId; - } - /** * @return the protocol name. */ @@ -375,15 +362,6 @@ public void setSubscribedTopics(Optional> subscribedTopics) { this.subscribedTopics = subscribedTopics; } - /** - * Sets pendingSyncGenerationId. - * - * @param generationId the value to set. - */ - public void setPendingSyncGenerationId(int generationId) { - this.pendingSyncGenerationId = generationId; - } - /** * @return whether the group is using the consumer protocol. */ From f2cb2ca670d11770277935af63ddcd0651516090 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Thu, 20 Jul 2023 16:28:47 -0400 Subject: [PATCH 06/11] remove empty line --- .../org/apache/kafka/coordinator/group/GroupMetadataManager.java | 1 - 1 file changed, 1 deletion(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 57e9ec2d7712c..804938e4029a5 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -2836,7 +2836,6 @@ private Optional validateSyncGroup( } else { if (request.generationId() != group.generationId()) { return Optional.of(Errors.ILLEGAL_GENERATION); - } else if (isProtocolInconsistent(request.protocolType(), group.protocolType().orElse(null)) || isProtocolInconsistent(request.protocolName(), group.protocolName().orElse(null))) { From a977981ece3301b7a6435b20b7612ee07a91bff7 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Thu, 20 Jul 2023 20:42:51 -0400 Subject: [PATCH 07/11] update group metadata record with new assignment --- .../group/GroupMetadataManager.java | 10 +- .../coordinator/group/RecordHelpers.java | 10 +- .../group/generic/GenericGroup.java | 9 + .../group/GroupMetadataManagerTest.java | 159 ++++++++++-------- .../coordinator/group/RecordHelpersTest.java | 10 +- 5 files changed, 117 insertions(+), 81 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 804938e4029a5..8f03c3799b4b8 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -1973,9 +1973,8 @@ private CoordinatorResult completeGenericGroupJoin( } }); - List records = Collections.singletonList( - RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion()) - ); + List records = Collections.singletonList(RecordHelpers.newGroupMetadataRecord( + group, Collections.emptyMap(), metadataImage.features().metadataVersion())); return new CoordinatorResult<>(records, appendFuture); @@ -2607,6 +2606,7 @@ private CoordinatorResult updateStaticMemberThenRebalanceOrComplet log.info("Static member which joins during Stable stage and doesn't affect " + "the selected protocol will not trigger a rebalance."); + Map groupAssignment = group.groupAssignment(); CompletableFuture appendFuture = new CompletableFuture<>(); appendFuture.whenComplete((__, t) -> { if (t != null) { @@ -2656,7 +2656,7 @@ private CoordinatorResult updateStaticMemberThenRebalanceOrComplet }); List records = Collections.singletonList( - RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion()) + RecordHelpers.newGroupMetadataRecord(group, groupAssignment, metadataImage.features().metadataVersion()) ); return new CoordinatorResult<>(records, appendFuture); @@ -2772,7 +2772,7 @@ public CoordinatorResult genericGroupSync( }); List records = Collections.singletonList( - RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion()) + RecordHelpers.newGroupMetadataRecord(group, assignment, metadataImage.features().metadataVersion()) ); return new CoordinatorResult<>(records, appendFuture); } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java index 863be1070a414..fba9df8d5f0b2 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java @@ -366,11 +366,13 @@ public static Record newCurrentAssignmentTombstoneRecord( * Creates a GroupMetadata record. * * @param group The generic group. + * @param assignment The generic group assignment. * @param metadataVersion The metadata version. * @return The record. */ public static Record newGroupMetadataRecord( GenericGroup group, + Map assignment, MetadataVersion metadataVersion ) { List members = new ArrayList<>(group.allMembers().size()); @@ -380,10 +382,10 @@ public static Record newGroupMetadataRecord( throw new IllegalStateException("Attempted to write non-empty group metadata with no defined protocol."); } - byte[] assignment = member.assignment(); - if (assignment == null) { + byte[] memberAssignment = assignment.get(member.memberId()); + if (memberAssignment == null) { throw new IllegalStateException("Attempted to write member " + member.memberId() + - " of group + " + group.groupId() + " with no assignment."); + " of group " + group.groupId() + " with no assignment."); } members.add( @@ -395,7 +397,7 @@ public static Record newGroupMetadataRecord( .setSessionTimeout(member.sessionTimeoutMs()) .setGroupInstanceId(member.groupInstanceId().orElse(null)) .setSubscription(subscription) - .setAssignment(assignment) + .setAssignment(memberAssignment) ); }); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java index 30fd963bef86b..813e2da304d03 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java @@ -1075,6 +1075,15 @@ public ListGroupsResponseData.ListedGroup asListedGroup() { .setGroupState(state.toString()); } + /** + * @return All member assignments keyed by member id. + */ + public Map groupAssignment() { + return allMembers().stream().collect(Collectors.toMap( + GenericGroupMember::memberId, GenericGroupMember::assignment + )); + } + /** * Checks whether the transition to the target state is valid. * diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 02d35db2a40f6..cf5b562140dba 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -4531,20 +4531,12 @@ public void testHeartbeatExpirationShouldRemoveMember() throws Exception { // A group that transitions to Empty after completing join phase will generate records. List> timeouts = context.sleep(context.genericGroupNewMemberJoinTimeoutMs); - List expectedRecords = Collections.singletonList(newGroupMetadataRecord("group-id", - new GroupMetadataValue() - .setMembers(Collections.emptyList()) - .setGeneration(1) - .setLeader(null) - .setProtocolType("consumer") - .setProtocol(null) - .setCurrentStateTimestamp(context.time.milliseconds()), - MetadataVersion.latest())); - assertEquals(1, timeouts.size()); timeouts.forEach(timeout -> { assertEquals(genericGroupHeartbeatKey("group-id", memberId), timeout.key); - assertEquals(expectedRecords, timeout.result.records()); + assertEquals(Collections.singletonList( + RecordHelpers.newGroupMetadataRecord(group, Collections.emptyMap(), MetadataVersion.latest())), + timeout.result.records()); }); assertTrue(responseFuture.isDone()); @@ -5075,7 +5067,7 @@ public void testReplaceStaticMemberInStableStateNoError( supportSkippingAssignment); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); assertFalse(responseFuture.isDone()); @@ -5230,7 +5222,7 @@ public void testReplaceStaticMemberInStableStateErrors() throws Exception { false); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); assertFalse(responseFuture.isDone()); @@ -5321,7 +5313,7 @@ public void testReplaceStaticMemberInStableStateSucceeds( supportSkippingAssignment); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); assertFalse(responseFuture.isDone()); @@ -5714,7 +5706,7 @@ public void testNewMemberTimeoutCompletion() throws Exception { result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate a successful write to the log. @@ -5786,7 +5778,7 @@ public void testNewMemberFailureAfterJoinGroupCompletion() throws Exception { CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncResponseFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate a successful write to the log. @@ -6150,7 +6142,7 @@ public void testStaticMemberRejoinWithKnownMemberId() throws Exception { result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Successful write to the log. @@ -6194,7 +6186,7 @@ public void testStaticMemberRejoinWithLeaderIdAndUnknownMemberId( supportSkippingAssignment); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate a successful write to the log. @@ -6514,7 +6506,7 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel true); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate a failed write to the log. @@ -6571,14 +6563,14 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel CompletableFuture leaderSyncFuture = new CompletableFuture<>(); result = context.sendGenericGroupSync(syncRequest, leaderSyncFuture); + // Simulate a successful write to the log. This will update the group with the new (empty) assignment. + result.appendFuture().complete(null); + assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate a successful write to the log. - result.appendFuture().complete(null); - assertTrue(leaderSyncFuture.isDone()); assertTrue(group.isInState(STABLE)); assertEquals(Errors.NONE.code(), leaderSyncFuture.get().errorCode()); @@ -6637,7 +6629,7 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel true); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); @@ -6679,7 +6671,7 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel .withGroupInstanceId("follower-instance-id") .withGenerationId(rebalanceResult.generationId) .withMemberId(rebalanceResult.followerId) - .withAssignments(Collections.emptyList()) + .withAssignment(Collections.emptyList()) .build(); CoordinatorResult syncResult = context.sendGenericGroupSync(syncRequest, syncFuture); @@ -6870,11 +6862,12 @@ public void testStaticMemberRejoinAsFollowerWithUnknownMemberId() throws Excepti true); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. result.appendFuture().complete(null); + assertTrue(followerJoinFuture.isDone()); // Old leader shouldn't be timed out. @@ -7093,7 +7086,7 @@ public void testGetDifferentStaticMemberIdAfterEachRejoin() throws Exception { true); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -7458,15 +7451,15 @@ private void testSyncGroupProtocolTypeAndNameWith( int generationId = leaderJoinFuture.get().generationId(); // SyncGroup with the provided Protocol Type and Name - List assignments = new ArrayList<>(); - assignments.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() .withGroupId("group-id") .withMemberId(leaderId) .withProtocolType(protocolType.orElse(null)) .withProtocolName(protocolName.orElse(null)) .withGenerationId(generationId) - .withAssignments(assignments) + .withAssignment(assignment) .build(); CompletableFuture leaderSyncFuture = new CompletableFuture<>(); @@ -7474,7 +7467,7 @@ private void testSyncGroupProtocolTypeAndNameWith( if (expectedError == Errors.NONE) { assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -7536,20 +7529,20 @@ public void testSyncGroupFromUnknownMember() throws Exception { String memberId = joinResponse.memberId(); int generationId = joinResponse.generationId(); - List assignments = new ArrayList<>(); - assignments.add(new SyncGroupRequestAssignment().setMemberId(memberId)); + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment().setMemberId(memberId)); SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() .withGroupId("group-id") .withMemberId(memberId) .withGenerationId(generationId) - .withAssignments(assignments) + .withAssignment(assignment) .build(); CompletableFuture syncFuture = new CompletableFuture<>(); CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -7557,7 +7550,7 @@ public void testSyncGroupFromUnknownMember() throws Exception { assertTrue(syncFuture.isDone()); assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - assertEquals(assignments.get(0).assignment(), syncFuture.get().assignment()); + assertEquals(assignment.get(0).assignment(), syncFuture.get().assignment()); syncFuture = new CompletableFuture<>(); result = context.sendGenericGroupSync(syncRequest.setMemberId("unknown-member-id"), syncFuture); @@ -7633,7 +7626,7 @@ public void testJoinGroupFromUnchangedFollowerDoesNotRebalance() throws Exceptio CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -7716,7 +7709,7 @@ public void testLeaderFailureInSyncGroup() throws Exception { CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -7812,7 +7805,7 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -7852,12 +7845,12 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { byte[] followerAssignment = new byte[]{1}; // Sync group with leader to get new assignment. - List assignments = new ArrayList<>(); - assignments.add(new SyncGroupRequestAssignment() + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment() .setMemberId(leaderId) .setAssignment(leaderAssignment) ); - assignments.add(new SyncGroupRequestAssignment() + assignment.add(new SyncGroupRequestAssignment() .setMemberId(followerId) .setAssignment(followerAssignment) ); @@ -7865,16 +7858,17 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { syncFuture = new CompletableFuture<>(); result = context.sendGenericGroupSync( syncRequest.setGenerationId(nextGenerationId) - .setAssignments(assignments), + .setAssignments(assignment), syncFuture ); + // Simulate successful write to log. This will update the group's assignment with the new assignment. + result.appendFuture().complete(null); + assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. - result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); @@ -7928,7 +7922,7 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -7978,12 +7972,12 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { assertFalse(followerSyncFuture.isDone()); // Sync group with leader to get new assignment. - List assignments = new ArrayList<>(); - assignments.add(new SyncGroupRequestAssignment() + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment() .setMemberId(leaderId) .setAssignment(leaderAssignment) ); - assignments.add(new SyncGroupRequestAssignment() + assignment.add(new SyncGroupRequestAssignment() .setMemberId(followerId) .setAssignment(followerAssignment) ); @@ -7992,16 +7986,22 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { result = context.sendGenericGroupSync( syncRequest.setMemberId(leaderId) .setGenerationId(nextGenerationId) - .setAssignments(assignments), + .setAssignments(assignment), syncFuture ); + // Simulate successful write to log. This will update the group assignment with the new assignment. + result.appendFuture().complete(null); + + Map updatedAssignment = assignment.stream().collect(Collectors.toMap( + SyncGroupRequestAssignment::memberId, SyncGroupRequestAssignment::assignment + )); + assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList( + RecordHelpers.newGroupMetadataRecord(group, updatedAssignment, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. - result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); @@ -8041,7 +8041,7 @@ public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception { CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -8051,7 +8051,7 @@ public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception { assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); // Join group from the leader should force the group to rebalance, which allows the - // leader to push new assignments when local metadata changes + // leader to push new assignment when local metadata changes CompletableFuture leaderJoinFuture = new CompletableFuture<>(); result = context.sendGenericGroupJoin( joinRequest.setMemberId(leaderId), @@ -8130,8 +8130,8 @@ private JoinGroupResponseData setupGroupWithPendingMember( String leaderId = leaderJoinResponse.memberId(); int generationId = leaderJoinResponse.generationId(); - List assignments = new ArrayList<>(); - assignments.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() .withGroupId("group-id") .withMemberId(leaderId) @@ -8143,7 +8143,7 @@ private JoinGroupResponseData setupGroupWithPendingMember( // Now the group is stable, with the one member that joined above assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -8186,7 +8186,7 @@ private JoinGroupResponseData setupGroupWithPendingMember( result = context.sendGenericGroupSync(syncRequest.setGenerationId(nextGenerationId), syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -8265,7 +8265,7 @@ public void testGenerationIdIncrementsOnRebalance() throws Exception { CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); // Simulate successful write to log. @@ -8354,7 +8354,7 @@ private static class SyncGroupRequestBuilder { String protocolType = "consumer"; String protocolName = "range"; int generationId = 0; - List assignments = Collections.emptyList(); + List assignment = Collections.emptyList(); JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); SyncGroupRequestBuilder withGroupId(String groupId) { @@ -8387,8 +8387,8 @@ SyncGroupRequestBuilder withProtocolName(String protocolName) { return this; } - SyncGroupRequestBuilder withAssignments(List assignments) { - this.assignments = assignments; + SyncGroupRequestBuilder withAssignment(List assignment) { + this.assignment = assignment; return this; } @@ -8401,7 +8401,7 @@ SyncGroupRequestData build() { .setGenerationId(generationId) .setProtocolType(protocolType) .setProtocolName(protocolName) - .setAssignments(assignments); + .setAssignments(assignment); } } @@ -8472,14 +8472,19 @@ private RebalanceResult staticMembersJoinAndRebalance( assertTrue(group.isInState(COMPLETING_REBALANCE)); String leaderId = leaderJoinResponseFuture.get().memberId(); - List assignments = new ArrayList<>(); - assignments.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); + String followerId = followerJoinResponseFuture.get().memberId(); + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderId) + .setAssignment(new byte[]{1})); + assignment.add(new SyncGroupRequestAssignment().setMemberId(followerId) + .setAssignment(new byte[]{2})); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() .withGroupId(groupId) .withGroupInstanceId(leaderInstanceId) .withMemberId(leaderId) .withGenerationId(1) - .withAssignments(assignments) + .withAssignment(assignment) .build(); CompletableFuture leaderSyncResponseFuture = new CompletableFuture<>(); @@ -8487,18 +8492,23 @@ private RebalanceResult staticMembersJoinAndRebalance( syncRequest, leaderSyncResponseFuture); - // Simulate a successful write to the log. + // The generated record should contain the new assignment. + Map groupAssignment = assignment.stream().collect(Collectors.toMap( + SyncGroupRequestAssignment::memberId, SyncGroupRequestAssignment::assignment + )); assertEquals( - Collections.singletonList(RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest())), + Collections.singletonList( + RecordHelpers.newGroupMetadataRecord(group, groupAssignment, MetadataVersion.latest())), result.records() ); + + // Simulate a successful write to the log. result.appendFuture().complete(null); assertTrue(leaderSyncResponseFuture.isDone()); assertEquals(Errors.NONE.code(), leaderSyncResponseFuture.get().errorCode()); assertTrue(group.isInState(STABLE)); - String followerId = followerJoinResponseFuture.get().memberId(); CompletableFuture followerSyncResponseFuture = new CompletableFuture<>(); result = context.sendGenericGroupSync( syncRequest.setGroupInstanceId(followerInstanceId) @@ -8524,6 +8534,13 @@ private RebalanceResult staticMembersJoinAndRebalance( ); } + private static Record newGroupMetadataRecord( + GenericGroup group, + MetadataVersion metadataVersion + ) { + return RecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), metadataVersion); + } + private static class RebalanceResult { int generationId; String leaderId; diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java index 2b436672b0ed5..41ea9777ec1e0 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java @@ -55,6 +55,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -467,6 +468,8 @@ public void testNewGroupMetadataRecord( time ); + Map assignment = new HashMap<>(); + expectedMembers.forEach(member -> { JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(); protocols.add(new JoinGroupRequestProtocol() @@ -482,13 +485,16 @@ public void testNewGroupMetadataRecord( member.sessionTimeout(), "consumer", protocols, - member.assignment() + GenericGroupMember.EMPTY_ASSIGNMENT )); + + assignment.put(member.memberId(), member.assignment()); }); group.initNextGeneration(); Record groupMetadataRecord = RecordHelpers.newGroupMetadataRecord( group, + assignment, metadataVersion ); @@ -554,6 +560,7 @@ public void testNewGroupMetadataRecordThrowsWhenNullSubscription() { assertThrows(IllegalStateException.class, () -> RecordHelpers.newGroupMetadataRecord( group, + Collections.emptyMap(), MetadataVersion.IBP_3_5_IV2 )); } @@ -604,6 +611,7 @@ public void testNewGroupMetadataRecordThrowsWhenEmptyAssignment() { assertThrows(IllegalStateException.class, () -> RecordHelpers.newGroupMetadataRecord( group, + Collections.emptyMap(), MetadataVersion.IBP_3_5_IV2 )); } From b000fa3ce5cb8251fe1d9d62ffc0f04b90f420f9 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Fri, 21 Jul 2023 13:52:29 -0400 Subject: [PATCH 08/11] address comments --- .../group/GroupMetadataManagerTest.java | 505 +++++++++--------- 1 file changed, 241 insertions(+), 264 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index cf5b562140dba..7de22259f9ce5 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -466,11 +466,6 @@ public List> sleep(long ms) { return timeouts; } - public void sleepAndAssertEmptyResult(long ms) { - List> timeouts = sleep(ms); - timeouts.forEach(timeout -> assertEquals(EMPTY_RESULT, timeout.result)); - } - public ScheduledTimeout assertSessionTimeout( String groupId, String memberId, @@ -684,6 +679,133 @@ public CoordinatorResult sendGenericGroupSync( ); } + + public RebalanceResult staticMembersJoinAndRebalance( + String groupId, + String leaderInstanceId, + String followerInstanceId + ) throws Exception { + return staticMembersJoinAndRebalance( + groupId, + leaderInstanceId, + followerInstanceId, + 10000, + 5000 + ); + } + + public RebalanceResult staticMembersJoinAndRebalance( + String groupId, + String leaderInstanceId, + String followerInstanceId, + int rebalanceTimeoutMs, + int sessionTimeoutMs + ) throws Exception { + GenericGroup group = createGenericGroup("group-id"); + + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId(groupId) + .withGroupInstanceId(leaderInstanceId) + .withMemberId(UNKNOWN_MEMBER_ID) + .withProtocolType("consumer") + .withProtocolSuperset() + .withRebalanceTimeoutMs(rebalanceTimeoutMs) + .withSessionTimeoutMs(sessionTimeoutMs) + .build(); + + CompletableFuture leaderJoinResponseFuture = new CompletableFuture<>(); + CoordinatorResult result = sendGenericGroupJoin(joinRequest, leaderJoinResponseFuture); + + assertTrue(result.records().isEmpty()); + + CompletableFuture followerJoinResponseFuture = new CompletableFuture<>(); + result = sendGenericGroupJoin( + joinRequest.setGroupInstanceId(followerInstanceId), + followerJoinResponseFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinResponseFuture.isDone()); + assertFalse(followerJoinResponseFuture.isDone()); + + // The goal for two timer advance is to let first group initial join complete and set newMemberAdded flag to false. Next advance is + // to trigger the rebalance as needed for follower delayed join. One large time advance won't help because we could only populate one + // delayed join from purgatory and the new delayed op is created at that time and never be triggered. + assertNoOrEmptyResult(sleep(genericGroupInitialRebalanceDelayMs)); + assertNoOrEmptyResult(sleep(genericGroupInitialRebalanceDelayMs)); + + assertTrue(leaderJoinResponseFuture.isDone()); + assertTrue(followerJoinResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinResponseFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinResponseFuture.get().errorCode()); + assertEquals(1, leaderJoinResponseFuture.get().generationId()); + assertEquals(1, followerJoinResponseFuture.get().generationId()); + assertEquals(2, group.size()); + assertEquals(1, group.generationId()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + + String leaderId = leaderJoinResponseFuture.get().memberId(); + String followerId = followerJoinResponseFuture.get().memberId(); + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderId) + .setAssignment(new byte[]{1})); + assignment.add(new SyncGroupRequestAssignment().setMemberId(followerId) + .setAssignment(new byte[]{2})); + + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId(groupId) + .withGroupInstanceId(leaderInstanceId) + .withMemberId(leaderId) + .withGenerationId(1) + .withAssignment(assignment) + .build(); + + CompletableFuture leaderSyncResponseFuture = new CompletableFuture<>(); + result = sendGenericGroupSync( + syncRequest, + leaderSyncResponseFuture); + + // The generated record should contain the new assignment. + Map groupAssignment = assignment.stream().collect(Collectors.toMap( + SyncGroupRequestAssignment::memberId, SyncGroupRequestAssignment::assignment + )); + assertEquals( + Collections.singletonList( + RecordHelpers.newGroupMetadataRecord(group, groupAssignment, MetadataVersion.latest())), + result.records() + ); + + // Simulate a successful write to the log. + result.appendFuture().complete(null); + + assertTrue(leaderSyncResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderSyncResponseFuture.get().errorCode()); + assertTrue(group.isInState(STABLE)); + + CompletableFuture followerSyncResponseFuture = new CompletableFuture<>(); + result = sendGenericGroupSync( + syncRequest.setGroupInstanceId(followerInstanceId) + .setMemberId(followerId) + .setAssignments(Collections.emptyList()), + followerSyncResponseFuture + ); + + assertTrue(result.records().isEmpty()); + assertTrue(followerSyncResponseFuture.isDone()); + assertEquals(Errors.NONE.code(), followerSyncResponseFuture.get().errorCode()); + assertTrue(group.isInState(STABLE)); + + assertEquals(2, group.size()); + assertEquals(1, group.generationId()); + + return new RebalanceResult( + 1, + leaderId, + leaderSyncResponseFuture.get().assignment(), + followerId, + followerSyncResponseFuture.get().assignment() + ); + } + private ApiMessage messageOrNull(ApiMessageAndVersion apiMessageAndVersion) { if (apiMessageAndVersion == null) { return null; @@ -5172,7 +5294,7 @@ public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebal @Test public void testReplaceStaticMemberInStableStateErrors() throws Exception { - // If the append future fails, we need to revert the soft state to the original member. + // If the append future fails, confirm that the member is not updated. GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); GenericGroup group = context.createGenericGroup("group-id"); @@ -5205,7 +5327,7 @@ public void testReplaceStaticMemberInStableStateErrors() throws Exception { // Simulate successful sync group phase group.transitionTo(STABLE); - // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. This reverts the soft state of the group. + // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. The group should not be updated. protocols.add(new JoinGroupRequestProtocol() .setName("roundrobin") .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( @@ -5561,7 +5683,7 @@ private List verifyGenericGroupJoinResponses( return memberIds; } - private void assertNoOrEmptyResult(List> timeouts) { + private static void assertNoOrEmptyResult(List> timeouts) { assertTrue(timeouts.size() <= 1); timeouts.forEach(timeout -> assertEquals(EMPTY_RESULT, timeout.result)); } @@ -5689,7 +5811,7 @@ public void testNewMemberTimeoutCompletion() throws Exception { assertTrue(result.records().isEmpty()); assertFalse(joinFuture.isDone()); - context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs); + assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs)); assertTrue(joinFuture.isDone()); assertEquals(Errors.NONE.code(), joinFuture.get().errorCode()); @@ -5717,20 +5839,21 @@ public void testNewMemberTimeoutCompletion() throws Exception { assertEquals(1, group.size()); // Make sure the NewMemberTimeout is not still in effect, and the member is not kicked - context.sleepAndAssertEmptyResult(context.genericGroupNewMemberJoinTimeoutMs); + assertNoOrEmptyResult(context.sleep(context.genericGroupNewMemberJoinTimeoutMs)); assertEquals(1, group.size()); // Member should be removed as heartbeat expires. List> timeouts = context.sleep(5000); - List expectedRecords = Collections.singletonList(newGroupMetadataRecord("group-id", - new GroupMetadataValue() - .setMembers(Collections.emptyList()) - .setGeneration(2) - .setLeader(null) - .setProtocolType("consumer") - .setProtocol(null) - .setCurrentStateTimestamp(context.time.milliseconds()), - MetadataVersion.latest())); + List expectedRecords = Collections.singletonList(newGroupMetadataRecord( + "group-id", + new GroupMetadataValue() + .setMembers(Collections.emptyList()) + .setGeneration(2) + .setLeader(null) + .setProtocolType("consumer") + .setProtocol(null) + .setCurrentStateTimestamp(context.time.milliseconds()), + MetadataVersion.latest())); assertEquals(1, timeouts.size()); String memberId = joinFuture.get().memberId(); @@ -5804,6 +5927,8 @@ public void testNewMemberFailureAfterJoinGroupCompletion() throws Exception { assertTrue(joinResult.records().isEmpty()); assertTrue(joinFuture.isDone()); assertTrue(otherJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), joinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), otherJoinFuture.get().errorCode()); verifySessionExpiration(context, group, 5000); } @@ -5814,8 +5939,7 @@ public void testStaticMemberFenceDuplicateRejoinedFollower() throws Exception { .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -5875,8 +5999,7 @@ public void testStaticMemberFenceDuplicateSyncingFollowerAfterMemberIdChanged() .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -5887,7 +6010,7 @@ public void testStaticMemberFenceDuplicateSyncingFollowerAfterMemberIdChanged() .withGroupId("group-id") .withGroupInstanceId("leader-instance-id") .withMemberId(rebalanceResult.leaderId) - .withDefaultProtocolTypeAndProtocols() + .withProtocolSuperset() .withRebalanceTimeoutMs(10000) .build(); @@ -5917,16 +6040,12 @@ public void testStaticMemberFenceDuplicateSyncingFollowerAfterMemberIdChanged() .setProtocolType("consumer") .setMembers(toJoinResponseMembers(group)); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - checkJoinGroupResponse( expectedLeaderResponse, leaderJoinFuture.get(), group, COMPLETING_REBALANCE, - expectedGroupInstanceIds + mkSet("leader-instance-id", "follower-instance-id") ); assertEquals(rebalanceResult.leaderId, leaderJoinFuture.get().memberId()); @@ -5983,8 +6102,10 @@ public void testStaticMemberFenceDuplicateSyncingFollowerAfterMemberIdChanged() assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldFollowerSyncFuture.get().errorCode()); // Advance clock by rebalance timeout so that the join phase completes with duplicate follower. - // Leader is kicked out. - context.sleepAndAssertEmptyResult(10000); + // Both heartbeats will expire but only the leader is kicked out. + List> timeouts = context.sleep(10000); + assertTrue(timeouts.size() == 2); + timeouts.forEach(timeout -> assertEquals(timeout.result, EMPTY_RESULT)); assertTrue(duplicateFollowerJoinFuture.isDone()); assertTrue(group.isInState(COMPLETING_REBALANCE)); @@ -6000,8 +6121,7 @@ public void testStaticMemberFenceDuplicateRejoiningFollowerAfterMemberIdChanged( .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6051,16 +6171,12 @@ public void testStaticMemberFenceDuplicateRejoiningFollowerAfterMemberIdChanged( .setProtocolType("consumer") .setMembers(toJoinResponseMembers(group)); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - checkJoinGroupResponse( expectedLeaderResponse, leaderJoinFuture.get(), group, COMPLETING_REBALANCE, - expectedGroupInstanceIds + mkSet("leader-instance-id", "follower-instance-id") ); assertTrue(duplicateFollowerJoinFuture.isDone()); @@ -6162,8 +6278,7 @@ public void testStaticMemberRejoinWithLeaderIdAndUnknownMemberId( .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6175,7 +6290,7 @@ public void testStaticMemberRejoinWithLeaderIdAndUnknownMemberId( .withGroupId("group-id") .withGroupInstanceId("leader-instance-id") .withMemberId(UNKNOWN_MEMBER_ID) - .withDefaultProtocolTypeAndProtocols() + .withProtocolSuperset() .build(); CompletableFuture joinResponseFuture = new CompletableFuture<>(); @@ -6193,10 +6308,6 @@ public void testStaticMemberRejoinWithLeaderIdAndUnknownMemberId( result.appendFuture().complete(null); assertTrue(joinResponseFuture.isDone()); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - String leader = supportSkippingAssignment ? joinResponseFuture.get().memberId() : rebalanceResult.leaderId; @@ -6218,8 +6329,43 @@ public void testStaticMemberRejoinWithLeaderIdAndUnknownMemberId( joinResponseFuture.get(), group, STABLE, - supportSkippingAssignment ? expectedGroupInstanceIds : Collections.emptySet() + supportSkippingAssignment ? mkSet("leader-instance-id", "follower-instance-id") : Collections.emptySet() ); + + CompletableFuture oldLeaderJoinFuture = new CompletableFuture<>(); + result = context.sendGenericGroupJoin( + joinRequest.setMemberId(rebalanceResult.leaderId), + oldLeaderJoinFuture, + true, + supportSkippingAssignment); + + assertTrue(result.records().isEmpty()); + assertTrue(oldLeaderJoinFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldLeaderJoinFuture.get().errorCode()); + + // Old leader will get fenced. + SyncGroupRequestData oldLeaderSyncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withGroupInstanceId("leader-instance-id") + .withGenerationId(rebalanceResult.generationId) + .withMemberId(rebalanceResult.leaderId) + .build(); + + CompletableFuture oldLeaderSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(oldLeaderSyncRequest, oldLeaderSyncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(oldLeaderSyncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldLeaderSyncFuture.get().errorCode()); + + // Calling sync on old leader id will fail because that leader id is no longer valid and replaced. + SyncGroupRequestData newLeaderSyncRequest = oldLeaderSyncRequest.setGroupInstanceId(null); + CompletableFuture newLeaderSyncFuture = new CompletableFuture<>(); + result = context.sendGenericGroupSync(newLeaderSyncRequest, newLeaderSyncFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(newLeaderSyncFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), newLeaderSyncFuture.get().errorCode()); } @Test @@ -6228,8 +6374,7 @@ public void testStaticMemberRejoinWithLeaderIdAndKnownMemberId() throws Exceptio .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6246,6 +6391,9 @@ public void testStaticMemberRejoinWithLeaderIdAndKnownMemberId() throws Exceptio JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(request, true, true, 10000); + // Follower's heartbeat expires as the leader rejoins. + assertFalse(group.hasMemberId(rebalanceResult.followerId)); + JoinGroupResponseData expectedResponse = new JoinGroupResponseData() .setErrorCode(Errors.NONE.code()) .setGenerationId(rebalanceResult.generationId + 1) @@ -6270,8 +6418,7 @@ public void testStaticMemberRejoinWithLeaderIdAndUnexpectedDeadGroup() throws Ex .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6300,8 +6447,7 @@ public void testStaticMemberRejoinWithLeaderIdAndUnexpectedEmptyGroup() throws E .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6333,8 +6479,7 @@ public void testStaticMemberRejoinWithFollowerIdAndChangeOfProtocol() throws Exc .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id", @@ -6365,17 +6510,13 @@ public void testStaticMemberRejoinWithFollowerIdAndChangeOfProtocol() throws Exc assertFalse(responseFuture.isDone()); // Old leader hasn't joined in the meantime, triggering a re-election. - context.sleepAndAssertEmptyResult(rebalanceTimeoutMs); + assertNoOrEmptyResult(context.sleep(rebalanceTimeoutMs)); assertTrue(responseFuture.isDone()); assertEquals(Errors.NONE.code(), responseFuture.get().errorCode()); assertTrue(group.hasStaticMember("leader-instance-id")); assertTrue(group.isLeader(rebalanceResult.followerId)); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - JoinGroupResponseData expectedResponse = new JoinGroupResponseData() .setErrorCode(Errors.NONE.code()) .setGenerationId(rebalanceResult.generationId + 1) @@ -6390,7 +6531,7 @@ public void testStaticMemberRejoinWithFollowerIdAndChangeOfProtocol() throws Exc responseFuture.get(), group, COMPLETING_REBALANCE, - expectedGroupInstanceIds + mkSet("leader-instance-id", "follower-instance-id") ); } @@ -6404,8 +6545,7 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWithSele .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id", @@ -6439,17 +6579,13 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWithSele assertEquals("roundrobin", group.selectProtocol()); // Old leader hasn't joined in the meantime, triggering a re-election. - context.sleepAndAssertEmptyResult(rebalanceTimeoutMs); + assertNoOrEmptyResult(context.sleep(rebalanceTimeoutMs)); assertTrue(responseFuture.isDone()); assertEquals(Errors.NONE.code(), responseFuture.get().errorCode()); assertTrue(group.hasStaticMember("leader-instance-id")); assertTrue(group.isLeader(responseFuture.get().memberId())); assertNotEquals(rebalanceResult.followerId, responseFuture.get().memberId()); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - JoinGroupResponseData expectedResponse = new JoinGroupResponseData() .setErrorCode(Errors.NONE.code()) .setGenerationId(rebalanceResult.generationId + 1) @@ -6464,20 +6600,18 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWithSele responseFuture.get(), group, COMPLETING_REBALANCE, - expectedGroupInstanceIds + mkSet("leader-instance-id", "follower-instance-id") ); } @Test - public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSelectProtocolUnchangedPersistenceFailure() - throws Exception { + public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSelectProtocolUnchangedPersistenceFailure() throws Exception { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6578,11 +6712,11 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel // Sync with old member id will also not fail because the member id is not updated because of persistence failure CompletableFuture oldMemberSyncFuture = new CompletableFuture<>(); result = context.sendGenericGroupSync( - syncRequest - .setGroupInstanceId("follower-instance-id") - .setMemberId(rebalanceResult.followerId), - oldMemberSyncFuture - ); + syncRequest + .setGroupInstanceId("follower-instance-id") + .setMemberId(rebalanceResult.followerId), + oldMemberSyncFuture + ); assertTrue(result.records().isEmpty()); assertTrue(oldMemberSyncFuture.isDone()); assertTrue(group.isInState(STABLE)); @@ -6590,15 +6724,13 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel } @Test - public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSelectProtocolUnchanged() - throws Exception { + public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSelectProtocolUnchanged() throws Exception { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6680,7 +6812,6 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel assertTrue(syncFuture.isDone()); assertEquals(Errors.FENCED_INSTANCE_ID.code(), syncFuture.get().errorCode()); - // Sync with new member id succeeds syncFuture = new CompletableFuture<>(); syncResult = context.sendGenericGroupSync( @@ -6702,8 +6833,7 @@ public void testStaticMemberRejoinWithKnownLeaderIdToTriggerRebalanceAndFollower .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6755,16 +6885,12 @@ public void testStaticMemberRejoinWithKnownLeaderIdToTriggerRebalanceAndFollower .setSkipAssignment(false) .setMembers(toJoinResponseMembers(group)); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - checkJoinGroupResponse( expectedLeaderResponse, leaderJoinFuture.get(), group, COMPLETING_REBALANCE, - expectedGroupInstanceIds + mkSet("leader-instance-id", "follower-instance-id") ); JoinGroupResponseData expectedFollowerResponse = new JoinGroupResponseData() @@ -6839,8 +6965,7 @@ public void testStaticMemberRejoinAsFollowerWithUnknownMemberId() throws Excepti .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6865,7 +6990,7 @@ public void testStaticMemberRejoinAsFollowerWithUnknownMemberId() throws Excepti Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(followerJoinFuture.isDone()); @@ -6914,8 +7039,7 @@ public void testStaticMemberRejoinAsFollowerWithKnownMemberIdAndNoProtocolChange .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -6968,8 +7092,7 @@ public void testStaticMemberRejoinAsFollowerWithMismatchedInstanceId() throws Ex .build(); context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -7000,8 +7123,7 @@ public void testStaticMemberRejoinAsLeaderWithMismatchedInstanceId() throws Exce .build(); context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -7032,8 +7154,7 @@ public void testStaticMemberSyncAsLeaderWithInvalidMemberId() throws Exception { .build(); context.createGenericGroup("group-id"); - staticMembersJoinAndRebalance( - context, + context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -7062,8 +7183,7 @@ public void testGetDifferentStaticMemberIdAfterEachRejoin() throws Exception { .build(); GenericGroup group = context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -7089,7 +7209,7 @@ public void testGetDifferentStaticMemberIdAfterEachRejoin() throws Exception { Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(leaderJoinFuture.isDone()); assertEquals(group.staticMemberId("leader-instance-id"), leaderJoinFuture.get().memberId()); @@ -7106,8 +7226,7 @@ public void testStaticMemberJoinWithUnknownInstanceIdAndKnownMemberId() throws E .build(); context.createGenericGroup("group-id"); - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -7138,8 +7257,7 @@ public void testStaticMemberReJoinWithIllegalStateAsUnknownMember() throws Excep .build(); GenericGroup group = context.createGenericGroup("group-id"); - staticMembersJoinAndRebalance( - context, + context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id" @@ -7174,8 +7292,7 @@ public void testStaticMemberFollowerFailToRejoinBeforeRebalanceTimeout() throws GenericGroup group = context.createGenericGroup("group-id"); // Increase session timeout so that the follower won't be evicted when rebalance timeout is reached. - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id", @@ -7231,17 +7348,12 @@ public void testStaticMemberFollowerFailToRejoinBeforeRebalanceTimeout() throws .setSkipAssignment(false) .setMembers(toJoinResponseMembers(group)); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - expectedGroupInstanceIds.add(newMemberInstanceId); - checkJoinGroupResponse( expectedLeaderResponse, leaderJoinFuture.get(), group, COMPLETING_REBALANCE, - expectedGroupInstanceIds + mkSet("leader-instance-id", "follower-instance-id", newMemberInstanceId) ); JoinGroupResponseData expectedNewMemberResponse = new JoinGroupResponseData() @@ -7270,8 +7382,7 @@ public void testStaticMemberLeaderFailToRejoinBeforeRebalanceTimeout() throws Ex GenericGroup group = context.createGenericGroup("group-id"); // Increase session timeout so that the leader won't be evicted when rebalance timeout is reached. - RebalanceResult rebalanceResult = staticMembersJoinAndRebalance( - context, + RebalanceResult rebalanceResult = context.staticMembersJoinAndRebalance( "group-id", "leader-instance-id", "follower-instance-id", @@ -7331,17 +7442,12 @@ public void testStaticMemberLeaderFailToRejoinBeforeRebalanceTimeout() throws Ex .setSkipAssignment(false) .setMembers(toJoinResponseMembers(group)); - Set expectedGroupInstanceIds = new HashSet<>(); - expectedGroupInstanceIds.add("leader-instance-id"); - expectedGroupInstanceIds.add("follower-instance-id"); - expectedGroupInstanceIds.add(newMemberInstanceId); - checkJoinGroupResponse( expectedLeaderResponse, newLeaderResponse, group, COMPLETING_REBALANCE, - expectedGroupInstanceIds + mkSet("leader-instance-id", "follower-instance-id", newMemberInstanceId) ); JoinGroupResponseData expectedNewMemberResponse = new JoinGroupResponseData() @@ -7470,7 +7576,7 @@ private void testSyncGroupProtocolTypeAndNameWith( Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); } else { assertTrue(result.records().isEmpty()); @@ -7545,7 +7651,7 @@ public void testSyncGroupFromUnknownMember() throws Exception { Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -7629,7 +7735,7 @@ public void testJoinGroupFromUnchangedFollowerDoesNotRebalance() throws Exceptio Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -7712,7 +7818,7 @@ public void testLeaderFailureInSyncGroup() throws Exception { Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -7808,7 +7914,7 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -7862,7 +7968,7 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { syncFuture ); - // Simulate successful write to log. This will update the group's assignment with the new assignment. + // Simulate a successful write to log. This will update the group's assignment with the new assignment. result.appendFuture().complete(null); assertEquals( @@ -7925,7 +8031,7 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -7990,7 +8096,7 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { syncFuture ); - // Simulate successful write to log. This will update the group assignment with the new assignment. + // Simulate a successful write to log. This will update the group assignment with the new assignment. result.appendFuture().complete(null); Map updatedAssignment = assignment.stream().collect(Collectors.toMap( @@ -8044,7 +8150,7 @@ public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception { Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -8146,7 +8252,7 @@ private JoinGroupResponseData setupGroupWithPendingMember( Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -8189,7 +8295,7 @@ private JoinGroupResponseData setupGroupWithPendingMember( Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -8268,7 +8374,7 @@ public void testGenerationIdIncrementsOnRebalance() throws Exception { Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), result.records() ); - // Simulate successful write to log. + // Simulate a successful write to log. result.appendFuture().complete(null); assertTrue(syncFuture.isDone()); @@ -8405,135 +8511,6 @@ SyncGroupRequestData build() { } } - private RebalanceResult staticMembersJoinAndRebalance( - GroupMetadataManagerTestContext context, - String groupId, - String leaderInstanceId, - String followerInstanceId - ) throws Exception { - return staticMembersJoinAndRebalance( - context, - groupId, - leaderInstanceId, - followerInstanceId, - 10000, - 5000 - ); - } - - private RebalanceResult staticMembersJoinAndRebalance( - GroupMetadataManagerTestContext context, - String groupId, - String leaderInstanceId, - String followerInstanceId, - int rebalanceTimeoutMs, - int sessionTimeoutMs - ) throws Exception { - GenericGroup group = context.createGenericGroup("group-id"); - - JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() - .withGroupId(groupId) - .withGroupInstanceId(leaderInstanceId) - .withMemberId(UNKNOWN_MEMBER_ID) - .withProtocolType("consumer") - .withProtocolSuperset() - .withRebalanceTimeoutMs(rebalanceTimeoutMs) - .withSessionTimeoutMs(sessionTimeoutMs) - .build(); - - CompletableFuture leaderJoinResponseFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupJoin(joinRequest, leaderJoinResponseFuture); - - assertTrue(result.records().isEmpty()); - - CompletableFuture followerJoinResponseFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( - joinRequest.setGroupInstanceId(followerInstanceId), - followerJoinResponseFuture); - - assertTrue(result.records().isEmpty()); - assertFalse(leaderJoinResponseFuture.isDone()); - assertFalse(followerJoinResponseFuture.isDone()); - - // The goal for two timer advance is to let first group initial join complete and set newMemberAdded flag to false. Next advance is - // to trigger the rebalance as needed for follower delayed join. One large time advance won't help because we could only populate one - // delayed join from purgatory and the new delayed op is created at that time and never be triggered. - context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs); - context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs); - - assertTrue(leaderJoinResponseFuture.isDone()); - assertTrue(followerJoinResponseFuture.isDone()); - assertEquals(Errors.NONE.code(), leaderJoinResponseFuture.get().errorCode()); - assertEquals(Errors.NONE.code(), followerJoinResponseFuture.get().errorCode()); - assertEquals(1, leaderJoinResponseFuture.get().generationId()); - assertEquals(1, followerJoinResponseFuture.get().generationId()); - assertEquals(2, group.size()); - assertEquals(1, group.generationId()); - assertTrue(group.isInState(COMPLETING_REBALANCE)); - - String leaderId = leaderJoinResponseFuture.get().memberId(); - String followerId = followerJoinResponseFuture.get().memberId(); - List assignment = new ArrayList<>(); - assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderId) - .setAssignment(new byte[]{1})); - assignment.add(new SyncGroupRequestAssignment().setMemberId(followerId) - .setAssignment(new byte[]{2})); - - SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() - .withGroupId(groupId) - .withGroupInstanceId(leaderInstanceId) - .withMemberId(leaderId) - .withGenerationId(1) - .withAssignment(assignment) - .build(); - - CompletableFuture leaderSyncResponseFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( - syncRequest, - leaderSyncResponseFuture); - - // The generated record should contain the new assignment. - Map groupAssignment = assignment.stream().collect(Collectors.toMap( - SyncGroupRequestAssignment::memberId, SyncGroupRequestAssignment::assignment - )); - assertEquals( - Collections.singletonList( - RecordHelpers.newGroupMetadataRecord(group, groupAssignment, MetadataVersion.latest())), - result.records() - ); - - // Simulate a successful write to the log. - result.appendFuture().complete(null); - - assertTrue(leaderSyncResponseFuture.isDone()); - assertEquals(Errors.NONE.code(), leaderSyncResponseFuture.get().errorCode()); - assertTrue(group.isInState(STABLE)); - - CompletableFuture followerSyncResponseFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( - syncRequest.setGroupInstanceId(followerInstanceId) - .setMemberId(followerId) - .setAssignments(Collections.emptyList()), - followerSyncResponseFuture - ); - - assertTrue(result.records().isEmpty()); - assertTrue(followerSyncResponseFuture.isDone()); - assertEquals(Errors.NONE.code(), followerSyncResponseFuture.get().errorCode()); - assertTrue(group.isInState(STABLE)); - - assertEquals(2, group.size()); - assertEquals(1, group.generationId()); - - return new RebalanceResult( - 1, - leaderId, - leaderSyncResponseFuture.get().assignment(), - followerId, - followerSyncResponseFuture.get().assignment() - ); - } - private static Record newGroupMetadataRecord( GenericGroup group, MetadataVersion metadataVersion From ca133ca9b0f946eb57e95519851f1e64b12291d6 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Mon, 24 Jul 2023 19:29:36 -0400 Subject: [PATCH 09/11] address comments --- .../group/GroupMetadataManagerTest.java | 889 ++++++++---------- 1 file changed, 372 insertions(+), 517 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 7de22259f9ce5..e4ac122135db0 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -281,7 +281,7 @@ static class Builder { private int consumerGroupMetadataRefreshIntervalMs = Integer.MAX_VALUE; private int genericGroupMaxSize = Integer.MAX_VALUE; private int genericGroupInitialRebalanceDelayMs = 3000; - private int genericGroupNewMemberJoinTimeoutMs = 5 * 60 * 1000; + final private int genericGroupNewMemberJoinTimeoutMs = 5 * 60 * 1000; private int genericGroupMinSessionTimeoutMs = 10; private int genericGroupMaxSessionTimeoutMs = 10 * 60 * 1000; @@ -567,7 +567,7 @@ public CoordinatorResult sendGenericGroupJoin( ); } - public JoinGroupResponseData joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin( + public JoinGroupResponseData joinGenericGroupAsDynamicMemberAndCompleteJoin( JoinGroupRequestData request ) throws ExecutionException, InterruptedException { boolean requireKnownMemberId = true; @@ -630,7 +630,7 @@ public JoinGroupResponseData joinGenericGroupAndCompleteJoin( int advanceClockMs ) throws ExecutionException, InterruptedException { if (requireKnownMemberId && request.groupInstanceId().isEmpty()) { - return joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + return joinGenericGroupAsDynamicMemberAndCompleteJoin(request); } try { @@ -652,10 +652,7 @@ public JoinGroupResponseData joinGenericGroupAndCompleteJoin( return null; } - public CoordinatorResult sendGenericGroupSync( - SyncGroupRequestData request, - CompletableFuture responseFuture - ) { + public SyncResult sendGenericGroupSync(SyncGroupRequestData request) { RequestContext context = new RequestContext( new RequestHeader( ApiKeys.SYNC_GROUP, @@ -672,11 +669,15 @@ public CoordinatorResult sendGenericGroupSync( false ); - return groupMetadataManager.genericGroupSync( + CompletableFuture responseFuture = new CompletableFuture<>(); + + CoordinatorResult coordinatorResult = groupMetadataManager.genericGroupSync( context, request, responseFuture ); + + return new SyncResult(responseFuture, coordinatorResult); } @@ -759,10 +760,7 @@ public RebalanceResult staticMembersJoinAndRebalance( .withAssignment(assignment) .build(); - CompletableFuture leaderSyncResponseFuture = new CompletableFuture<>(); - result = sendGenericGroupSync( - syncRequest, - leaderSyncResponseFuture); + SyncResult leaderSyncResult = sendGenericGroupSync(syncRequest); // The generated record should contain the new assignment. Map groupAssignment = assignment.stream().collect(Collectors.toMap( @@ -771,27 +769,25 @@ public RebalanceResult staticMembersJoinAndRebalance( assertEquals( Collections.singletonList( RecordHelpers.newGroupMetadataRecord(group, groupAssignment, MetadataVersion.latest())), - result.records() + leaderSyncResult.records ); // Simulate a successful write to the log. - result.appendFuture().complete(null); + leaderSyncResult.appendFuture.complete(null); - assertTrue(leaderSyncResponseFuture.isDone()); - assertEquals(Errors.NONE.code(), leaderSyncResponseFuture.get().errorCode()); + assertTrue(leaderSyncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderSyncResult.syncFuture.get().errorCode()); assertTrue(group.isInState(STABLE)); - CompletableFuture followerSyncResponseFuture = new CompletableFuture<>(); - result = sendGenericGroupSync( + SyncResult followerSyncResult = sendGenericGroupSync( syncRequest.setGroupInstanceId(followerInstanceId) .setMemberId(followerId) - .setAssignments(Collections.emptyList()), - followerSyncResponseFuture + .setAssignments(Collections.emptyList()) ); - assertTrue(result.records().isEmpty()); - assertTrue(followerSyncResponseFuture.isDone()); - assertEquals(Errors.NONE.code(), followerSyncResponseFuture.get().errorCode()); + assertTrue(followerSyncResult.records.isEmpty()); + assertTrue(followerSyncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), followerSyncResult.syncFuture.get().errorCode()); assertTrue(group.isInState(STABLE)); assertEquals(2, group.size()); @@ -800,10 +796,153 @@ public RebalanceResult staticMembersJoinAndRebalance( return new RebalanceResult( 1, leaderId, - leaderSyncResponseFuture.get().assignment(), + leaderSyncResult.syncFuture.get().assignment(), followerId, - followerSyncResponseFuture.get().assignment() + followerSyncResult.syncFuture.get().assignment() + ); + } + + public JoinGroupResponseData setupGroupWithPendingMember(GenericGroup group) throws Exception { + // Add the first member + JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(UNKNOWN_MEMBER_ID) + .withDefaultProtocolTypeAndProtocols() + .withRebalanceTimeoutMs(10000) + .withSessionTimeoutMs(5000) + .build(); + + JoinGroupResponseData leaderJoinResponse = + joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest); + + String leaderId = leaderJoinResponse.memberId(); + int generationId = leaderJoinResponse.generationId(); + + List assignment = new ArrayList<>(); + assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); + SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() + .withGroupId("group-id") + .withMemberId(leaderId) + .withGenerationId(generationId) + .withAssignment(assignment) + .build(); + + SyncResult syncResult = sendGenericGroupSync(syncRequest); + + // Now the group is stable, with the one member that joined above + assertEquals( + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), + syncResult.records + ); + // Simulate a successful write to log. + syncResult.appendFuture.complete(null); + + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); + + // Start the join for the second member + CompletableFuture followerJoinFuture = new CompletableFuture<>(); + CoordinatorResult result = sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + + CompletableFuture leaderJoinFuture = new CompletableFuture<>(); + result = sendGenericGroupJoin(joinRequest.setMemberId(leaderId), leaderJoinFuture); + + assertTrue(result.records().isEmpty()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); + assertTrue(leaderJoinFuture.isDone()); + assertTrue(followerJoinFuture.isDone()); + assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); + assertEquals(leaderJoinFuture.get().generationId(), followerJoinFuture.get().generationId()); + assertEquals(leaderId, leaderJoinFuture.get().leader()); + assertEquals(leaderId, followerJoinFuture.get().leader()); + + int nextGenerationId = leaderJoinFuture.get().generationId(); + String followerId = followerJoinFuture.get().memberId(); + + // Stabilize the group + syncResult = sendGenericGroupSync(syncRequest.setGenerationId(nextGenerationId)); + + assertEquals( + Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), + syncResult.records ); + // Simulate a successful write to log. + syncResult.appendFuture.complete(null); + + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); + assertTrue(group.isInState(STABLE)); + + // Re-join an existing member, to transition the group to PreparingRebalance state. + leaderJoinFuture = new CompletableFuture<>(); + result = sendGenericGroupJoin(joinRequest.setMemberId(leaderId), leaderJoinFuture); + + assertTrue(result.records().isEmpty()); + assertFalse(leaderJoinFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + + // Create a pending member in the group + CompletableFuture pendingMemberJoinFuture = new CompletableFuture<>(); + result = sendGenericGroupJoin( + joinRequest.setMemberId(UNKNOWN_MEMBER_ID).setSessionTimeoutMs(2500), + pendingMemberJoinFuture, + true + ); + + assertTrue(result.records().isEmpty()); + assertTrue(pendingMemberJoinFuture.isDone()); + assertEquals(Errors.MEMBER_ID_REQUIRED.code(), pendingMemberJoinFuture.get().errorCode()); + assertEquals(1, group.numPendingJoinMembers()); + + // Re-join the second existing member + followerJoinFuture = new CompletableFuture<>(); + result = sendGenericGroupJoin( + joinRequest.setMemberId(followerId).setSessionTimeoutMs(5000), + followerJoinFuture + ); + + assertTrue(result.records().isEmpty()); + assertFalse(followerJoinFuture.isDone()); + assertTrue(group.isInState(PREPARING_REBALANCE)); + assertEquals(1, group.numPendingJoinMembers()); + + return pendingMemberJoinFuture.get(); + } + + public void verifySessionExpiration(GenericGroup group, int timeoutMs) { + Set expectedHeartbeatKeys = group.allMembers().stream() + .map(member -> genericGroupHeartbeatKey(group.groupId(), member.memberId())).collect(Collectors.toSet()); + + // Member should be removed as session expires. + List> timeouts = sleep(timeoutMs); + List expectedRecords = Collections.singletonList(newGroupMetadataRecord( + "group-id", + new GroupMetadataValue() + .setMembers(Collections.emptyList()) + .setGeneration(group.generationId()) + .setLeader(null) + .setProtocolType("consumer") + .setProtocol(null) + .setCurrentStateTimestamp(time.milliseconds()), + MetadataVersion.latest())); + + + Set heartbeatKeys = timeouts.stream().map(timeout -> timeout.key).collect(Collectors.toSet()); + assertEquals(expectedHeartbeatKeys, heartbeatKeys); + + // Only the last member leaving the group should result in the empty group metadata record. + int timeoutsSize = timeouts.size(); + assertEquals(expectedRecords, timeouts.get(timeoutsSize - 1).result.records()); + assertNoOrEmptyResult(timeouts.subList(0, timeoutsSize - 1)); + assertTrue(group.isInState(EMPTY)); + assertEquals(0, group.size()); } private ApiMessage messageOrNull(ApiMessageAndVersion apiMessageAndVersion) { @@ -4078,21 +4217,19 @@ public void testGenericGroupJoinInconsistentProtocolType() throws Exception { .build(); context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol().setName("range")); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) .withDefaultProtocolTypeAndProtocols() .build(); - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("connect") - .withProtocols(protocols) + .withProtocols(defaultProtocolCollection()) .build(); CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4109,13 +4246,11 @@ public void testJoinGroupWithEmptyProtocolType() throws Exception { .build(); context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol().setName("range")); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("") - .withProtocols(protocols) + .withProtocols(defaultProtocolCollection()) .build(); CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4138,12 +4273,11 @@ public void testJoinGroupWithEmptyGroupProtocol() throws Exception { .build(); context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("consumer") - .withProtocols(protocols) + .withProtocols(new JoinGroupRequestProtocolCollection(0)) .build(); CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4173,7 +4307,7 @@ public void testNewMemberJoinExpiration() throws Exception { .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs) .build(); - JoinGroupResponseData firstResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); String firstMemberId = firstResponse.memberId(); assertEquals(firstResponse.leader(), firstMemberId); assertEquals(Errors.NONE.code(), firstResponse.errorCode()); @@ -4247,13 +4381,6 @@ public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception { .build(); context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); - JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) @@ -4282,7 +4409,7 @@ public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception { // Sending consistent protocol should be accepted responseFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin(request.setProtocols(protocols), responseFuture, true); + result = context.sendGenericGroupJoin(request.setProtocols(defaultProtocolCollection()), responseFuture, true); assertTrue(result.records().isEmpty()); assertFalse(responseFuture.isDone()); @@ -4352,7 +4479,7 @@ public void testJoinGroupUnknownConsumerExistingGroup() throws Exception { .withRebalanceTimeoutMs(5000) .build(); - JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4678,7 +4805,7 @@ public void testExistingMemberJoinDeadGroup() throws Exception { .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4728,11 +4855,7 @@ public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() throws .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array())); + JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -4741,7 +4864,7 @@ public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() throws .withProtocols(protocols) .build(); - JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4790,7 +4913,7 @@ public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4819,11 +4942,7 @@ public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInS .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array())); + JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -4832,7 +4951,7 @@ public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInS .withProtocols(protocols) .build(); - JoinGroupResponseData leaderResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), leaderResponse.errorCode()); String leaderId = leaderResponse.leader(); assertEquals(1, group.generationId()); @@ -4904,7 +5023,7 @@ public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() th .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData leaderResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), leaderResponse.errorCode()); String leaderId = leaderResponse.leader(); assertEquals(1, group.generationId()); @@ -4955,7 +5074,7 @@ public void testJoinGroupExistingMemberInEmptyState() throws Exception { .withDefaultProtocolTypeAndProtocols() .build(); - JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), response.errorCode()); String memberId = response.memberId(); @@ -4988,7 +5107,7 @@ public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exceptio .withRebalanceTimeoutMs(1000) .build(); - JoinGroupResponseData leaderResponse = context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(request); + JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); assertEquals(Errors.NONE.code(), leaderResponse.errorCode()); assertEquals(1, group.generationId()); @@ -5138,20 +5257,12 @@ public void testReplaceStaticMemberInStableStateNoError( .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); - JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withGroupInstanceId("group-instance-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("consumer") - .withProtocols(protocols) + .withProtocols(defaultProtocolCollection()) .build(); JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, true, supportSkippingAssignment); @@ -5166,12 +5277,7 @@ public void testReplaceStaticMemberInStableStateNoError( group.transitionTo(STABLE); // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id. - protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); protocols.add(new JoinGroupRequestProtocol() .setName("roundrobin") @@ -5241,13 +5347,7 @@ public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebal .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); protocols.add(new JoinGroupRequestProtocol() .setName("roundrobin") @@ -5299,13 +5399,7 @@ public void testReplaceStaticMemberInStableStateErrors() throws Exception { .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -5387,13 +5481,7 @@ public void testReplaceStaticMemberInStableStateSucceeds( .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -5714,11 +5802,7 @@ JoinGroupRequestBuilder withMemberId(String memberId) { } JoinGroupRequestBuilder withDefaultProtocolTypeAndProtocols() { - this.protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + this.protocols = defaultProtocolCollection(); return this; } @@ -5811,10 +5895,13 @@ public void testNewMemberTimeoutCompletion() throws Exception { assertTrue(result.records().isEmpty()); assertFalse(joinFuture.isDone()); + // Advance clock by initial rebalance delay to complete join phase. assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs)); assertTrue(joinFuture.isDone()); assertEquals(Errors.NONE.code(), joinFuture.get().errorCode()); + assertEquals(1, group.generationId()); + assertTrue(group.isInState(COMPLETING_REBALANCE)); assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count()); @@ -5824,25 +5911,20 @@ public void testNewMemberTimeoutCompletion() throws Exception { .withGenerationId(joinFuture.get().generationId()) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to the log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); assertEquals(1, group.size()); // Make sure the NewMemberTimeout is not still in effect, and the member is not kicked assertNoOrEmptyResult(context.sleep(context.genericGroupNewMemberJoinTimeoutMs)); assertEquals(1, group.size()); - // Member should be removed as heartbeat expires. + // Member should be removed as heartbeat expires. The group is now empty. List> timeouts = context.sleep(5000); List expectedRecords = Collections.singletonList(newGroupMetadataRecord( "group-id", @@ -5863,6 +5945,7 @@ public void testNewMemberTimeoutCompletion() throws Exception { }); assertEquals(0, group.size()); + assertTrue(group.isInState(EMPTY)); } @Test @@ -5897,18 +5980,13 @@ public void testNewMemberFailureAfterJoinGroupCompletion() throws Exception { .withGenerationId(1) .build(); - CompletableFuture syncResponseFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncResponseFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to the log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncResponseFuture.isDone()); - assertEquals(Errors.NONE.code(), syncResponseFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); assertTrue(group.isInState(STABLE)); assertEquals(1, group.generationId()); @@ -5930,7 +6008,7 @@ public void testNewMemberFailureAfterJoinGroupCompletion() throws Exception { assertEquals(Errors.NONE.code(), joinFuture.get().errorCode()); assertEquals(Errors.NONE.code(), otherJoinFuture.get().errorCode()); - verifySessionExpiration(context, group, 5000); + context.verifySessionExpiration(group, 5000); } @Test @@ -6084,11 +6162,10 @@ public void testStaticMemberFenceDuplicateSyncingFollowerAfterMemberIdChanged() .withMemberId(oldFollowerJoinFuture.get().memberId()) .build(); - CompletableFuture oldFollowerSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest, oldFollowerSyncFuture); + SyncResult oldFollowerSyncResult = context.sendGenericGroupSync(syncRequest); assertTrue(result.records().isEmpty()); - assertFalse(oldFollowerSyncFuture.isDone()); + assertFalse(oldFollowerSyncResult.syncFuture.isDone()); CompletableFuture duplicateFollowerJoinFuture = new CompletableFuture<>(); result = context.sendGenericGroupJoin( @@ -6098,13 +6175,13 @@ public void testStaticMemberFenceDuplicateSyncingFollowerAfterMemberIdChanged() assertTrue(result.records().isEmpty()); assertTrue(group.isInState(PREPARING_REBALANCE)); assertFalse(duplicateFollowerJoinFuture.isDone()); - assertTrue(oldFollowerSyncFuture.isDone()); - assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldFollowerSyncFuture.get().errorCode()); + assertTrue(oldFollowerSyncResult.syncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldFollowerSyncResult.syncFuture.get().errorCode()); // Advance clock by rebalance timeout so that the join phase completes with duplicate follower. // Both heartbeats will expire but only the leader is kicked out. List> timeouts = context.sleep(10000); - assertTrue(timeouts.size() == 2); + assertEquals(2, timeouts.size()); timeouts.forEach(timeout -> assertEquals(timeout.result, EMPTY_RESULT)); assertTrue(duplicateFollowerJoinFuture.isDone()); @@ -6254,18 +6331,13 @@ public void testStaticMemberRejoinWithKnownMemberId() throws Exception { .withGroupInstanceId("group-instance-id") .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Successful write to the log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); assertTrue(group.isInState(STABLE)); } @@ -6351,21 +6423,19 @@ public void testStaticMemberRejoinWithLeaderIdAndUnknownMemberId( .withMemberId(rebalanceResult.leaderId) .build(); - CompletableFuture oldLeaderSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(oldLeaderSyncRequest, oldLeaderSyncFuture); + SyncResult oldLeaderSyncResult = context.sendGenericGroupSync(oldLeaderSyncRequest); assertTrue(result.records().isEmpty()); - assertTrue(oldLeaderSyncFuture.isDone()); - assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldLeaderSyncFuture.get().errorCode()); + assertTrue(oldLeaderSyncResult.syncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldLeaderSyncResult.syncFuture.get().errorCode()); // Calling sync on old leader id will fail because that leader id is no longer valid and replaced. SyncGroupRequestData newLeaderSyncRequest = oldLeaderSyncRequest.setGroupInstanceId(null); - CompletableFuture newLeaderSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(newLeaderSyncRequest, newLeaderSyncFuture); + SyncResult newLeaderSyncResult = context.sendGenericGroupSync(newLeaderSyncRequest); assertTrue(result.records().isEmpty()); - assertTrue(newLeaderSyncFuture.isDone()); - assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), newLeaderSyncFuture.get().errorCode()); + assertTrue(newLeaderSyncResult.syncFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), newLeaderSyncResult.syncFuture.get().errorCode()); } @Test @@ -6694,33 +6764,30 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel .withGenerationId(rebalanceResult.generationId + 1) .build(); - CompletableFuture leaderSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest, leaderSyncFuture); + SyncResult leaderSyncResult = context.sendGenericGroupSync(syncRequest); // Simulate a successful write to the log. This will update the group with the new (empty) assignment. - result.appendFuture().complete(null); + leaderSyncResult.appendFuture.complete(null); assertEquals( Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() + leaderSyncResult.records ); - assertTrue(leaderSyncFuture.isDone()); + assertTrue(leaderSyncResult.syncFuture.isDone()); assertTrue(group.isInState(STABLE)); - assertEquals(Errors.NONE.code(), leaderSyncFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), leaderSyncResult.syncFuture.get().errorCode()); // Sync with old member id will also not fail because the member id is not updated because of persistence failure - CompletableFuture oldMemberSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( + SyncResult oldMemberSyncResult = context.sendGenericGroupSync( syncRequest .setGroupInstanceId("follower-instance-id") - .setMemberId(rebalanceResult.followerId), - oldMemberSyncFuture + .setMemberId(rebalanceResult.followerId) ); assertTrue(result.records().isEmpty()); - assertTrue(oldMemberSyncFuture.isDone()); + assertTrue(oldMemberSyncResult.syncFuture.isDone()); assertTrue(group.isInState(STABLE)); - assertEquals(Errors.NONE.code(), oldMemberSyncFuture.get().errorCode()); + assertEquals(Errors.NONE.code(), oldMemberSyncResult.syncFuture.get().errorCode()); } @Test @@ -6797,7 +6864,6 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel assertEquals(Errors.FENCED_INSTANCE_ID.code(), followerJoinFuture.get().errorCode()); // Sync with old member id will fail because the member id is updated - CompletableFuture syncFuture = new CompletableFuture<>(); SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() .withGroupId("group-id") .withGroupInstanceId("follower-instance-id") @@ -6806,23 +6872,19 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel .withAssignment(Collections.emptyList()) .build(); - CoordinatorResult syncResult = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertTrue(syncResult.records().isEmpty()); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.FENCED_INSTANCE_ID.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.records.isEmpty()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), syncResult.syncFuture.get().errorCode()); // Sync with new member id succeeds - syncFuture = new CompletableFuture<>(); - syncResult = context.sendGenericGroupSync( - syncRequest.setMemberId(newFollowerId), - syncFuture - ); + syncResult = context.sendGenericGroupSync(syncRequest.setMemberId(newFollowerId)); - assertTrue(syncResult.records().isEmpty()); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - assertEquals(rebalanceResult.followerAssignment, syncFuture.get().assignment()); + assertTrue(syncResult.records.isEmpty()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); + assertEquals(rebalanceResult.followerAssignment, syncResult.syncFuture.get().assignment()); } @Test @@ -7017,7 +7079,6 @@ public void testStaticMemberRejoinAsFollowerWithUnknownMemberId() throws Excepti ); assertNotEquals(rebalanceResult.followerId, followerJoinFuture.get().memberId()); - CompletableFuture syncFuture = new CompletableFuture<>(); SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() .withGroupId("group-id") .withGroupInstanceId("follower-instance-id") @@ -7025,12 +7086,12 @@ public void testStaticMemberRejoinAsFollowerWithUnknownMemberId() throws Excepti .withMemberId(followerJoinFuture.get().memberId()) .build(); - result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertTrue(result.records().isEmpty()); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - assertEquals(rebalanceResult.followerAssignment, syncFuture.get().assignment()); + assertTrue(syncResult.records.isEmpty()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); + assertEquals(rebalanceResult.followerAssignment, syncResult.syncFuture.get().assignment()); } @Test @@ -7166,15 +7227,11 @@ public void testStaticMemberSyncAsLeaderWithInvalidMemberId() throws Exception { .withMemberId("invalid-member-id") .build(); - CompletableFuture leaderSyncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync( - request, - leaderSyncFuture - ); + SyncResult leaderSyncResult = context.sendGenericGroupSync(request); - assertTrue(result.records().isEmpty()); - assertTrue(leaderSyncFuture.isDone()); - assertEquals(Errors.FENCED_INSTANCE_ID.code(), leaderSyncFuture.get().errorCode()); + assertTrue(leaderSyncResult.records.isEmpty()); + assertTrue(leaderSyncResult.syncFuture.isDone()); + assertEquals(Errors.FENCED_INSTANCE_ID.code(), leaderSyncResult.syncFuture.get().errorCode()); } @Test @@ -7522,7 +7579,7 @@ private void testSyncGroupProtocolTypeAndNameWith( ) throws Exception { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); - GenericGroup group = context.createGenericGroup("group-id"); + context.createGenericGroup("group-id"); // JoinGroup(leader) with the Protocol Type of the group JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() @@ -7568,33 +7625,22 @@ private void testSyncGroupProtocolTypeAndNameWith( .withAssignment(assignment) .build(); - CompletableFuture leaderSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest, leaderSyncFuture); - - if (expectedError == Errors.NONE) { - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); - // Simulate a successful write to log. - result.appendFuture().complete(null); - } else { - assertTrue(result.records().isEmpty()); - } + SyncResult leaderSyncResult = context.sendGenericGroupSync(syncRequest); + // Simulate a successful write to the log. + leaderSyncResult.appendFuture.complete(null); - assertTrue(leaderSyncFuture.isDone()); - assertEquals(expectedError.code(), leaderSyncFuture.get().errorCode()); - assertEquals(expectedProtocolType.orElse(null), leaderSyncFuture.get().protocolType()); - assertEquals(expectedProtocolName.orElse(null), leaderSyncFuture.get().protocolName()); + assertTrue(leaderSyncResult.syncFuture.isDone()); + assertEquals(expectedError.code(), leaderSyncResult.syncFuture.get().errorCode()); + assertEquals(expectedProtocolType.orElse(null), leaderSyncResult.syncFuture.get().protocolType()); + assertEquals(expectedProtocolName.orElse(null), leaderSyncResult.syncFuture.get().protocolName()); - CompletableFuture followerSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest.setMemberId(followerId), followerSyncFuture); + SyncResult followerSyncResult = context.sendGenericGroupSync(syncRequest.setMemberId(followerId)); - assertTrue(result.records().isEmpty()); - assertTrue(followerSyncFuture.isDone()); - assertEquals(expectedError.code(), followerSyncFuture.get().errorCode()); - assertEquals(expectedProtocolType.orElse(null), followerSyncFuture.get().protocolType()); - assertEquals(expectedProtocolName.orElse(null), followerSyncFuture.get().protocolName()); + assertTrue(followerSyncResult.records.isEmpty()); + assertTrue(followerSyncResult.syncFuture.isDone()); + assertEquals(expectedError.code(), followerSyncResult.syncFuture.get().errorCode()); + assertEquals(expectedProtocolType.orElse(null), followerSyncResult.syncFuture.get().protocolType()); + assertEquals(expectedProtocolName.orElse(null), followerSyncResult.syncFuture.get().protocolName()); } @Test @@ -7609,19 +7655,18 @@ public void testSyncGroupFromUnknownGroup() throws Exception { .withGenerationId(1) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertTrue(result.records().isEmpty()); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.records.isEmpty()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncResult.syncFuture.get().errorCode()); } @Test public void testSyncGroupFromUnknownMember() throws Exception { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); - GenericGroup group = context.createGenericGroup("group-id"); + context.createGenericGroup("group-id"); JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -7644,34 +7689,28 @@ public void testSyncGroupFromUnknownMember() throws Exception { .withAssignment(assignment) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - assertEquals(assignment.get(0).assignment(), syncFuture.get().assignment()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); + assertEquals(assignment.get(0).assignment(), syncResult.syncFuture.get().assignment()); - syncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest.setMemberId("unknown-member-id"), syncFuture); + syncResult = context.sendGenericGroupSync(syncRequest.setMemberId("unknown-member-id")); - assertTrue(result.records().isEmpty()); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncFuture.get().errorCode()); - assertEquals(GenericGroupMember.EMPTY_ASSIGNMENT, syncFuture.get().assignment()); + assertTrue(syncResult.records.isEmpty()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncResult.syncFuture.get().errorCode()); + assertEquals(GenericGroupMember.EMPTY_ASSIGNMENT, syncResult.syncFuture.get().assignment()); } @Test public void testSyncGroupFromIllegalGeneration() throws Exception { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); - GenericGroup group = context.createGenericGroup("group-id"); + context.createGenericGroup("group-id"); JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -7692,12 +7731,11 @@ public void testSyncGroupFromIllegalGeneration() throws Exception { .withGenerationId(generationId + 1) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertTrue(result.records().isEmpty()); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.ILLEGAL_GENERATION.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.records.isEmpty()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.ILLEGAL_GENERATION.code(), syncResult.syncFuture.get().errorCode()); } @Test @@ -7708,7 +7746,7 @@ public void testJoinGroupFromUnchangedFollowerDoesNotRebalance() throws Exceptio GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); - GenericGroup group = context.createGenericGroup("group-id"); + context.createGenericGroup("group-id"); JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -7717,7 +7755,7 @@ public void testJoinGroupFromUnchangedFollowerDoesNotRebalance() throws Exceptio .build(); JoinGroupResponseData leaderJoinResponse = - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + context.joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest); String leaderId = leaderJoinResponse.memberId(); int generationId = leaderJoinResponse.generationId(); @@ -7728,21 +7766,16 @@ public void testJoinGroupFromUnchangedFollowerDoesNotRebalance() throws Exceptio .withGenerationId(generationId) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); CompletableFuture followerJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( + CoordinatorResult result = context.sendGenericGroupJoin( joinRequest.setMemberId(UNKNOWN_MEMBER_ID), followerJoinFuture ); @@ -7800,7 +7833,7 @@ public void testLeaderFailureInSyncGroup() throws Exception { .build(); JoinGroupResponseData leaderJoinResponse = - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + context.joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest); String leaderId = leaderJoinResponse.memberId(); int generationId = leaderJoinResponse.generationId(); @@ -7811,21 +7844,16 @@ public void testLeaderFailureInSyncGroup() throws Exception { .withGenerationId(generationId) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); CompletableFuture followerJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( + CoordinatorResult result = context.sendGenericGroupJoin( joinRequest.setMemberId(UNKNOWN_MEMBER_ID), followerJoinFuture ); @@ -7854,14 +7882,11 @@ public void testLeaderFailureInSyncGroup() throws Exception { // With no leader SyncGroup, the follower's sync request should fail with an error indicating // that it should rejoin - CompletableFuture followerSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( - syncRequest.setMemberId(followerId) - .setGenerationId(nextGenerationId), - followerSyncFuture); + SyncResult followerSyncResult = context.sendGenericGroupSync(syncRequest.setMemberId(followerId) + .setGenerationId(nextGenerationId)); - assertTrue(result.records().isEmpty()); - assertFalse(followerSyncFuture.isDone()); + assertTrue(followerSyncResult.records.isEmpty()); + assertFalse(followerSyncResult.syncFuture.isDone()); // Advance clock by session timeout to expire leader heartbeat and prepare rebalance. // This should complete follower's sync response. The follower's heartbeat expiration will not kick @@ -7870,8 +7895,8 @@ public void testLeaderFailureInSyncGroup() throws Exception { assertTrue(timeouts.size() <= 2); timeouts.forEach(timeout -> assertTrue(timeout.result.records().isEmpty())); - assertTrue(followerSyncFuture.isDone()); - assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), followerSyncFuture.get().errorCode()); + assertTrue(followerSyncResult.syncFuture.isDone()); + assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), followerSyncResult.syncFuture.get().errorCode()); assertEquals(1, group.size()); assertTrue(group.hasMemberId(followerId)); assertTrue(group.isInState(PREPARING_REBALANCE)); @@ -7896,7 +7921,7 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { .build(); JoinGroupResponseData leaderJoinResponse = - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + context.joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest); String leaderId = leaderJoinResponse.memberId(); int generationId = leaderJoinResponse.generationId(); @@ -7907,21 +7932,16 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { .withGenerationId(generationId) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); CompletableFuture followerJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( + CoordinatorResult result = context.sendGenericGroupJoin( joinRequest.setMemberId(UNKNOWN_MEMBER_ID), followerJoinFuture ); @@ -7961,36 +7981,24 @@ public void testSyncGroupFollowerAfterLeader() throws Exception { .setAssignment(followerAssignment) ); - syncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( - syncRequest.setGenerationId(nextGenerationId) - .setAssignments(assignment), - syncFuture - ); + syncResult = context.sendGenericGroupSync(syncRequest.setGenerationId(nextGenerationId) + .setAssignments(assignment)); // Simulate a successful write to log. This will update the group's assignment with the new assignment. - result.appendFuture().complete(null); - - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - assertEquals(leaderAssignment, syncFuture.get().assignment()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); + assertEquals(leaderAssignment, syncResult.syncFuture.get().assignment()); // Sync group with follower to get new assignment. - CompletableFuture followerSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( - syncRequest.setMemberId(followerId) - .setGenerationId(nextGenerationId), - followerSyncFuture); + SyncResult followerSyncResult = context.sendGenericGroupSync(syncRequest.setMemberId(followerId) + .setGenerationId(nextGenerationId)); - assertTrue(result.records().isEmpty()); - assertTrue(followerSyncFuture.isDone()); - assertEquals(Errors.NONE.code(), followerSyncFuture.get().errorCode()); - assertEquals(followerAssignment, followerSyncFuture.get().assignment()); + assertTrue(followerSyncResult.records.isEmpty()); + assertTrue(followerSyncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), followerSyncResult.syncFuture.get().errorCode()); + assertEquals(followerAssignment, followerSyncResult.syncFuture.get().assignment()); assertTrue(group.isInState(STABLE)); } @@ -8013,7 +8021,7 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { .build(); JoinGroupResponseData leaderJoinResponse = - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + context.joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest); String leaderId = leaderJoinResponse.memberId(); int generationId = leaderJoinResponse.generationId(); @@ -8024,21 +8032,16 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { .withGenerationId(generationId) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); CompletableFuture followerJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( + CoordinatorResult result = context.sendGenericGroupJoin( joinRequest.setMemberId(UNKNOWN_MEMBER_ID), followerJoinFuture ); @@ -8068,14 +8071,11 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { byte[] followerAssignment = new byte[]{1}; // Sync group with follower to get new assignment. - CompletableFuture followerSyncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( - syncRequest.setMemberId(followerId) - .setGenerationId(nextGenerationId), - followerSyncFuture); + SyncResult followerSyncResult = context.sendGenericGroupSync(syncRequest.setMemberId(followerId) + .setGenerationId(nextGenerationId)); - assertTrue(result.records().isEmpty()); - assertFalse(followerSyncFuture.isDone()); + assertTrue(followerSyncResult.records.isEmpty()); + assertFalse(followerSyncResult.syncFuture.isDone()); // Sync group with leader to get new assignment. List assignment = new ArrayList<>(); @@ -8088,16 +8088,12 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { .setAssignment(followerAssignment) ); - syncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync( - syncRequest.setMemberId(leaderId) - .setGenerationId(nextGenerationId) - .setAssignments(assignment), - syncFuture - ); + syncResult = context.sendGenericGroupSync(syncRequest.setMemberId(leaderId) + .setGenerationId(nextGenerationId) + .setAssignments(assignment)); // Simulate a successful write to log. This will update the group assignment with the new assignment. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); Map updatedAssignment = assignment.stream().collect(Collectors.toMap( SyncGroupRequestAssignment::memberId, SyncGroupRequestAssignment::assignment @@ -8106,16 +8102,16 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { assertEquals( Collections.singletonList( RecordHelpers.newGroupMetadataRecord(group, updatedAssignment, MetadataVersion.latest())), - result.records() + syncResult.records ); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - assertEquals(leaderAssignment, syncFuture.get().assignment()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); + assertEquals(leaderAssignment, syncResult.syncFuture.get().assignment()); // Follower sync group should also be completed. - assertEquals(Errors.NONE.code(), followerSyncFuture.get().errorCode()); - assertEquals(followerAssignment, followerSyncFuture.get().assignment()); + assertEquals(Errors.NONE.code(), followerSyncResult.syncFuture.get().errorCode()); + assertEquals(followerAssignment, followerSyncResult.syncFuture.get().assignment()); assertTrue(group.isInState(STABLE)); } @@ -8123,7 +8119,7 @@ public void testSyncGroupLeaderAfterFollower() throws Exception { public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); - GenericGroup group = context.createGenericGroup("group-id"); + context.createGenericGroup("group-id"); JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -8132,7 +8128,7 @@ public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception { .build(); JoinGroupResponseData leaderJoinResponse = - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + context.joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest); String leaderId = leaderJoinResponse.memberId(); int generationId = leaderJoinResponse.generationId(); @@ -8143,23 +8139,18 @@ public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception { .withGenerationId(generationId) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); // Join group from the leader should force the group to rebalance, which allows the // leader to push new assignment when local metadata changes CompletableFuture leaderJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( + CoordinatorResult result = context.sendGenericGroupJoin( joinRequest.setMemberId(leaderId), leaderJoinFuture ); @@ -8178,7 +8169,7 @@ public void testJoinGroupCompletionWhenPendingMemberJoins() throws Exception { // Set up a group in with a pending member. The test checks if the pending member joining // completes the rebalancing operation - JoinGroupResponseData pendingMemberResponse = setupGroupWithPendingMember(context, group); + JoinGroupResponseData pendingMemberResponse = context.setupGroupWithPendingMember(group); // Compete join group for the pending member JoinGroupRequestData request = new JoinGroupRequestBuilder() @@ -8206,7 +8197,7 @@ public void testJoinGroupCompletionWhenPendingMemberTimesOut() throws Exception // Set up a group in with a pending member. The test checks if the timeout of the pending member will // cause the group to return to a CompletingRebalance state. - setupGroupWithPendingMember(context, group); + context.setupGroupWithPendingMember(group); // Advancing clock by > 2500 (session timeout for the third member) // and < 5000 (for first and second members). This will force the coordinator to attempt join @@ -8217,135 +8208,11 @@ public void testJoinGroupCompletionWhenPendingMemberTimesOut() throws Exception assertEquals(0, group.numPendingJoinMembers()); } - private JoinGroupResponseData setupGroupWithPendingMember( - GroupMetadataManagerTestContext context, - GenericGroup group - ) throws Exception { - // Add the first member - JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() - .withGroupId("group-id") - .withMemberId(UNKNOWN_MEMBER_ID) - .withDefaultProtocolTypeAndProtocols() - .withRebalanceTimeoutMs(10000) - .withSessionTimeoutMs(5000) - .build(); - - JoinGroupResponseData leaderJoinResponse = - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); - - String leaderId = leaderJoinResponse.memberId(); - int generationId = leaderJoinResponse.generationId(); - - List assignment = new ArrayList<>(); - assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderId)); - SyncGroupRequestData syncRequest = new SyncGroupRequestBuilder() - .withGroupId("group-id") - .withMemberId(leaderId) - .withGenerationId(generationId) - .build(); - - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); - - // Now the group is stable, with the one member that joined above - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); - // Simulate a successful write to log. - result.appendFuture().complete(null); - - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - - // Start the join for the second member - CompletableFuture followerJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( - joinRequest.setMemberId(UNKNOWN_MEMBER_ID), - followerJoinFuture - ); - - assertTrue(result.records().isEmpty()); - assertFalse(followerJoinFuture.isDone()); - - CompletableFuture leaderJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( - joinRequest.setMemberId(leaderId), - leaderJoinFuture - ); - - assertTrue(result.records().isEmpty()); - assertTrue(group.isInState(COMPLETING_REBALANCE)); - assertTrue(leaderJoinFuture.isDone()); - assertTrue(followerJoinFuture.isDone()); - assertEquals(Errors.NONE.code(), leaderJoinFuture.get().errorCode()); - assertEquals(Errors.NONE.code(), followerJoinFuture.get().errorCode()); - assertEquals(leaderJoinFuture.get().generationId(), followerJoinFuture.get().generationId()); - assertEquals(leaderId, leaderJoinFuture.get().leader()); - assertEquals(leaderId, followerJoinFuture.get().leader()); - - int nextGenerationId = leaderJoinFuture.get().generationId(); - String followerId = followerJoinFuture.get().memberId(); - - // Stabilize the group - syncFuture = new CompletableFuture<>(); - result = context.sendGenericGroupSync(syncRequest.setGenerationId(nextGenerationId), syncFuture); - - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); - // Simulate a successful write to log. - result.appendFuture().complete(null); - - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); - assertTrue(group.isInState(STABLE)); - - // Re-join an existing member, to transition the group to PreparingRebalance state. - leaderJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( - joinRequest.setMemberId(leaderId), - leaderJoinFuture - ); - - assertTrue(result.records().isEmpty()); - assertFalse(leaderJoinFuture.isDone()); - assertTrue(group.isInState(PREPARING_REBALANCE)); - - // Create a pending member in the group - CompletableFuture pendingMemberJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( - joinRequest.setMemberId(UNKNOWN_MEMBER_ID).setSessionTimeoutMs(2500), - pendingMemberJoinFuture, - true - ); - - assertTrue(result.records().isEmpty()); - assertTrue(pendingMemberJoinFuture.isDone()); - assertEquals(Errors.MEMBER_ID_REQUIRED.code(), pendingMemberJoinFuture.get().errorCode()); - assertEquals(1, group.numPendingJoinMembers()); - - // Re-join the second existing member - followerJoinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin( - joinRequest.setMemberId(followerId).setSessionTimeoutMs(5000), - followerJoinFuture - ); - - assertTrue(result.records().isEmpty()); - assertFalse(followerJoinFuture.isDone()); - assertTrue(group.isInState(PREPARING_REBALANCE)); - assertEquals(1, group.numPendingJoinMembers()); - - return pendingMemberJoinFuture.get(); - } - @Test public void testGenerationIdIncrementsOnRebalance() throws Exception { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); - GenericGroup group = context.createGenericGroup("group-id"); + context.createGenericGroup("group-id"); JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -8354,7 +8221,7 @@ public void testGenerationIdIncrementsOnRebalance() throws Exception { .build(); JoinGroupResponseData leaderJoinResponse = - context.joinGenericGroupAndCompleteJoinAsDynamicMemberAndCompleteJoin(joinRequest); + context.joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest); String leaderId = leaderJoinResponse.memberId(); int generationId = leaderJoinResponse.generationId(); @@ -8367,21 +8234,16 @@ public void testGenerationIdIncrementsOnRebalance() throws Exception { .withGenerationId(generationId) .build(); - CompletableFuture syncFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupSync(syncRequest, syncFuture); + SyncResult syncResult = context.sendGenericGroupSync(syncRequest); - assertEquals( - Collections.singletonList(newGroupMetadataRecord(group, MetadataVersion.latest())), - result.records() - ); // Simulate a successful write to log. - result.appendFuture().complete(null); + syncResult.appendFuture.complete(null); - assertTrue(syncFuture.isDone()); - assertEquals(Errors.NONE.code(), syncFuture.get().errorCode()); + assertTrue(syncResult.syncFuture.isDone()); + assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode()); CompletableFuture joinFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin(joinRequest.setMemberId(leaderId), joinFuture); + CoordinatorResult result = context.sendGenericGroupJoin(joinRequest.setMemberId(leaderId), joinFuture); assertTrue(result.records().isEmpty()); assertTrue(joinFuture.isDone()); @@ -8420,37 +8282,16 @@ private void checkJoinGroupResponse( assertEquals(expectedGroupInstanceIds, groupInstanceIds); } - private void verifySessionExpiration( - GroupMetadataManagerTestContext context, - GenericGroup group, - int timeoutMs - ) { - Set expectedHeartbeatKeys = group.allMembers().stream() - .map(member -> genericGroupHeartbeatKey(group.groupId(), member.memberId())).collect(Collectors.toSet()); - // Member should be removed as session expires. - List> timeouts = context.sleep(timeoutMs); - List expectedRecords = Collections.singletonList(newGroupMetadataRecord( - "group-id", - new GroupMetadataValue() - .setMembers(Collections.emptyList()) - .setGeneration(group.generationId()) - .setLeader(null) - .setProtocolType("consumer") - .setProtocol(null) - .setCurrentStateTimestamp(context.time.milliseconds()), - MetadataVersion.latest())); - - Set heartbeatKeys = timeouts.stream().map(timeout -> timeout.key).collect(Collectors.toSet()); - assertEquals(expectedHeartbeatKeys, heartbeatKeys); - - // Only the last member leaving the group should result in the empty group metadata record. - int timeoutsSize = timeouts.size(); - assertEquals(expectedRecords, timeouts.get(timeoutsSize - 1).result.records()); - assertNoOrEmptyResult(timeouts.subList(0, timeoutsSize - 1)); - assertTrue(group.isInState(EMPTY)); - assertEquals(0, group.size()); + private static JoinGroupRequestProtocolCollection defaultProtocolCollection() { + JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); + protocols.add(new JoinGroupRequestProtocol() + .setName("range") + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList("foo"))).array()) + ); + return protocols; } private static class SyncGroupRequestBuilder { @@ -8461,7 +8302,6 @@ private static class SyncGroupRequestBuilder { String protocolName = "range"; int generationId = 0; List assignment = Collections.emptyList(); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); SyncGroupRequestBuilder withGroupId(String groupId) { this.groupId = groupId; @@ -8539,5 +8379,20 @@ private static class RebalanceResult { this.followerAssignment = followerAssignment; } } + + private static class SyncResult { + CompletableFuture syncFuture; + List records; + CompletableFuture appendFuture; + + public SyncResult( + CompletableFuture syncFuture, + CoordinatorResult coordinatorResult + ) { + this.syncFuture = syncFuture; + this.records = coordinatorResult.records(); + this.appendFuture = coordinatorResult.appendFuture(); + } + } } From d036478c28918546ab787feb62e6844ddce5e2ce Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Tue, 25 Jul 2023 16:31:32 -0400 Subject: [PATCH 10/11] address comments --- .../group/GroupCoordinatorService.java | 2 +- .../group/GroupMetadataManagerTest.java | 139 +++++------------- 2 files changed, 35 insertions(+), 106 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 0dae52f5e4d72..6c8f2fbefbff6 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -342,7 +342,7 @@ public CompletableFuture syncGroup( coordinator -> coordinator.genericGroupSync(context, request, responseFuture) ).exceptionally(exception -> { if (!(exception instanceof KafkaException)) { - log.error("Request {} hit an unexpected exception: {}", + log.error("SyncGroup request {} hit an unexpected exception: {}", request, exception.getMessage()); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index e4ac122135db0..8d8d61df876b7 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -923,7 +923,7 @@ public void verifySessionExpiration(GenericGroup group, int timeoutMs) { // Member should be removed as session expires. List> timeouts = sleep(timeoutMs); List expectedRecords = Collections.singletonList(newGroupMetadataRecord( - "group-id", + group.groupId(), new GroupMetadataValue() .setMembers(Collections.emptyList()) .setGeneration(group.generationId()) @@ -4229,7 +4229,7 @@ public void testGenericGroupJoinInconsistentProtocolType() throws Exception { .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("connect") - .withProtocols(defaultProtocolCollection()) + .withProtocols(toProtocols("range")) .build(); CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4250,7 +4250,7 @@ public void testJoinGroupWithEmptyProtocolType() throws Exception { .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("") - .withProtocols(defaultProtocolCollection()) + .withProtocols(toProtocols("range")) .build(); CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4409,7 +4409,7 @@ public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception { // Sending consistent protocol should be accepted responseFuture = new CompletableFuture<>(); - result = context.sendGenericGroupJoin(request.setProtocols(defaultProtocolCollection()), responseFuture, true); + result = context.sendGenericGroupJoin(request.setProtocols(toProtocols("range")), responseFuture, true); assertTrue(result.records().isEmpty()); assertFalse(responseFuture.isDone()); @@ -4855,7 +4855,7 @@ public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() throws .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); + JoinGroupRequestProtocolCollection protocols = toProtocols("range"); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -4874,16 +4874,7 @@ public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() throws assertTrue(group.isInState(COMPLETING_REBALANCE)); assertEquals(1, group.generationId()); - protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array())); - - protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("bar"))).array())); + protocols = toProtocols("range", "roundrobin"); // Send updated member metadata. This should trigger a rebalance and complete the join phase. CompletableFuture responseFuture = new CompletableFuture<>(); @@ -4942,13 +4933,12 @@ public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInS .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("consumer") - .withProtocols(protocols) + .withProtocols(toProtocols("range")) .build(); JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request); @@ -4981,16 +4971,7 @@ public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInS // Member rejoins with updated metadata. This should trigger a rebalance. String memberId = memberResponseFuture.get().memberId(); - protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array())); - - protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("bar"))).array())); + JoinGroupRequestProtocolCollection protocols = toProtocols("range", "roundrobin"); memberResponseFuture = new CompletableFuture<>(); result = context.sendGenericGroupJoin(request.setMemberId(memberId).setProtocols(protocols), memberResponseFuture); @@ -5262,7 +5243,7 @@ public void testReplaceStaticMemberInStableStateNoError( .withGroupInstanceId("group-instance-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("consumer") - .withProtocols(defaultProtocolCollection()) + .withProtocols(toProtocols("range")) .build(); JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, true, supportSkippingAssignment); @@ -5277,12 +5258,7 @@ public void testReplaceStaticMemberInStableStateNoError( group.transitionTo(STABLE); // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id. - JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); - - protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("bar"))).array())); + JoinGroupRequestProtocolCollection protocols = toProtocols("range", "roundrobin"); CompletableFuture responseFuture = new CompletableFuture<>(); CoordinatorResult result = context.sendGenericGroupJoin( @@ -5347,20 +5323,12 @@ public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebal .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); - - protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("bar"))).array()) - ); - JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withGroupInstanceId("group-instance-id") .withMemberId(UNKNOWN_MEMBER_ID) .withProtocolType("consumer") - .withProtocols(protocols) + .withProtocols(toProtocols("range", "roundrobin")) .build(); JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, true, true); @@ -5373,16 +5341,12 @@ public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebal group.transitionTo(STABLE); // Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance. - protocols = new JoinGroupRequestProtocolCollection(0); - - protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("bar"))).array()) - ); CompletableFuture responseFuture = new CompletableFuture<>(); - CoordinatorResult result = context.sendGenericGroupJoin(request.setProtocols(protocols), responseFuture); + CoordinatorResult result = context.sendGenericGroupJoin( + request.setProtocols(toProtocols("roundrobin")), + responseFuture + ); assertTrue(result.records().isEmpty()); assertTrue(responseFuture.isDone()); @@ -5399,7 +5363,7 @@ public void testReplaceStaticMemberInStableStateErrors() throws Exception { .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); + JoinGroupRequestProtocolCollection protocols = toProtocols("range"); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -5481,7 +5445,7 @@ public void testReplaceStaticMemberInStableStateSucceeds( .build(); GenericGroup group = context.createGenericGroup("group-id"); - JoinGroupRequestProtocolCollection protocols = defaultProtocolCollection(); + JoinGroupRequestProtocolCollection protocols = toProtocols("range"); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -5802,22 +5766,12 @@ JoinGroupRequestBuilder withMemberId(String memberId) { } JoinGroupRequestBuilder withDefaultProtocolTypeAndProtocols() { - this.protocols = defaultProtocolCollection(); + this.protocols = toProtocols("range"); return this; } JoinGroupRequestBuilder withProtocolSuperset() { - this.protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); - - this.protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("bar"))).array()) - ); + this.protocols = toProtocols("range", "roundrobin"); return this; } @@ -5927,7 +5881,7 @@ public void testNewMemberTimeoutCompletion() throws Exception { // Member should be removed as heartbeat expires. The group is now empty. List> timeouts = context.sleep(5000); List expectedRecords = Collections.singletonList(newGroupMetadataRecord( - "group-id", + group.groupId(), new GroupMetadataValue() .setMembers(Collections.emptyList()) .setGeneration(2) @@ -6558,12 +6512,7 @@ public void testStaticMemberRejoinWithFollowerIdAndChangeOfProtocol() throws Exc ); // A static follower rejoin with changed protocol will trigger rebalance. - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = toProtocols("roundrobin"); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") .withGroupInstanceId("follower-instance-id") @@ -6626,12 +6575,7 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWithSele assertNotEquals("roundrobin", group.selectProtocol()); // A static follower rejoin with changed protocol will trigger rebalance. - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("roundrobin") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = toProtocols("roundrobin"); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -6687,13 +6631,7 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel "follower-instance-id" ); - String selectedProtocol = group.selectProtocol(); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName(selectedProtocol) - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = toProtocols(group.selectProtocol()); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -6805,13 +6743,7 @@ public void testStaticMemberRejoinWithUnknownMemberIdAndChangeOfProtocolWhileSel // A static follower rejoin with protocol changing to leader protocol subset won't trigger rebalance if updated // group's selectProtocol remain unchanged. - String selectedProtocol = group.selectProtocol(); - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName(selectedProtocol) - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + JoinGroupRequestProtocolCollection protocols = toProtocols(group.selectProtocol()); JoinGroupRequestData request = new JoinGroupRequestBuilder() .withGroupId("group-id") @@ -6974,11 +6906,7 @@ public void testStaticMemberRejoinWithKnownLeaderIdToTriggerRebalanceAndFollower ); // The follower protocol changed from protocolSuperset to general protocols. - JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array())); + JoinGroupRequestProtocolCollection protocols = toProtocols("range"); followerJoinFuture = new CompletableFuture<>(); result = context.sendGenericGroupJoin( @@ -8282,15 +8210,16 @@ private void checkJoinGroupResponse( assertEquals(expectedGroupInstanceIds, groupInstanceIds); } - - - private static JoinGroupRequestProtocolCollection defaultProtocolCollection() { + private static JoinGroupRequestProtocolCollection toProtocols(String... protocolNames) { JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0); - protocols.add(new JoinGroupRequestProtocol() - .setName("range") - .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("foo"))).array()) - ); + List topicNames = Arrays.asList("foo", "bar", "baz"); + for (int i = 0; i < protocolNames.length; i++) { + protocols.add(new JoinGroupRequestProtocol() + .setName(protocolNames[i]) + .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription( + Collections.singletonList(topicNames.get(i % topicNames.size())))).array()) + ); + } return protocols; } From ec7f6766d4e210f0cb13e37a6547d84707f12658 Mon Sep 17 00:00:00 2001 From: Jeff Kim Date: Wed, 26 Jul 2023 09:46:47 -0400 Subject: [PATCH 11/11] checkstyle --- .../kafka/coordinator/group/GroupMetadataManagerTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 8d8d61df876b7..792e4ec777a21 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -8318,9 +8318,9 @@ public SyncResult( CompletableFuture syncFuture, CoordinatorResult coordinatorResult ) { - this.syncFuture = syncFuture; - this.records = coordinatorResult.records(); - this.appendFuture = coordinatorResult.appendFuture(); + this.syncFuture = syncFuture; + this.records = coordinatorResult.records(); + this.appendFuture = coordinatorResult.appendFuture(); } } }