From 6d72677eda35cdfa228c80a48328f64357b7c742 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 31 Mar 2025 13:21:06 +0200 Subject: [PATCH] KAFKA-18613: Add StreamsGroupHeartbeat handler in the group coordinator (#19114) Basic streams group heartbeat handling. The main part of are the unit tests that make sure that we behave, for the most part, like a consumer group. - No support for static membership - No support for configurations (using constants instead) - No support for regular expressions Reviewers: Bill Bejeck , Bruno Cadonna --- .../group/GroupCoordinatorShard.java | 2 +- .../group/GroupMetadataManager.java | 850 ++++++- .../streams/CurrentAssignmentBuilder.java | 2 +- .../group/streams/StreamsGroup.java | 58 +- .../group/streams/StreamsGroupMember.java | 15 + .../group/streams/StreamsTopology.java | 14 + .../coordinator/group/streams/TasksTuple.java | 71 +- .../streams/assignor/MemberAssignment.java | 4 +- .../group/GroupMetadataManagerTest.java | 2094 ++++++++++++++++- .../GroupMetadataManagerTestContext.java | 49 + .../group/streams/MockTaskAssignor.java | 65 + .../group/streams/StreamsGroupBuilder.java | 4 +- .../group/streams/StreamsGroupTest.java | 52 +- .../group/streams/StreamsTopologyTest.java | 62 + 14 files changed, 3255 insertions(+), 87 deletions(-) create mode 100644 group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/MockTaskAssignor.java diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java index ed1e94d4783..a51e55eaf93 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java @@ -400,7 +400,7 @@ public class GroupCoordinatorShard implements CoordinatorShard streamsGroupHeartbeat( 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 718c368f9fd..63be7d1c5eb 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 @@ -30,6 +30,7 @@ import org.apache.kafka.common.errors.InconsistentGroupProtocolException; import org.apache.kafka.common.errors.InvalidRegularExpression; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.RebalanceInProgressException; +import org.apache.kafka.common.errors.StreamsInvalidTopologyException; import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnreleasedInstanceIdException; @@ -39,6 +40,7 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.message.ConsumerProtocolSubscription; +import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic; import org.apache.kafka.common.message.DescribeGroupsResponseData; import org.apache.kafka.common.message.HeartbeatRequestData; import org.apache.kafka.common.message.HeartbeatResponseData; @@ -56,6 +58,13 @@ import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData.Endpoint; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData.KeyValue; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData.TaskIds; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData.TaskOffset; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData.Topology; +import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; +import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData.Status; import org.apache.kafka.common.message.SyncGroupRequestData; import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.protocol.Errors; @@ -63,6 +72,7 @@ import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.RequestContext; import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest; +import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -142,6 +152,12 @@ import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; import org.apache.kafka.coordinator.group.streams.StreamsTopology; import org.apache.kafka.coordinator.group.streams.TasksTuple; +import org.apache.kafka.coordinator.group.streams.assignor.StickyTaskAssignor; +import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor; +import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignorException; +import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology; +import org.apache.kafka.coordinator.group.streams.topics.InternalTopicManager; +import org.apache.kafka.coordinator.group.streams.topics.TopicConfigurationException; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.TopicImage; @@ -167,6 +183,7 @@ import org.slf4j.Logger; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -174,6 +191,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.OptionalInt; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -227,11 +245,18 @@ import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPA import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.STABLE; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME; import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CONSUMER_GROUP_REBALANCES_SENSOR_NAME; -import static org.apache.kafka.coordinator.group.modern.ModernGroupMember.hasAssignedPartitionsChanged; +import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.STREAMS_GROUP_REBALANCES_SENSOR_NAME; +import static org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember.hasAssignedPartitionsChanged; +import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.convertToStreamsGroupTopologyRecord; +import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord; import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord; import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord; +import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord; import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord; +import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord; import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord; +import static org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord; +import static org.apache.kafka.coordinator.group.streams.StreamsGroupMember.hasAssignedTasksChanged; /** @@ -274,6 +299,7 @@ public class GroupMetadataManager { private ShareGroupPartitionAssignor shareGroupAssignor = null; private GroupCoordinatorMetricsShard metrics; private Optional authorizer = null; + private List streamsGroupAssignors = null; Builder withLogContext(LogContext logContext) { this.logContext = logContext; @@ -310,6 +336,11 @@ public class GroupMetadataManager { return this; } + Builder withStreamsGroupAssignors(List streamsGroupAssignors) { + this.streamsGroupAssignors = streamsGroupAssignors; + return this; + } + Builder withMetadataImage(MetadataImage metadataImage) { this.metadataImage = metadataImage; return this; @@ -349,6 +380,8 @@ public class GroupMetadataManager { throw new IllegalArgumentException("GroupCoordinatorMetricsShard must be set."); if (groupConfigManager == null) throw new IllegalArgumentException("GroupConfigManager must be set."); + if (streamsGroupAssignors == null) + streamsGroupAssignors = List.of(new StickyTaskAssignor()); return new GroupMetadataManager( snapshotRegistry, @@ -361,7 +394,8 @@ public class GroupMetadataManager { config, groupConfigManager, shareGroupAssignor, - authorizer + authorizer, + streamsGroupAssignors ); } } @@ -443,9 +477,9 @@ public class GroupMetadataManager { private final GroupConfigManager groupConfigManager; /** - * The session timeout for streams groups. + * The supported task assignors keyed by their name. */ - private final int streamsGroupSessionTimeoutMs; + private final Map streamsGroupAssignors; /** * The metadata image. @@ -501,7 +535,8 @@ public class GroupMetadataManager { GroupCoordinatorConfig config, GroupConfigManager groupConfigManager, ShareGroupPartitionAssignor shareGroupAssignor, - Optional authorizer + Optional authorizer, + List streamsGroupAssignors ) { this.logContext = logContext; this.log = logContext.logger(GroupMetadataManager.class); @@ -522,8 +557,8 @@ public class GroupMetadataManager { this.shareGroupPartitionMetadata = new TimelineHashMap<>(snapshotRegistry, 0); this.groupConfigManager = groupConfigManager; this.shareGroupAssignor = shareGroupAssignor; - this.streamsGroupSessionTimeoutMs = 45000; this.authorizer = authorizer; + this.streamsGroupAssignors = streamsGroupAssignors.stream().collect(Collectors.toMap(TaskAssignor::name, Function.identity())); } /** @@ -802,40 +837,58 @@ public class GroupMetadataManager { } /** - * Gets or maybe creates a streams group without updating the groups map. + * Gets or creates a streams group without updating the groups map. * The group will be materialized during the replay. * - * @param groupId The group id. - * @param createIfNotExists A boolean indicating whether the group should be - * created if it does not exist or is an empty classic group. + * @param groupId The group ID. * * @return A StreamsGroup. - * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or - * if the group is not a streams group. * * Package private for testing. */ - StreamsGroup getOrMaybeCreateStreamsGroup( - String groupId, - boolean createIfNotExists - ) throws GroupIdNotFoundException { + StreamsGroup getOrCreateStreamsGroup( + String groupId + ) { Group group = groups.get(groupId); - if (group == null && !createIfNotExists) { - throw new GroupIdNotFoundException(String.format("Streams group %s not found.", groupId)); - } - if (group == null) { return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); } else { - if (group.type() == STREAMS) { - return (StreamsGroup) group; - } else { - throw new GroupIdNotFoundException(String.format("Group %s is not a streams group.", groupId)); - } + return castToStreamsGroup(group); } } - + + /** + * Gets a streams group without updating the groups map. If the group does not exist, + * a GroupIdNotFoundException is thrown. + * + * @param groupId The group ID. + * + * @return A StreamsGroup. + * @throws GroupIdNotFoundException if the group does not exist + * + * Package private for testing. + */ + StreamsGroup getStreamsGroupOrThrow( + String groupId + ) throws GroupIdNotFoundException { + Group group = groups.get(groupId); + + if (group == null) { + throw new GroupIdNotFoundException(String.format("Streams group %s not found.", groupId)); + } else { + return castToStreamsGroup(group); + } + } + + private StreamsGroup castToStreamsGroup(final Group group) { + if (group.type() == STREAMS) { + return (StreamsGroup) group; + } else { + throw new GroupIdNotFoundException(String.format("Group %s is not a streams group.", group.groupId())); + } + } + /** * Gets a streams group by committed offset. * @@ -1362,7 +1415,7 @@ public class GroupMetadataManager { * @param error The error message. * @throws InvalidRequestException */ - private void throwIfEmptyString( + private static void throwIfEmptyString( String value, String error ) throws InvalidRequestException { @@ -1371,6 +1424,37 @@ public class GroupMetadataManager { } } + /** + * Throws an InvalidRequestException if the value is null or non-empty. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + private static void throwIfNotEmptyCollection( + Collection value, + String error + ) throws InvalidRequestException { + if (value == null || !value.isEmpty()) { + throw new InvalidRequestException(error); + } + } + + private static void throwIfInvalidTopology( + StreamsGroupHeartbeatRequestData.Topology topology + ) throws StreamsInvalidTopologyException { + for (StreamsGroupHeartbeatRequestData.Subtopology subtopology: topology.subtopologies()) { + for (StreamsGroupHeartbeatRequestData.TopicInfo topicInfo: subtopology.stateChangelogTopics()) { + if (topicInfo.partitions() != 0) { + throw new StreamsInvalidTopologyException(String.format( + "Changelog topic %s must have an undefined partition count, but it is set to %d.", + topicInfo.name(), topicInfo.partitions() + )); + } + } + } + } + /** * Throws an InvalidRequestException if the value is non-null. * @@ -1378,7 +1462,7 @@ public class GroupMetadataManager { * @param error The error message. * @throws InvalidRequestException */ - private void throwIfNotNull( + private static void throwIfNotNull( Object value, String error ) throws InvalidRequestException { @@ -1394,7 +1478,7 @@ public class GroupMetadataManager { * @param error The error message. * @throws InvalidRequestException */ - private void throwIfNull( + private static void throwIfNull( Object value, String error ) throws InvalidRequestException { @@ -1475,6 +1559,50 @@ public class GroupMetadataManager { } } + /** + * Validates the request. + * + * @param request The request to validate. + * @throws InvalidRequestException if the request is not valid. + * @throws UnsupportedAssignorException if the assignor is not supported. + */ + private static void throwIfStreamsGroupHeartbeatRequestIsInvalid( + StreamsGroupHeartbeatRequestData request + ) throws InvalidRequestException { + throwIfEmptyString(request.memberId(), "MemberId can't be empty."); + throwIfEmptyString(request.groupId(), "GroupId can't be empty."); + throwIfEmptyString(request.instanceId(), "InstanceId can't be empty."); + throwIfEmptyString(request.rackId(), "RackId can't be empty."); + + if (request.memberEpoch() == 0) { + if (request.rebalanceTimeoutMs() == -1) { + throw new InvalidRequestException("RebalanceTimeoutMs must be provided in first request."); + } + throwIfNotEmptyCollection(request.activeTasks(), "ActiveTasks must be empty when (re-)joining."); + throwIfNotEmptyCollection(request.standbyTasks(), "StandbyTasks must be empty when (re-)joining."); + throwIfNotEmptyCollection(request.warmupTasks(), "WarmupTasks must be empty when (re-)joining."); + throwIfNull(request.topology(), "Topology must be non-null when (re-)joining."); + if (request.topology() != null) { + throwIfInvalidTopology(request.topology()); + } + } else if (request.memberEpoch() == LEAVE_GROUP_STATIC_MEMBER_EPOCH) { + throwIfNull(request.instanceId(), "InstanceId can't be null."); + } else if (request.memberEpoch() < LEAVE_GROUP_STATIC_MEMBER_EPOCH) { + throw new InvalidRequestException(String.format("MemberEpoch is %d, but must be greater than or equal to -2.", + request.memberEpoch())); + } + + if (request.activeTasks() != null || request.standbyTasks() != null || request.warmupTasks() != null) { + throwIfNull(request.activeTasks(), "If one task-type is non-null, all must be non-null."); + throwIfNull(request.standbyTasks(), "If one task-type is non-null, all must be non-null."); + throwIfNull(request.warmupTasks(), "If one task-type is non-null, all must be non-null."); + } + + if (request.memberEpoch() != 0) { + throwIfNotNull(request.topology(), "Topology can only be provided when (re-)joining."); + } + } + /** * Verifies that the partitions currently owned by the member (the ones set in the * request) matches the ones that the member should own. It matches if the consumer @@ -1486,7 +1614,7 @@ public class GroupMetadataManager { * * @return A boolean indicating whether the owned partitions are a subset or not. */ - private boolean isSubset( + private static boolean isSubset( List ownedTopicPartitions, Map> target ) { @@ -1503,6 +1631,34 @@ public class GroupMetadataManager { return true; } + /** + * Verifies that the tasks currently owned by the member (the ones set in the + * request) matches the ones that the member should own. It matches if the streams + * group member only owns tasks which are in the assigned tasks. It does not match if + * it owns any other tasks. + * + * @param ownedTasks The tasks provided by the streams group member in the request. + * @param assignedTasks The tasks that the member should have. + * + * @return A boolean indicating whether the owned partitions are a subset or not. + */ + private static boolean areOwnedTasksContainedInAssignedTasks( + List ownedTasks, + Map> assignedTasks + ) { + if (ownedTasks == null) return false; + + for (StreamsGroupHeartbeatRequestData.TaskIds ownedTasksOfSubtopology : ownedTasks) { + Set partitions = assignedTasks.get(ownedTasksOfSubtopology.subtopologyId()); + if (partitions == null) return false; + for (Integer partitionId : ownedTasksOfSubtopology.partitions()) { + if (!partitions.contains(partitionId)) return false; + } + } + + return true; + } + /** * Checks whether the consumer group can accept a new member or not based on the * max group size defined. @@ -1656,6 +1812,43 @@ public class GroupMetadataManager { } } + /** + * Validates the member epoch provided in the heartbeat request. + * + * @param member The Streams group member. + * @param receivedMemberEpoch The member epoch. + * @param ownedActiveTasks The owned active tasks. + * @param ownedStandbyTasks The owned standby tasks. + * @param ownedWarmupTasks The owned warmup tasks. + * + * @throws FencedMemberEpochException if the provided epoch is ahead or behind the epoch known + * by this coordinator. + */ + private static void throwIfStreamsGroupMemberEpochIsInvalid( + StreamsGroupMember member, + int receivedMemberEpoch, + List ownedActiveTasks, + List ownedStandbyTasks, + List ownedWarmupTasks + ) { + if (receivedMemberEpoch > member.memberEpoch()) { + throw new FencedMemberEpochException("The streams group member has a greater member " + + "epoch (" + receivedMemberEpoch + ") than the one known by the group coordinator (" + + member.memberEpoch() + "). The member must abandon all its partitions and rejoin."); + } else if (receivedMemberEpoch < member.memberEpoch()) { + // If the member comes with the previous epoch and has a subset of the current assignment partitions, + // we accept it because the response with the bumped epoch may have been lost. + if (receivedMemberEpoch != member.previousMemberEpoch() + || !areOwnedTasksContainedInAssignedTasks(ownedActiveTasks, member.assignedTasks().activeTasks()) + || !areOwnedTasksContainedInAssignedTasks(ownedStandbyTasks, member.assignedTasks().standbyTasks()) + || !areOwnedTasksContainedInAssignedTasks(ownedWarmupTasks, member.assignedTasks().warmupTasks())) { + throw new FencedMemberEpochException("The streams group member has a smaller member " + + "epoch (" + receivedMemberEpoch + ") than the one known by the group coordinator (" + + member.memberEpoch() + "). The member must abandon all its partitions and rejoin."); + } + } + } + /** * Validates if the received classic member protocols are supported by the group. * @@ -1828,6 +2021,287 @@ public class GroupMetadataManager { .toList(); } + /** + * Handles a regular heartbeat from a Streams group member. + * It mainly consists of five parts: + * 1) Create or update the member. + * The group epoch is bumped if the member has been created or updated. + * 2) Initialize or update the topology. + * The group epoch is bumped if the topology has been created or updated. + * 3) Determine the partition metadata and any internal topics that need to be created. + * 4) Update the target assignment for the streams group if the group epoch + * is larger than the current target assignment epoch. + * 5) Reconcile the member's assignment with the target assignment. + * + * @param groupId The group ID from the request. + * @param memberId The member ID from the request. + * @param memberEpoch The member epoch from the request. + * @param instanceId The instance ID from the request or null. + * @param rackId The rack ID from the request or null. + * @param rebalanceTimeoutMs The rebalance timeout from the request or -1. + * @param clientId The client ID. + * @param clientHost The client host. + * @param topology The topology from the request or null. + * @param ownedActiveTasks The list of owned active tasks from the request or null. + * @param ownedStandbyTasks The list of owned standby tasks from the request or null. + * @param ownedWarmupTasks The list of owned warmup tasks from the request or null. + * @param userEndpoint User-defined endpoint for Interactive Queries, or null. + * @param clientTags Used for rack-aware assignment algorithm, or null. + * @param taskEndOffsets Cumulative changelog offsets for tasks, or null. + * @param taskOffsets Cumulative changelog end-offsets for tasks, or null. + * @param shutdownApplication Whether all Streams clients in the group should shut down. + * @return A result containing the StreamsGroupHeartbeat response and a list of records to update the state machine. + */ + private CoordinatorResult streamsGroupHeartbeat( + String groupId, + String memberId, + int memberEpoch, + String instanceId, + String rackId, + int rebalanceTimeoutMs, + String clientId, + String clientHost, + StreamsGroupHeartbeatRequestData.Topology topology, + List ownedActiveTasks, + List ownedStandbyTasks, + List ownedWarmupTasks, + String processId, + Endpoint userEndpoint, + List clientTags, + List taskOffsets, + List taskEndOffsets, + boolean shutdownApplication + ) throws ApiException { + final long currentTimeMs = time.milliseconds(); + final List records = new ArrayList<>(); + final List returnedStatus = new ArrayList<>(); + + // Get or create the streams group. + boolean isJoining = memberEpoch == 0; + final StreamsGroup group = isJoining ? getOrCreateStreamsGroup(groupId) : getStreamsGroupOrThrow(groupId); + + // Get or create the member. + StreamsGroupMember member; + if (instanceId == null) { + member = getOrMaybeCreateDynamicStreamsGroupMember( + group, + memberId, + memberEpoch, + ownedActiveTasks, + ownedStandbyTasks, + ownedWarmupTasks, + isJoining + ); + } else { + throw new UnsupportedOperationException("Static members are not supported yet."); + } + + // 1. Create or update the member. + StreamsGroupMember updatedMember = new StreamsGroupMember.Builder(member) + .maybeUpdateInstanceId(Optional.empty()) + .maybeUpdateRackId(Optional.ofNullable(rackId)) + .maybeUpdateRebalanceTimeoutMs(ofSentinel(rebalanceTimeoutMs)) + .maybeUpdateTopologyEpoch(topology != null ? OptionalInt.of(topology.epoch()) : OptionalInt.empty()) + .setClientId(clientId) + .setClientHost(clientHost) + .maybeUpdateProcessId(Optional.ofNullable(processId)) + .maybeUpdateClientTags(Optional.ofNullable(clientTags).map(x -> x.stream().collect(Collectors.toMap(KeyValue::key, KeyValue::value)))) + .maybeUpdateUserEndpoint(Optional.ofNullable(userEndpoint).map(x -> new StreamsGroupMemberMetadataValue.Endpoint().setHost(x.host()).setPort(x.port()))) + .build(); + + // If the member is new or has changed, a StreamsGroupMemberMetadataValue record is written to the __consumer_offsets partition + // to persist the change, and bump the group epoch later. + boolean bumpGroupEpoch = hasStreamsMemberMetadataChanged(groupId, member, updatedMember, records); + + // 2. Initialize/Update the group topology. + // If the topology is new or has changed, a StreamsGroupTopologyValue record is written to the __consumer_offsets partition to persist + // the change. The group epoch is bumped if the topology has changed. + StreamsTopology updatedTopology = maybeUpdateTopology(groupId, memberId, topology, group, records); + maybeSetTopologyStaleStatus(group, updatedMember, returnedStatus); + + // 3. Determine the partition metadata and any internal topics if needed. + ConfiguredTopology updatedConfiguredTopology; + Map updatedPartitionMetadata; + boolean reconfigureTopology = group.topology().isEmpty(); + if (reconfigureTopology || group.hasMetadataExpired(currentTimeMs)) { + + updatedPartitionMetadata = group.computePartitionMetadata( + metadataImage.topics(), + updatedTopology + ); + + if (!updatedPartitionMetadata.equals(group.partitionMetadata())) { + log.info("[GroupId {}][MemberId {}] Computed new partition metadata: {}.", + groupId, memberId, updatedPartitionMetadata); + bumpGroupEpoch = true; + reconfigureTopology = true; + records.add(newStreamsGroupPartitionMetadataRecord(groupId, updatedPartitionMetadata)); + group.setPartitionMetadata(updatedPartitionMetadata); + } + + if (reconfigureTopology || group.configuredTopology().isEmpty()) { + log.info("[GroupId {}][MemberId {}] Configuring the topology {}", groupId, memberId, updatedTopology); + updatedConfiguredTopology = InternalTopicManager.configureTopics(logContext, updatedTopology, updatedPartitionMetadata); + } else { + updatedConfiguredTopology = group.configuredTopology().get(); + } + } else { + updatedConfiguredTopology = group.configuredTopology().get(); + updatedPartitionMetadata = group.partitionMetadata(); + } + + // Actually bump the group epoch + int groupEpoch = group.groupEpoch(); + if (bumpGroupEpoch) { + groupEpoch += 1; + records.add(newStreamsGroupEpochRecord(groupId, groupEpoch)); + log.info("[GroupId {}][MemberId {}] Bumped streams group epoch to {}.", groupId, memberId, groupEpoch); + metrics.record(STREAMS_GROUP_REBALANCES_SENSOR_NAME); + group.setMetadataRefreshDeadline(currentTimeMs + METADATA_REFRESH_INTERVAL_MS, groupEpoch); + } + + // 4. Update the target assignment if the group epoch is larger than the target assignment epoch or a static member + // replaces an existing static member. + // The delta between the existing and the new target assignment is persisted to the partition. + int targetAssignmentEpoch; + TasksTuple targetAssignment; + if (groupEpoch > group.assignmentEpoch()) { + targetAssignment = updateStreamsTargetAssignment( + group, + groupEpoch, + updatedMember, + updatedConfiguredTopology, + updatedPartitionMetadata, + records + ); + targetAssignmentEpoch = groupEpoch; + } else { + targetAssignmentEpoch = group.assignmentEpoch(); + targetAssignment = group.targetAssignment(updatedMember.memberId()); + } + + // 5. Reconcile the member's assignment with the target assignment if the member is not + // fully reconciled yet. + updatedMember = maybeReconcile( + groupId, + updatedMember, + group::currentActiveTaskProcessId, + group::currentStandbyTaskProcessIds, + group::currentWarmupTaskProcessIds, + targetAssignmentEpoch, + targetAssignment, + ownedActiveTasks, + ownedStandbyTasks, + ownedWarmupTasks, + records + ); + + scheduleStreamsGroupSessionTimeout(groupId, memberId); + + // Prepare the response. + StreamsGroupHeartbeatResponseData response = new StreamsGroupHeartbeatResponseData() + .setMemberId(updatedMember.memberId()) + .setMemberEpoch(updatedMember.memberEpoch()) + .setHeartbeatIntervalMs(streamsGroupHeartbeatIntervalMs(groupId)); + + // The assignment is only provided in the following cases: + // 1. The member is joining. + // 2. The member's assignment has been updated. + if (memberEpoch == 0 || hasAssignedTasksChanged(member, updatedMember)) { + response.setActiveTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().activeTasks())); + response.setStandbyTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().standbyTasks())); + response.setWarmupTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().warmupTasks())); + } + + Map internalTopicsToBeCreated = Collections.emptyMap(); + if (updatedConfiguredTopology.topicConfigurationException().isPresent()) { + TopicConfigurationException exception = updatedConfiguredTopology.topicConfigurationException().get(); + internalTopicsToBeCreated = updatedConfiguredTopology.internalTopicsToBeCreated(); + returnedStatus.add( + new StreamsGroupHeartbeatResponseData.Status() + .setStatusCode(exception.status().code()) + .setStatusDetail(exception.getMessage()) + ); + } + + if (!returnedStatus.isEmpty()) { + response.setStatus(returnedStatus); + } + return new CoordinatorResult<>(records, new StreamsGroupHeartbeatResult(response, internalTopicsToBeCreated)); + } + + /** + * Checks if the member's topology epoch is behind the group's topology epoch, and sets the corresponding status. + * + * @param group The streams group. + * @param member The streams group member. + * @param returnedStatus A mutable collection of status to be returned in the response. + */ + private static void maybeSetTopologyStaleStatus(final StreamsGroup group, final StreamsGroupMember member, final List returnedStatus) { + if (group.topology().isPresent() && member.topologyEpoch() < group.topology().get().topologyEpoch()) { + returnedStatus.add( + new Status() + .setStatusCode(StreamsGroupHeartbeatResponse.Status.STALE_TOPOLOGY.code()) + .setStatusDetail( + String.format( + "The member's topology epoch %d is behind the group's topology epoch %d.", + member.topologyEpoch(), + group.topology().get().topologyEpoch() + ) + ) + ); + } + } + + /** + * Compares the topology from the request with the one in the group. + * + * - If the topology of the group is uninitialized, it is initialized with the topology from the request. A corresponding + * record is added to records. + * - If the topology of the group is initialized, and the request defines a topology, they are compared. If they + * are not empty, an InvalidRequestException is thrown. + * + * @param groupId The group ID. + * @param memberId The member ID. + * @param topology The topology provided in the request. May be null. + * @param group The streams group. + * @param records A mutable collection of records to be written to the __consumer_offsets partition. + * @return The new topology of the group (which may be the same as the current one). + */ + private StreamsTopology maybeUpdateTopology(final String groupId, + final String memberId, + final Topology topology, + final StreamsGroup group, + final List records) { + StreamsTopology updatedTopology; + if (topology != null) { + StreamsGroupTopologyValue recordValue = convertToStreamsGroupTopologyRecord(topology); + + updatedTopology = StreamsTopology.fromHeartbeatRequest(topology); + + if (group.topology().isEmpty()) { + log.info("[GroupId {}][MemberId {}] Member initialized the topology with epoch {}", groupId, memberId, topology.epoch()); + + records.add(newStreamsGroupTopologyRecord(groupId, recordValue)); + } else if (!updatedTopology.equals(group.topology().get())) { + throw new InvalidRequestException("Topology updates are not supported yet."); + } + } else if (group.topology().isPresent()) { + updatedTopology = group.topology().get(); + } else { + throw new IllegalStateException("The topology is null and the group topology is also null."); + } + return updatedTopology; + } + + private List createStreamsGroupHeartbeatResponseTaskIds(final Map> taskIds) { + return taskIds.entrySet().stream() + .map(entry -> new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(entry.getKey()) + .setPartitions(entry.getValue().stream().sorted().toList())) + .collect(Collectors.toList()); + } + /** * Handles a regular heartbeat from a consumer group member. It mainly consists of * three parts: @@ -2485,6 +2959,36 @@ public class GroupMetadataManager { return member; } + /** + * Gets or creates a new dynamic streams group member. + * + * @param group The streams group. + * @param memberId The member id. + * @param memberEpoch The member epoch. + * @param ownedActiveTasks The owned active tasks reported by the member. + * @param ownedStandbyTasks The owned standby tasks reported by the member. + * @param ownedWarmupTasks The owned warmup tasks reported by the member. + * @param memberIsJoining Whether the member should be created or not. + * + * @return The existing streams group member or a new one. + */ + private StreamsGroupMember getOrMaybeCreateDynamicStreamsGroupMember( + StreamsGroup group, + String memberId, + int memberEpoch, + List ownedActiveTasks, + List ownedStandbyTasks, + List ownedWarmupTasks, + boolean memberIsJoining + ) { + StreamsGroupMember member = memberIsJoining ? group.getOrCreateDefaultMember(memberId) : group.getMemberOrThrow(memberId); + throwIfStreamsGroupMemberEpochIsInvalid(member, memberEpoch, ownedActiveTasks, ownedStandbyTasks, ownedWarmupTasks); + if (memberIsJoining) { + log.info("[GroupId {}][MemberId {}] Member joins the streams group.", group.groupId(), memberId); + } + return member; + } + /** * Gets or subscribes a static consumer group member. This method also replaces the * previous static member if allowed. @@ -2962,6 +3466,36 @@ public class GroupMetadataManager { return false; } + /** + * Creates the member metadata record record if the updatedMember is different from + * the old member. Returns true if the metadata has changed, which is always the case + * when a member is first created. + * + * @param groupId The group id. + * @param member The old member. + * @param updatedMember The updated member. + * @param records The list to accumulate any new records. + * @return A boolean indicating whether the group epoch should be bumped + * following this change + */ + private boolean hasStreamsMemberMetadataChanged( + String groupId, + StreamsGroupMember member, + StreamsGroupMember updatedMember, + List records + ) { + String memberId = updatedMember.memberId(); + if (!updatedMember.equals(member)) { + records.add(newStreamsGroupMemberRecord(groupId, updatedMember)); + log.info("[GroupId {}] Member {} updated its member metdata to {}.", + groupId, memberId, updatedMember); + + return true; + } + return false; + } + + /** /** * Reconciles the current assignment of the member towards the target assignment if needed. * @@ -3018,7 +3552,7 @@ public class GroupMetadataManager { updatedMember.rebalanceTimeoutMs() ); } else { - cancelConsumerGroupRebalanceTimeout(groupId, updatedMember.memberId()); + cancelGroupRebalanceTimeout(groupId, updatedMember.memberId()); } } } @@ -3064,6 +3598,87 @@ public class GroupMetadataManager { return updatedMember; } + /** + * Reconciles the current assignment of the member towards the target assignment if needed. + * + * @param groupId The group id. + * @param member The member to reconcile. + * @param currentActiveTaskProcessId The function returning the current process ID of + * a given active task. + * @param currentStandbyTaskProcessIds The function returning the current process IDs of + * a given standby task. + * @param currentWarmupTaskProcessIds The function returning the current process IDs of + * a given warmup task. + * @param targetAssignmentEpoch The target assignment epoch. + * @param targetAssignment The target assignment. + * @param ownedActiveTasks The list of active tasks owned by the member. + * This is reported in the StreamsGroupHeartbeat API, and + * it could be null if not provided. + * @param ownedStandbyTasks The list of standby owned by the member. + * This is reported in the StreamsGroupHeartbeat API, and + * it could be null if not provided. + * @param ownedWarmupTasks The list of warmup tasks owned by the member. + * This is reported in the StreamsGroupHeartbeat API, and + * it could be null if not provided. + * @param records The list to accumulate any new records. + * @return The received member if no changes have been made; or a new + * member containing the new assignment. + */ + private StreamsGroupMember maybeReconcile( + String groupId, + StreamsGroupMember member, + BiFunction currentActiveTaskProcessId, + BiFunction> currentStandbyTaskProcessIds, + BiFunction> currentWarmupTaskProcessIds, + int targetAssignmentEpoch, + org.apache.kafka.coordinator.group.streams.TasksTuple targetAssignment, + List ownedActiveTasks, + List ownedStandbyTasks, + List ownedWarmupTasks, + List records + ) { + if (member.isReconciledTo(targetAssignmentEpoch)) { + return member; + } + + TasksTuple ownedTasks = null; + if (ownedActiveTasks != null && ownedStandbyTasks != null && ownedWarmupTasks != null) { + ownedTasks = TasksTuple.fromHeartbeatRequest(ownedActiveTasks, ownedStandbyTasks, ownedWarmupTasks); + } + + StreamsGroupMember updatedMember = new org.apache.kafka.coordinator.group.streams.CurrentAssignmentBuilder(member) + .withTargetAssignment(targetAssignmentEpoch, targetAssignment) + .withCurrentActiveTaskProcessId(currentActiveTaskProcessId) + .withCurrentStandbyTaskProcessIds(currentStandbyTaskProcessIds) + .withCurrentWarmupTaskProcessIds(currentWarmupTaskProcessIds) + .withOwnedAssignment(ownedTasks) + .build(); + + if (!updatedMember.equals(member)) { + records.add(newStreamsGroupCurrentAssignmentRecord(groupId, updatedMember)); + + log.info("[GroupId {}][MemberId {}] Member's new assignment state: epoch={}, previousEpoch={}, state={}, " + + "assignedTasks={} and tasksPendingRevocation={}.", + groupId, updatedMember.memberId(), updatedMember.memberEpoch(), updatedMember.previousMemberEpoch(), updatedMember.state(), + updatedMember.assignedTasks().toString(), + updatedMember.tasksPendingRevocation().toString()); + + // Schedule/cancel the rebalance timeout. + if (updatedMember.state() == org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS) { + scheduleStreamsGroupRebalanceTimeout( + groupId, + updatedMember.memberId(), + updatedMember.memberEpoch(), + updatedMember.rebalanceTimeoutMs() + ); + } else { + cancelGroupRebalanceTimeout(groupId, updatedMember.memberId()); + } + } + + return updatedMember; + } + /** * Updates the subscription metadata and bumps the group epoch if needed. * @@ -3268,6 +3883,65 @@ public class GroupMetadataManager { } } + /** + * Updates the target assignment according to the updated member and subscription metadata. + * + * @param group The StreamsGroup. + * @param groupEpoch The group epoch. + * @param updatedMember The updated member. + * @param subscriptionMetadata The subscription metadata. + * @param records The list to accumulate any new records. + * @return The new target assignment. + */ + private TasksTuple updateStreamsTargetAssignment( + StreamsGroup group, + int groupEpoch, + StreamsGroupMember updatedMember, + ConfiguredTopology configuredTopology, + Map subscriptionMetadata, + List records + ) { + TaskAssignor assignor = streamsGroupAssignor(group.groupId()); + Map assignmentConfigs = streamsGroupAssignmentConfigs(group.groupId()); + try { + org.apache.kafka.coordinator.group.streams.TargetAssignmentBuilder assignmentResultBuilder = + new org.apache.kafka.coordinator.group.streams.TargetAssignmentBuilder( + group.groupId(), + groupEpoch, + assignor, + assignmentConfigs + ) + .withMembers(group.members()) + .withTopology(configuredTopology) + .withStaticMembers(group.staticMembers()) + .withPartitionMetadata(subscriptionMetadata) + .withTargetAssignment(group.targetAssignment()) + .addOrUpdateMember(updatedMember.memberId(), updatedMember); + + long startTimeMs = time.milliseconds(); + org.apache.kafka.coordinator.group.streams.TargetAssignmentBuilder.TargetAssignmentResult assignmentResult = + assignmentResultBuilder.build(); + long assignorTimeMs = time.milliseconds() - startTimeMs; + + if (log.isDebugEnabled()) { + log.debug("[GroupId {}] Computed a new target assignment for epoch {} with '{}' assignor in {}ms: {}.", + group.groupId(), groupEpoch, assignor, assignorTimeMs, assignmentResult.targetAssignment()); + } else { + log.info("[GroupId {}] Computed a new target assignment for epoch {} with '{}' assignor in {}ms.", + group.groupId(), groupEpoch, assignor, assignorTimeMs); + } + + records.addAll(assignmentResult.records()); + + return assignmentResult.targetAssignment().get(updatedMember.memberId()); + } catch (TaskAssignorException ex) { + String msg = String.format("Failed to compute a new target assignment for epoch %d: %s", + groupEpoch, ex.getMessage()); + log.error("[GroupId {}] {}.", group.groupId(), msg); + throw new UnknownServerException(msg, ex); + } + } + /** * Handles leave request from a consumer group member. * @param groupId The group id from the request. @@ -3308,6 +3982,35 @@ public class GroupMetadataManager { } } + /** + * Handles leave request from a streams group member. + * @param groupId The group id from the request. + * @param memberId The member id from the request. + * @param memberEpoch The member epoch from the request. + * + * @return A result containing the StreamsGroupHeartbeat response and + * a list of records to update the state machine. + */ + private CoordinatorResult streamsGroupLeave( + String groupId, + String instanceId, + String memberId, + int memberEpoch + ) throws ApiException { + StreamsGroup group = streamsGroup(groupId); + StreamsGroupHeartbeatResponseData response = new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(memberEpoch); + + if (instanceId == null) { + StreamsGroupMember member = group.getMemberOrThrow(memberId); + log.info("[GroupId {}][MemberId {}] Member {} left the streams group.", groupId, memberId, memberId); + return streamsGroupFenceMember(group, member, new StreamsGroupHeartbeatResult(response, Map.of())); + } else { + throw new UnsupportedOperationException("Static members are not supported in streams groups."); + } + } + /** * Handles the case when a static member decides to leave the group. * The member is not actually fenced from the group, and instead it's @@ -3363,8 +4066,7 @@ public class GroupMetadataManager { return shareGroupFenceMember(group, member, response); } - /** - * Fences a member from a consumer group and maybe downgrade the consumer group to a classic group. + /** Fences a member from a consumer group and maybe downgrade the consumer group to a classic group. * * @param group The group. * @param member The member. @@ -3613,7 +4315,7 @@ public class GroupMetadataManager { */ private void cancelTimers(String groupId, String memberId) { cancelGroupSessionTimeout(groupId, memberId); - cancelConsumerGroupRebalanceTimeout(groupId, memberId); + cancelGroupRebalanceTimeout(groupId, memberId); cancelConsumerGroupJoinTimeout(groupId, memberId); cancelConsumerGroupSyncTimeout(groupId, memberId); } @@ -3630,7 +4332,7 @@ public class GroupMetadataManager { ) { timer.schedule( groupSessionTimeoutKey(groupId, memberId), - streamsGroupSessionTimeoutMs, + streamsGroupSessionTimeoutMs(groupId), TimeUnit.MILLISECONDS, true, () -> streamsGroupFenceMemberOperation(groupId, memberId, "the member session expired.") @@ -3746,7 +4448,7 @@ public class GroupMetadataManager { ) { try { StreamsGroup group = streamsGroup(groupId); - StreamsGroupMember member = group.getOrMaybeCreateMember(memberId, false); + StreamsGroupMember member = group.getMemberOrThrow(memberId); log.info("[GroupId {}] Streams member {} fenced from the group because {}.", groupId, memberId, reason); @@ -3877,7 +4579,7 @@ public class GroupMetadataManager { timer.schedule(key, rebalanceTimeoutMs, TimeUnit.MILLISECONDS, true, () -> { try { StreamsGroup group = streamsGroup(groupId); - StreamsGroupMember member = group.getOrMaybeCreateMember(memberId, false); + StreamsGroupMember member = group.getMemberOrThrow(memberId); if (member.memberEpoch() == memberEpoch) { log.info("[GroupId {}] Member {} fenced from the group because " + @@ -3908,7 +4610,7 @@ public class GroupMetadataManager { * @param groupId The group id. * @param memberId The member id. */ - private void cancelConsumerGroupRebalanceTimeout( + private void cancelGroupRebalanceTimeout( String groupId, String memberId ) { @@ -4031,14 +4733,46 @@ public class GroupMetadataManager { * @param context The request context. * @param request The actual StreamsGroupHeartbeat request. * - * @return A Result containing the StreamsGroupHeartbeat response, a list of internal topics to create and + * @return A result containing the StreamsGroupHeartbeat response, a list of internal topics to create and * a list of records to update the state machine. */ public CoordinatorResult streamsGroupHeartbeat( RequestContext context, StreamsGroupHeartbeatRequestData request ) throws ApiException { - throw new UnsupportedOperationException("StreamsGroupHeartbeat is not implemented yet."); + throwIfStreamsGroupHeartbeatRequestIsInvalid(request); + + if (request.memberEpoch() == LEAVE_GROUP_MEMBER_EPOCH || request.memberEpoch() == LEAVE_GROUP_STATIC_MEMBER_EPOCH) { + // -1 means that the member wants to leave the group. + // -2 means that a static member wants to leave the group. + return streamsGroupLeave( + request.groupId(), + request.instanceId(), + request.memberId(), + request.memberEpoch() + ); + } else { + return streamsGroupHeartbeat( + request.groupId(), + request.memberId(), + request.memberEpoch(), + request.instanceId(), + request.rackId(), + request.rebalanceTimeoutMs(), + context.clientId(), + context.clientAddress.toString(), + request.topology(), + request.activeTasks(), + request.standbyTasks(), + request.warmupTasks(), + request.processId(), + request.userEndpoint(), + request.clientTags(), + request.taskOffsets(), + request.taskEndOffsets(), + request.shutdownApplication() + ); + } } /** @@ -4688,14 +5422,14 @@ public class GroupMetadataManager { } if (value != null) { - StreamsGroupMember oldMember = streamsGroup.getOrMaybeCreateMember(memberId, true); + StreamsGroupMember oldMember = streamsGroup.getOrCreateUninitializedMember(memberId); streamsGroup.updateMember(new StreamsGroupMember.Builder(oldMember) .updateWith(value) .build()); } else { StreamsGroupMember oldMember; try { - oldMember = streamsGroup.getOrMaybeCreateMember(memberId, false); + oldMember = streamsGroup.getMemberOrThrow(memberId); } catch (UnknownMemberIdException ex) { // If the member does not exist, we can ignore it. return; @@ -4793,7 +5527,7 @@ public class GroupMetadataManager { if (value != null) { StreamsGroup streamsGroup = getOrMaybeCreatePersistedStreamsGroup(groupId, true); - StreamsGroupMember oldMember = streamsGroup.getOrMaybeCreateMember(memberId, true); + StreamsGroupMember oldMember = streamsGroup.getOrCreateUninitializedMember(memberId); StreamsGroupMember newMember = new StreamsGroupMember.Builder(oldMember) .updateWith(value) .build(); @@ -4809,7 +5543,7 @@ public class GroupMetadataManager { StreamsGroupMember oldMember; try { - oldMember = streamsGroup.getOrMaybeCreateMember(memberId, false); + oldMember = streamsGroup.getMemberOrThrow(memberId); } catch (UnknownMemberIdException ex) { // If the member does not exist, we can ignore the tombstone. return; @@ -7387,6 +8121,34 @@ public class GroupMetadataManager { .orElse(config.shareGroupHeartbeatIntervalMs()); } + /** + * Get the session timeout of the provided streams group. + */ + private int streamsGroupSessionTimeoutMs(String groupId) { + return 45000; + } + + /** + * Get the heartbeat interval of the provided streams group. + */ + private int streamsGroupHeartbeatIntervalMs(String groupId) { + return 5000; + } + + /** + * Get the assignor of the provided streams group. + */ + private TaskAssignor streamsGroupAssignor(String groupId) { + return streamsGroupAssignors.get("sticky"); + } + + /** + * Get the assignor of the provided streams group. + */ + private Map streamsGroupAssignmentConfigs(String groupId) { + return Map.of("group.streams.num.standby.replicas", "0"); + } + /** * Generate a classic group heartbeat key for the timer. * diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java index 22f33eb8260..2330497a7be 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java @@ -146,7 +146,7 @@ public class CurrentAssignmentBuilder { * @param ownedAssignment A collection of active, standby and warm-up tasks * @return This object. */ - protected CurrentAssignmentBuilder withOwnedAssignment(TasksTuple ownedAssignment) { + public CurrentAssignmentBuilder withOwnedAssignment(TasksTuple ownedAssignment) { this.ownedTasks = Optional.ofNullable(ownedAssignment); return this; } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java index 723c4e6ab86..dacb041d93b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java @@ -330,31 +330,63 @@ public class StreamsGroup implements Group { } /** - * Gets or creates a new member but without adding it to the group. Adding a member is done via the - * {@link StreamsGroup#updateMember(StreamsGroupMember)} method. + * Gets a new member or throws an exception, if the member does not exist. * - * @param memberId The member ID. - * @param createIfNotExists Booleans indicating whether the member must be created if it does not exist. + * @param memberId The member ID. + * @throws UnknownMemberIdException If the member is not found. * @return A StreamsGroupMember. */ - public StreamsGroupMember getOrMaybeCreateMember( - String memberId, - boolean createIfNotExists + public StreamsGroupMember getMemberOrThrow( + String memberId ) throws UnknownMemberIdException { StreamsGroupMember member = members.get(memberId); if (member != null) { return member; } - if (!createIfNotExists) { - throw new UnknownMemberIdException( - String.format("Member %s is not a member of group %s.", memberId, groupId) - ); + throw new UnknownMemberIdException( + String.format("Member %s is not a member of group %s.", memberId, groupId) + ); + } + + /** + * Gets or creates a new member, but keeping its fields uninitialized. This is used on the replay-path. + * The member is not added to the group, adding a member is done via the + * {@link StreamsGroup#updateMember(StreamsGroupMember)} method. + * + * @param memberId The member ID. + * @return A StreamsGroupMember. + */ + public StreamsGroupMember getOrCreateUninitializedMember( + String memberId + ) throws UnknownMemberIdException { + StreamsGroupMember member = members.get(memberId); + if (member != null) { + return member; } return new StreamsGroupMember.Builder(memberId).build(); } + /** + * Gets or creates a new member, setting default values on the fields. This is used on the replay-path. + * The member is not added to the group, adding a member is done via the + * {@link StreamsGroup#updateMember(StreamsGroupMember)} method. + * + * @param memberId The member ID. + * @return A StreamsGroupMember. + */ + public StreamsGroupMember getOrCreateDefaultMember( + String memberId + ) throws UnknownMemberIdException { + StreamsGroupMember member = members.get(memberId); + if (member != null) { + return member; + } + + return StreamsGroupMember.Builder.withDefaults(memberId).build(); + } + /** * Gets a static member. * @@ -363,7 +395,7 @@ public class StreamsGroup implements Group { */ public StreamsGroupMember staticMember(String instanceId) { String existingMemberId = staticMemberId(instanceId); - return existingMemberId == null ? null : getOrMaybeCreateMember(existingMemberId, false); + return existingMemberId == null ? null : getMemberOrThrow(existingMemberId); } /** @@ -656,7 +688,7 @@ public class StreamsGroup implements Group { memberId.equals(JoinGroupRequest.UNKNOWN_MEMBER_ID) && groupInstanceId == null) return; - final StreamsGroupMember member = getOrMaybeCreateMember(memberId, false); + final StreamsGroupMember member = getMemberOrThrow(memberId); // If the commit is not transactional and the member uses the new streams protocol (KIP-1071), // the member should be using the OffsetCommit API version >= 9. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java index a3862abb5ae..8b475241e2e 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java @@ -278,6 +278,21 @@ public record StreamsGroupMember(String memberId, taskIds -> Set.copyOf(taskIds.partitions()))); } + public static Builder withDefaults(String memberId) { + return new Builder(memberId) + .setRebalanceTimeoutMs(-1) + .setTopologyEpoch(-1) + .setInstanceId(null) + .setRackId(null) + .setProcessId("") + .setClientTags(Collections.emptyMap()) + .setState(MemberState.STABLE) + .setMemberEpoch(0) + .setAssignedTasks(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .setUserEndpoint(null); + } + public StreamsGroupMember build() { return new StreamsGroupMember( memberId, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java index 49ce9f9b4fd..498ff00a595 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.coordinator.group.streams; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.Subtopology; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.TopicInfo; @@ -81,4 +82,17 @@ public record StreamsTopology(int topologyEpoch, record.subtopologies().stream().collect(Collectors.toMap(Subtopology::subtopologyId, x -> x)) ); } + + /** + * Creates an instance of StreamsTopology from a StreamsGroupHeartbeatRequestData request. + * + * @param topology The topology supplied in the request. + * @return The instance of StreamsTopology created from the request. + */ + public static StreamsTopology fromHeartbeatRequest(StreamsGroupHeartbeatRequestData.Topology topology) { + StreamsGroupTopologyValue recordValue = StreamsCoordinatorRecordHelpers.convertToStreamsGroupTopologyRecord(topology); + final Map subtopologyMap = recordValue.subtopologies().stream() + .collect(Collectors.toMap(StreamsGroupTopologyValue.Subtopology::subtopologyId, x -> x)); + return new StreamsTopology(topology.epoch(), subtopologyMap); + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java index 85c22384311..aab08da812a 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java @@ -16,12 +16,16 @@ */ package org.apache.kafka.coordinator.group.streams; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData; import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMemberValue; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -30,11 +34,11 @@ import java.util.stream.Collectors; * An immutable tuple containing active, standby and warm-up tasks. * * @param activeTasks Active tasks. - * The key of the map is the subtopology ID and the value is the set of partition IDs. + * The key of the map is the subtopology ID, and the value is the set of partition IDs. * @param standbyTasks Standby tasks. - * The key of the map is the subtopology ID and the value is the set of partition IDs. + * The key of the map is the subtopology ID, and the value is the set of partition IDs. * @param warmupTasks Warm-up tasks. - * The key of the map is the subtopology ID and the value is the set of partition IDs. + * The key of the map is the subtopology ID, and the value is the set of partition IDs. */ public record TasksTuple(Map> activeTasks, Map> standbyTasks, @@ -88,7 +92,7 @@ public record TasksTuple(Map> activeTasks, /** * Checks if this task tuple contains any of the tasks in another task tuple. * - * @param other The other task tuple. + * @param other Another task tuple. * @return true if there is at least one active, standby or warm-up task that is present in both tuples. */ public boolean containsAny(TasksTuple other) { @@ -130,4 +134,63 @@ public record TasksTuple(Map> activeTasks, ) ); } + + public String toString() { + return "(active=" + taskAssignmentToString(activeTasks) + + ", standby=" + taskAssignmentToString(standbyTasks) + + ", warmup=" + taskAssignmentToString(warmupTasks) + + ')'; + } + + public static TasksTuple fromHeartbeatRequest(final List ownedActiveTasks, + final List ownedStandbyTasks, + final List ownedWarmupTasks) { + return new TasksTuple( + ownedActiveTasks.stream() + .collect(Collectors.toMap( + StreamsGroupHeartbeatRequestData.TaskIds::subtopologyId, + taskId -> new HashSet<>(taskId.partitions()) + ) + ), + ownedStandbyTasks.stream() + .collect(Collectors.toMap( + StreamsGroupHeartbeatRequestData.TaskIds::subtopologyId, + taskId -> new HashSet<>(taskId.partitions()) + ) + ), + ownedWarmupTasks.stream() + .collect(Collectors.toMap( + StreamsGroupHeartbeatRequestData.TaskIds::subtopologyId, + taskId -> new HashSet<>(taskId.partitions()) + ) + ) + ); + } + + /** + * @return The provided assignment as a String. + * + * Example: + * [subtopologyID1-0, subtopologyID1-1, subtopologyID2-0, subtopologyID2-1] + */ + private static String taskAssignmentToString( + Map> assignment + ) { + StringBuilder builder = new StringBuilder("["); + Iterator>> subtopologyIterator = assignment.entrySet().iterator(); + while (subtopologyIterator.hasNext()) { + Map.Entry> entry = subtopologyIterator.next(); + Iterator partitionsIterator = entry.getValue().iterator(); + while (partitionsIterator.hasNext()) { + builder.append(entry.getKey()); + builder.append("-"); + builder.append(partitionsIterator.next()); + if (partitionsIterator.hasNext() || subtopologyIterator.hasNext()) { + builder.append(", "); + } + } + } + builder.append("]"); + return builder.toString(); + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java index cf873992371..d56a8d0d71b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/MemberAssignment.java @@ -23,7 +23,9 @@ import java.util.Set; /** * The task assignment for a Streams group member. * - * @param activeTasks The target tasks assigned to this member keyed by subtopologyId. + * @param activeTasks The active tasks assigned to this member keyed by subtopologyId. + * @param standbyTasks The standby tasks assigned to this member keyed by subtopologyId. + * @param warmupTasks The warm-up tasks assigned to this member keyed by subtopologyId. */ public record MemberAssignment(Map> activeTasks, Map> standbyTasks, 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 c203ffec6ed..4567cdd766a 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 @@ -31,6 +31,7 @@ import org.apache.kafka.common.errors.InvalidRegularExpression; import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.NotLeaderOrFollowerException; import org.apache.kafka.common.errors.RebalanceInProgressException; +import org.apache.kafka.common.errors.StreamsInvalidTopologyException; import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; @@ -57,6 +58,10 @@ import org.apache.kafka.common.message.ShareGroupDescribeResponseData; import org.apache.kafka.common.message.ShareGroupHeartbeatRequestData; import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData.Subtopology; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData.Topology; +import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; import org.apache.kafka.common.message.SyncGroupRequestData; import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment; import org.apache.kafka.common.message.SyncGroupResponseData; @@ -99,15 +104,19 @@ import org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpress import org.apache.kafka.coordinator.group.modern.share.ShareGroup; import org.apache.kafka.coordinator.group.modern.share.ShareGroupBuilder; import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember; +import org.apache.kafka.coordinator.group.streams.MockTaskAssignor; import org.apache.kafka.coordinator.group.streams.StreamsCoordinatorRecordHelpers; import org.apache.kafka.coordinator.group.streams.StreamsGroup; import org.apache.kafka.coordinator.group.streams.StreamsGroup.StreamsGroupState; import org.apache.kafka.coordinator.group.streams.StreamsGroupBuilder; +import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; import org.apache.kafka.coordinator.group.streams.StreamsTopology; import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil; import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole; import org.apache.kafka.coordinator.group.streams.TasksTuple; +import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor; +import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignorException; import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.MetadataProvenance; @@ -171,6 +180,7 @@ import static org.apache.kafka.coordinator.group.GroupMetadataManager.groupRebal import static org.apache.kafka.coordinator.group.GroupMetadataManager.groupSessionTimeoutKey; import static org.apache.kafka.coordinator.group.GroupMetadataManagerTestContext.DEFAULT_CLIENT_ADDRESS; import static org.apache.kafka.coordinator.group.GroupMetadataManagerTestContext.DEFAULT_CLIENT_ID; +import static org.apache.kafka.coordinator.group.GroupMetadataManagerTestContext.DEFAULT_PROCESS_ID; import static org.apache.kafka.coordinator.group.classic.ClassicGroupMember.EMPTY_ASSIGNMENT; import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE; import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD; @@ -4023,9 +4033,9 @@ public class GroupMetadataManagerTest { Utils.mkEntry(StreamsGroup.StreamsGroupState.STABLE, 1L) ))); - context.groupMetadataManager.getOrMaybeCreateStreamsGroup(groupIds.get(1), false) + context.groupMetadataManager.getStreamsGroupOrThrow(groupIds.get(1)) .removeMember(streamsMemberIds.get(0)); - context.groupMetadataManager.getOrMaybeCreateStreamsGroup(groupIds.get(3), false) + context.groupMetadataManager.getStreamsGroupOrThrow(groupIds.get(3)) .updateMember(streamsGroupMemberBuilderWithDefaults(streamsMemberIds.get(2)).setMemberEpoch(10).build()); context.groupMetadataManager.updateGroupSizeCounter(); @@ -4044,13 +4054,15 @@ public class GroupMetadataManagerTest { .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) .setRackId(null) .setInstanceId(null) - .setRebalanceTimeoutMs(1000) + .setRebalanceTimeoutMs(1500) .setAssignedTasks(TasksTuple.EMPTY) .setTasksPendingRevocation(TasksTuple.EMPTY) - .setTopologyEpoch(1) + .setTopologyEpoch(0) .setClientTags(Map.of()) - .setProcessId("process-id") - .setUserEndpoint(new Endpoint().setHost("localhost").setPort(1500)); + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setProcessId(DEFAULT_PROCESS_ID) + .setUserEndpoint(null); } @Test @@ -15444,6 +15456,2066 @@ public class GroupMetadataManagerTest { assertEquals(ShareGroup.ShareGroupState.STABLE, context.shareGroupState(groupId)); } + @Test + public void testStreamsHeartbeatRequestValidation() { + String memberId = Uuid.randomUuid().toString(); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + Exception ex; + + // MemberId must be present in all requests. + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData())); + assertEquals("MemberId can't be empty.", ex.getMessage()); + + // MemberId can't be all whitespaces. + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(" "))); + assertEquals("MemberId can't be empty.", ex.getMessage()); + + // GroupId must be present in all requests. + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId))); + assertEquals("GroupId can't be empty.", ex.getMessage()); + + // GroupId can't be all whitespaces. + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId(" "))); + assertEquals("GroupId can't be empty.", ex.getMessage()); + + // RebalanceTimeoutMs must be present in the first request (epoch == 0). + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(0))); + assertEquals("RebalanceTimeoutMs must be provided in first request.", ex.getMessage()); + + // ActiveTasks must be present and empty in the first request (epoch == 0). + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500))); + assertEquals("ActiveTasks must be empty when (re-)joining.", ex.getMessage()); + + // StandbyTasks must be present and empty in the first request (epoch == 0). + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()))); + assertEquals("StandbyTasks must be empty when (re-)joining.", ex.getMessage()); + + // WarmupTasks must be present and empty in the first request (epoch == 0). + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()))); + assertEquals("WarmupTasks must be empty when (re-)joining.", ex.getMessage()); + + // Topology must be present in the first request (epoch == 0). + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()))); + assertEquals("Topology must be non-null when (re-)joining.", ex.getMessage()); + + // InstanceId must be non-empty if provided in all requests. + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(memberId) + .setMemberEpoch(1) + .setInstanceId(""))); + assertEquals("InstanceId can't be empty.", ex.getMessage()); + + // RackId must be non-empty if provided in all requests. + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(memberId) + .setMemberEpoch(1) + .setRackId(""))); + assertEquals("RackId can't be empty.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(memberId) + .setMemberEpoch(LEAVE_GROUP_STATIC_MEMBER_EPOCH) + .setRebalanceTimeoutMs(1500) + .setTopology(new StreamsGroupHeartbeatRequestData.Topology()) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()))); + assertEquals("InstanceId can't be null.", ex.getMessage()); + + // Member epoch cannot be < -2 + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(-3) + .setRebalanceTimeoutMs(1500) + )); + assertEquals("MemberEpoch is -3, but must be greater than or equal to -2.", ex.getMessage()); + + // Topology must not be present in the later requests (epoch != 0). + ex = assertThrows(InvalidRequestException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(1) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()) + .setTopology(new StreamsGroupHeartbeatRequestData.Topology()) + )); + assertEquals("Topology can only be provided when (re-)joining.", ex.getMessage()); + + // Topology must not contain changelog topics with fixed partition numbers + StreamsInvalidTopologyException topoEx = assertThrows(StreamsInvalidTopologyException.class, () -> context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setMemberId(memberId) + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()) + .setTopology(new StreamsGroupHeartbeatRequestData.Topology().setSubtopologies( + List.of( + new StreamsGroupHeartbeatRequestData.Subtopology() + .setStateChangelogTopics( + List.of( + new StreamsGroupHeartbeatRequestData.TopicInfo() + .setName("changelog_topic_with_fixed_partition") + .setPartitions(3) + ) + ) + ) + )) + )); + assertEquals("Changelog topic changelog_topic_with_fixed_partition must have an undefined partition count, but it is set to 3.", + topoEx.getMessage()); + } + + @Test + public void testUnknownStreamsGroupId() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + + GroupIdNotFoundException e = assertThrows(GroupIdNotFoundException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(100) // Epoch must be > 0. + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()))); + assertEquals("Streams group fooup not found.", e.getMessage()); + } + + @Test + public void testUnknownMemberIdJoinsStreamsGroup() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + Topology topology = new Topology(); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .build(); + + assignor.prepareGroupAssignment(Map.of(memberId, TasksTuple.EMPTY)); + + // A first member joins to create the group. + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + // The second member is rejected because the member id is unknown and + // the member epoch is not zero. + final String memberId2 = Uuid.randomUuid().toString(); + UnknownMemberIdException e = assertThrows(UnknownMemberIdException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(1) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()))); + assertEquals(String.format("Member %s is not a member of group %s.", memberId2, groupId), e.getMessage()); + } + + @Test + public void testStreamsGroupMemberEpochValidation() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .build()) + .build(); + assignor.prepareGroupAssignment(Map.of(memberId, TasksTuple.EMPTY)); + + StreamsGroupMember member = streamsGroupMemberBuilderWithDefaults(memberId) + .setMemberEpoch(100) + .setPreviousMemberEpoch(99) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) + .build(); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, member)); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 100)); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord(groupId, topology)); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3) + ))); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 100)); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, member)); + + // Member epoch is greater than the expected epoch. + FencedMemberEpochException e1 = assertThrows(FencedMemberEpochException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(200) + .setRebalanceTimeoutMs(1500))); + assertEquals("The streams group member has a greater member epoch (200) than the one known by the group coordinator (100). " + + "The member must abandon all its partitions and rejoin.", e1.getMessage()); + + // Member epoch is smaller than the expected epoch. + FencedMemberEpochException e2 = assertThrows(FencedMemberEpochException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(50) + .setRebalanceTimeoutMs(1500))); + assertEquals("The streams group member has a smaller member epoch (50) than the one known by the group coordinator (100). " + + "The member must abandon all its partitions and rejoin.", e2.getMessage()); + + // Member joins with previous epoch but without providing tasks. + FencedMemberEpochException e3 = assertThrows(FencedMemberEpochException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setRebalanceTimeoutMs(1500))); + assertEquals("The streams group member has a smaller member epoch (99) than the one known by the group coordinator (100). " + + "The member must abandon all its partitions and rejoin.", e3.getMessage()); + + // Member joins with previous epoch and has a subset of the owned tasks. + // This is accepted as the response with the bumped epoch may have been lost. + // In this case, we provide back the correct epoch to the member. + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of(new StreamsGroupHeartbeatRequestData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(1, 2)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + assertEquals(100, result.response().data().memberEpoch()); + } + + @Test + public void testMemberJoinsEmptyStreamsGroup() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)), + new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .build(); + + assignor.prepareGroupAssignment(Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2) + ))); + + assertThrows(GroupIdNotFoundException.class, () -> + context.groupMetadataManager.streamsGroup(groupId)); + + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setProcessId("process-id") + .setRebalanceTimeoutMs(1500) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1, 2, 3, 4, 5)), + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(0, 1, 2)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + StreamsGroupMember expectedMember = streamsGroupMemberBuilderWithDefaults(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(1) + .setPreviousMemberEpoch(0) + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setRebalanceTimeoutMs(1500) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2))) + .build(); + + List expectedRecords = List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, expectedMember), + StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord(groupId, topology), + StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 1), + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertRecordsEquals(expectedRecords, result.records()); + } + + @Test + public void testStreamsUpdatingMemberMetadataTriggersNewTargetAssignment() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)), + new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) + .build()) + .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) + .withTargetAssignmentEpoch(10) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) + .withPartitionMetadata(Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) + )) + ) + .build(); + + assignor.prepareGroupAssignment( + Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2) + )) + ); + + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(10) + .setProcessId("process-id2") + ); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1, 2, 3, 4, 5)), + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(0, 1, 2)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + StreamsGroupMember expectedMember = streamsGroupMemberBuilderWithDefaults(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2))) + .setProcessId("process-id2") + .build(); + + List expectedRecords = List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, expectedMember), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertRecordsEquals(expectedRecords, result.records()); + } + + @Test + public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() { + int changedPartitionCount = 6; // New partition count for the topic. + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)), + new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, changedPartitionCount) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) + .build()) + .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) + .withTargetAssignmentEpoch(10) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) + .withPartitionMetadata(Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) + )) + ) + .build(); + + assignor.prepareGroupAssignment( + Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2) + )) + ); + + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(10) + ); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1, 2, 3, 4, 5)), + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(0, 1, 2)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + StreamsGroupMember expectedMember = streamsGroupMemberBuilderWithDefaults(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2))) + .setProcessId("process-id2") + .build(); + + List expectedRecords = List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, changedPartitionCount) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1, 2) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertRecordsEquals(expectedRecords, result.records()); + } + + @Test + public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)), + new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId1) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) + .build()) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .build()) + .withTargetAssignment(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) + .withTargetAssignment(memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .withTargetAssignmentEpoch(10) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) + .withPartitionMetadata(Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) + )) + ) + .build(); + + assignor.prepareGroupAssignment(Map.of( + memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0) + ), + memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 2, 3), + TaskAssignmentTestUtil.mkTasks(subtopology2, 1) + ), + memberId3, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2) + ) + )); + + // Member 3 joins the streams group. + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setTopology(topology) + .setProcessId(DEFAULT_PROCESS_ID) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + } + + @Test + public void testStreamsLeavingMemberRemovesMemberAndBumpsGroupEpoch() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) + .build()) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .build()) + .withTargetAssignment(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) + .withTargetAssignment(memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .withTargetAssignmentEpoch(10)) + .build(); + + // Member 2 leaves the streams group. + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH), + result.response().data() + ); + + List expectedRecords = List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord(groupId, memberId2), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord(groupId, memberId2), + StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord(groupId, memberId2), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11) + ); + + assertRecordsEquals(expectedRecords, result.records()); + } + + @Test + public void testStreamsGroupHeartbeatPartialResponseWhenNothingChanges() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 2) + .build()) + .build(); + + // Prepare new assignment for the group. + assignor.prepareGroupAssignment( + Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1)))); + + CoordinatorResult result; + + // A full response should be sent back on joining. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + // Otherwise, a partial response should be sent back. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(result.response().data().memberEpoch())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000), + result.response().data() + ); + } + + @Test + public void testStreamsReconciliationProcess() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)), + new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) + .build()) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .build()) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) + .withTargetAssignment(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) + .withTargetAssignment(memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .withTargetAssignmentEpoch(10) + .withPartitionMetadata(Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) + )) + ) + .build(); + + // Prepare new assignment for the group. + assignor.prepareGroupAssignment(Map.of( + memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0) + ), + memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 2, 3), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2) + ), + memberId3, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 1) + ) + )); + + CoordinatorResult result; + + // Members in the group are in Stable state. + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.STABLE, context.streamsGroupMemberState(groupId, memberId1)); + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.STABLE, context.streamsGroupMemberState(groupId, memberId2)); + assertEquals(StreamsGroup.StreamsGroupState.STABLE, context.streamsGroupState(groupId)); + + // Member 3 joins the group. This triggers the computation of a new target assignment + // for the group. Member 3 does not get any assigned tasks yet because they are + // all owned by other members. However, it transitions to epoch 11 and the + // Unreleased Tasks state. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + // We only check the last record as the subscription/target assignment updates are + // already covered by other tests. + assertRecordEquals( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId3) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(0) + .build()), + result.records().get(result.records().size() - 1) + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS, + context.streamsGroupMemberState(groupId, memberId3)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 1 heartbeats. It remains at epoch 10 but transitions to Unrevoked Tasks + // state until it acknowledges the revocation of its tasks. The response contains the new + // assignment without the tasks that must be revoked. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(10)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1)), + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(0)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertRecordsEquals(List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId1) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0))) + .setTasksPendingRevocation(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 2), + TaskAssignmentTestUtil.mkTasks(subtopology2, 1))) + .build())), + result.records() + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS, + context.streamsGroupMemberState(groupId, memberId1)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 2 heartbeats. It remains at epoch 10 but transitions to Unrevoked Tasks + // state until it acknowledges the revocation of its tasks. The response contains the new + // assignment without the tasks that must be revoked. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(3)), + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(2)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertRecordsEquals(List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId2) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 3), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .setTasksPendingRevocation(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5))) + .build())), + result.records() + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS, + context.streamsGroupMemberState(groupId, memberId2)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 3 heartbeats. The response does not contain any assignment + // because the member is still waiting on other members to revoke tasks. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000), + result.response().data() + ); + + assertRecordsEquals(List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId3) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .build())), + result.records() + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS, + context.streamsGroupMemberState(groupId, memberId3)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 1 acknowledges the revocation of the tasks. It does so by providing the + // tasks that it still owns in the request. This allows him to transition to epoch 11 + // and to the Stable state. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(10) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatRequestData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1)), + new StreamsGroupHeartbeatRequestData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(0)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000), + result.response().data() + ); + + assertRecordsEquals(List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId1) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0))) + .build())), + result.records() + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.STABLE, context.streamsGroupMemberState(groupId, memberId1)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 2 heartbeats but without acknowledging the revocation yet. This is basically a no-op. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000), + result.response().data() + ); + + assertEquals(List.of(), result.records()); + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS, + context.streamsGroupMemberState(groupId, memberId2)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 3 heartbeats. It receives the tasks revoked by member 1 but remains + // in Unreleased tasks state because it still waits on other tasks. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(1)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertRecordsEquals(List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId3) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology2, 1))) + .build())), + result.records() + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS, + context.streamsGroupMemberState(groupId, memberId3)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 3 heartbeats. Member 2 has not acknowledged the revocation of its tasks so + // member keeps its current assignment. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000), + result.response().data() + ); + + assertEquals(List.of(), result.records()); + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS, + context.streamsGroupMemberState(groupId, memberId3)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 2 acknowledges the revocation of the tasks. It does so by providing the + // tasks that it still owns in the request. This allows him to transition to epoch 11 + // and to the Stable state. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatRequestData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(3)), + new StreamsGroupHeartbeatRequestData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(2)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()) + ); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(2, 3)), + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(2)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertRecordsEquals(List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId2) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 2, 3), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) + .build())), + result.records() + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.STABLE, context.streamsGroupMemberState(groupId, memberId2)); + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + // Member 3 heartbeats to acknowledge its current assignment. It receives all its tasks and + // transitions to Stable state. + result = context.streamsGroupHeartbeat(new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatRequestData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(1)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(4, 5)), + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology2) + .setPartitions(List.of(1)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertRecordsEquals(List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId3) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 1))) + .build())), + result.records() + ); + + assertEquals(org.apache.kafka.coordinator.group.streams.MemberState.STABLE, context.streamsGroupMemberState(groupId, memberId3)); + assertEquals(StreamsGroup.StreamsGroupState.STABLE, context.streamsGroupState(groupId)); + } + + @Test + public void testStreamsStreamsGroupStates() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)), + new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10)) + .build(); + + assertEquals(StreamsGroup.StreamsGroupState.EMPTY, context.streamsGroupState(groupId)); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord(groupId, topology)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId1) + .build())); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11)); + + assertEquals(StreamsGroupState.NOT_READY, context.streamsGroupState(groupId)); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, + Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) + ) + )); + + assertEquals(StreamsGroup.StreamsGroupState.ASSIGNING, context.streamsGroupState(groupId)); + + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId1, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3)))); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11)); + + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + context.replay( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId1) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks( + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) + .build())); + + assertEquals(StreamsGroup.StreamsGroupState.RECONCILING, context.streamsGroupState(groupId)); + + context.replay( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId1) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks( + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 1, 2, 3))) + .build())); + + assertEquals(StreamsGroup.StreamsGroupState.STABLE, context.streamsGroupState(groupId)); + } + + @Test + public void testStreamsTaskAssignorExceptionOnRegularHeartbeat() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + String subtopology2 = "subtopology2"; + String barTopicName = "bar"; + Uuid barTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)), + new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName)) + )); + + TaskAssignor assignor = mock(TaskAssignor.class); + when(assignor.name()).thenReturn("sticky"); + when(assignor.assign(any(), any())).thenThrow(new TaskAssignorException("Assignment failed.")); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .build(); + + // Member 1 joins the streams group. The request fails because the + // target assignment computation failed. + UnknownServerException e = assertThrows(UnknownServerException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(1500) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()))); + assertEquals("Failed to compute a new target assignment for epoch 1: Assignment failed.", e.getMessage()); + } + + @Test + public void testStreamsPartitionMetadataRefreshedAfterGroupIsLoaded() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) + .build()) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) + .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) + .withTargetAssignmentEpoch(10) + .withPartitionMetadata( + // foo only has 3 tasks stored in the metadata but foo has + // 6 partitions the metadata image. + Map.of(fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 3)) + )) + .build(); + + // The metadata refresh flag should be true. + StreamsGroup streamsGroup = context.groupMetadataManager + .streamsGroup(groupId); + assertTrue(streamsGroup.hasMetadataExpired(context.time.milliseconds())); + + // Prepare the assignment result. + assignor.prepareGroupAssignment(Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) + ))); + + // Heartbeat. + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(10)); + + // The member gets tasks 3, 4 and 5 assigned. + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1, 2, 3, 4, 5)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + StreamsGroupMember expectedMember = streamsGroupMemberBuilderWithDefaults(memberId) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) + .build(); + + List expectedRecords = List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, + Map.of(fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6)) + ), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertRecordsEquals(expectedRecords, result.records()); + + // Check next refresh time. + assertFalse(streamsGroup.hasMetadataExpired(context.time.milliseconds())); + assertEquals(context.time.milliseconds() + Integer.MAX_VALUE, streamsGroup.metadataRefreshDeadline().deadlineMs); + assertEquals(11, streamsGroup.metadataRefreshDeadline().epoch); + } + + @Test + public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId) + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) + .build()) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) + .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) + .withTargetAssignmentEpoch(10) + .withPartitionMetadata( + // foo only has 3 partitions stored in the metadata but foo has + // 6 partitions the metadata image. + Map.of(fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 3)) + )) + .build(); + + // The metadata refresh flag should be true. + StreamsGroup streamsGroup = context.groupMetadataManager + .streamsGroup(groupId); + assertTrue(streamsGroup.hasMetadataExpired(context.time.milliseconds())); + + // Prepare the assignment result. + assignor.prepareGroupAssignment( + Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) + )) + ); + + // Heartbeat. + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(10)); + + // The metadata refresh flag is set to a future time. + assertFalse(streamsGroup.hasMetadataExpired(context.time.milliseconds())); + assertEquals(context.time.milliseconds() + Integer.MAX_VALUE, streamsGroup.metadataRefreshDeadline().deadlineMs); + assertEquals(11, streamsGroup.metadataRefreshDeadline().epoch); + + // Rollback the uncommitted changes. This does not rollback the metadata flag + // because it is not using a timeline data structure. + context.rollback(); + + // However, the next heartbeat should detect the divergence based on the epoch and trigger + // a metadata refresh. + CoordinatorResult result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(10)); + + // The member gets tasks 3, 4 and 5 assigned. + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1, 2, 3, 4, 5)) + )) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + StreamsGroupMember expectedMember = streamsGroupMemberBuilderWithDefaults(memberId) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + List expectedRecords = List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, + Map.of(fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6)) + ), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertRecordsEquals(expectedRecords, result.records()); + + // Check next refresh time. + assertFalse(streamsGroup.hasMetadataExpired(context.time.milliseconds())); + assertEquals(context.time.milliseconds() + Integer.MAX_VALUE, streamsGroup.metadataRefreshDeadline().deadlineMs); + assertEquals(11, streamsGroup.metadataRefreshDeadline().epoch); + } + + @Test + public void testStreamsSessionTimeoutLifecycle() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .build()) + .build(); + + assignor.prepareGroupAssignment(Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) + ))); + + // Session timer is scheduled on first heartbeat. + CoordinatorResult result = + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(90000) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + assertEquals(1, result.response().data().memberEpoch()); + + // Verify that there is a session time. + context.assertSessionTimeout(groupId, memberId, 45000); + + // Advance time. + assertEquals( + List.of(), + context.sleep(result.response().data().heartbeatIntervalMs()) + ); + + // Session timer is rescheduled on second heartbeat. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(result.response().data().memberEpoch())); + assertEquals(1, result.response().data().memberEpoch()); + + // Verify that there is a session time. + context.assertSessionTimeout(groupId, memberId, 45000); + + // Advance time. + assertEquals( + List.of(), + context.sleep(result.response().data().heartbeatIntervalMs()) + ); + + // Session timer is cancelled on leave. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH)); + assertEquals(LEAVE_GROUP_MEMBER_EPOCH, result.response().data().memberEpoch()); + + // Verify that there are no timers. + context.assertNoSessionTimeout(groupId, memberId); + context.assertNoRebalanceTimeout(groupId, memberId); + } + + @Test + public void testStreamsSessionTimeoutExpiration() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .build()) + .build(); + + assignor.prepareGroupAssignment(Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) + ))); + + // Session timer is scheduled on first heartbeat. + CoordinatorResult result = + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(90000) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + assertEquals(1, result.response().data().memberEpoch()); + + // Verify that there is a session time. + context.assertSessionTimeout(groupId, memberId, 45000); + + // Advance time past the session timeout. + List> timeouts = context.sleep(45000 + 1); + + // Verify the expired timeout. + assertEquals( + List.of(new ExpiredTimeout( + groupSessionTimeoutKey(groupId, memberId), + new CoordinatorResult<>( + List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord(groupId, memberId), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord(groupId, memberId), + StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord(groupId, memberId), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 2) + ) + ) + )), + timeouts + ); + + // Verify that there are no timers. + context.assertNoSessionTimeout(groupId, memberId); + context.assertNoRebalanceTimeout(groupId, memberId); + } + + @Test + public void testStreamsRebalanceTimeoutLifecycle() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 3) + .build()) + .build(); + + assignor.prepareGroupAssignment(Map.of(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2) + ))); + + // Member 1 joins the group. + CoordinatorResult result = + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(12000) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1, 2)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertEquals( + List.of(), + context.sleep(result.response().data().heartbeatIntervalMs()) + ); + + // Prepare next assignment. + assignor.prepareGroupAssignment(Map.of( + memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1) + ), + memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 2) + ) + )); + + // Member 2 joins the group. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(90000) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(2) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertEquals( + List.of(), + context.sleep(result.response().data().heartbeatIntervalMs()) + ); + + // Member 1 heartbeats and transitions to unrevoked tasks. The rebalance timeout + // is scheduled. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(1) + .setRebalanceTimeoutMs(12000)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + // Verify that there is a revocation timeout. Keep a reference + // to the timeout for later. + ScheduledTimeout scheduledTimeout = + context.assertRebalanceTimeout(groupId, memberId1, 12000); + + assertEquals( + List.of(), + context.sleep(result.response().data().heartbeatIntervalMs()) + ); + + // Member 1 acks the revocation. The revocation timeout is cancelled. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(1) + .setActiveTasks(List.of(new StreamsGroupHeartbeatRequestData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(2) + .setHeartbeatIntervalMs(5000), + result.response().data() + ); + + // Verify that there is not revocation timeout. + context.assertNoRebalanceTimeout(groupId, memberId1); + + // Execute the scheduled revocation timeout captured earlier to simulate a + // stale timeout. This should be a no-op. + assertEquals(List.of(), scheduledTimeout.operation.generateRecords().records()); + } + + @Test + public void testStreamsRebalanceTimeoutExpiration() { + final int rebalanceTimeoutMs = 10000; + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 3) + .build()) + .build(); + + assignor.prepareGroupAssignment( + Map.of(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2)))); + + // Member 1 joins the group. + CoordinatorResult result = + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(rebalanceTimeoutMs) // Use timeout smaller than session timeout. + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1, 2)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertEquals( + List.of(), + context.sleep(result.response().data().heartbeatIntervalMs()) + ); + + // Prepare next assignment. + assignor.prepareGroupAssignment(Map.of( + memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1) + ), + memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 2) + ) + )); + + // Member 2 joins the group. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(rebalanceTimeoutMs) + .setTopology(topology) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of())); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(2) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + assertEquals( + List.of(), + context.sleep(result.response().data().heartbeatIntervalMs()) + ); + + // Member 1 heartbeats and transitions to revoking. The revocation timeout + // is scheduled. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(1)); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setActiveTasks(List.of( + new StreamsGroupHeartbeatResponseData.TaskIds() + .setSubtopologyId(subtopology1) + .setPartitions(List.of(0, 1)))) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()), + result.response().data() + ); + + // Advance time past the revocation timeout. + List> timeouts = context.sleep(rebalanceTimeoutMs + 1); + + // Verify the expired timeout. + assertEquals( + List.of(new ExpiredTimeout( + groupRebalanceTimeoutKey(groupId, memberId1), + new CoordinatorResult<>( + List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord(groupId, memberId1), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord(groupId, memberId1), + StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord(groupId, memberId1), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 3) + ) + ) + )), + timeouts + ); + + // Verify that there are no timers. + context.assertNoSessionTimeout(groupId, memberId1); + context.assertNoRebalanceTimeout(groupId, memberId1); + } + @Test public void testConsumerGroupDynamicConfigs() { String groupId = "fooup"; @@ -15848,7 +17920,7 @@ public class GroupMetadataManagerTest { // The group and the member are created if they do not exist. context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord("foo", member)); - assertEquals(member, context.groupMetadataManager.streamsGroup("foo").getOrMaybeCreateMember("member", false)); + assertEquals(member, context.groupMetadataManager.streamsGroup("foo").getMemberOrThrow("member")); } @Test @@ -15860,7 +17932,7 @@ public class GroupMetadataManagerTest { // StreamsGroupMemberMetadata tombstone should be a no-op. context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord("foo", "m1")); - assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.streamsGroup("foo").getOrMaybeCreateMember("m1", false)); + assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.streamsGroup("foo").getMemberOrThrow("m1")); // The group may not exist at all. Replaying the StreamsGroupMemberMetadata tombstone // should be a no-op. @@ -16135,7 +18207,7 @@ public class GroupMetadataManagerTest { // The group and the member are created if they do not exist. context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord("bar", member)); - assertEquals(member, context.groupMetadataManager.streamsGroup("bar").getOrMaybeCreateMember("member", false)); + assertEquals(member, context.groupMetadataManager.streamsGroup("bar").getMemberOrThrow("member")); } @Test @@ -16147,7 +18219,7 @@ public class GroupMetadataManagerTest { // StreamsGroupCurrentMemberAssignment tombstone should be a no-op. context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord("foo", "m1")); - assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.streamsGroup("foo").getOrMaybeCreateMember("m1", false)); + assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.streamsGroup("foo").getMemberOrThrow("m1")); // The group may not exist at all. Replaying the StreamsGroupCurrentMemberAssignment tombstone // should be a no-op. @@ -16179,7 +18251,7 @@ public class GroupMetadataManagerTest { context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord("foo", "m1")); - final StreamsGroupMember member = context.groupMetadataManager.streamsGroup("foo").getOrMaybeCreateMember("m1", false); + final StreamsGroupMember member = context.groupMetadataManager.streamsGroup("foo").getMemberOrThrow("m1"); assertEquals(LEAVE_GROUP_MEMBER_EPOCH, member.memberEpoch()); assertEquals(LEAVE_GROUP_MEMBER_EPOCH, member.previousMemberEpoch()); assertTrue(member.assignedTasks().isEmpty()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java index 13242caecdc..846c0a3d475 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java @@ -108,8 +108,13 @@ import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup; import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupBuilder; import org.apache.kafka.coordinator.group.modern.share.ShareGroup; import org.apache.kafka.coordinator.group.modern.share.ShareGroupBuilder; +import org.apache.kafka.coordinator.group.streams.MockTaskAssignor; +import org.apache.kafka.coordinator.group.streams.StreamsGroup; import org.apache.kafka.coordinator.group.streams.StreamsGroupBuilder; import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult; +import org.apache.kafka.coordinator.group.streams.StreamsGroupMember; +import org.apache.kafka.coordinator.group.streams.TasksTuple; +import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor; import org.apache.kafka.image.MetadataImage; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.common.ApiMessageAndVersion; @@ -158,6 +163,7 @@ import static org.mockito.Mockito.mock; public class GroupMetadataManagerTestContext { static final String DEFAULT_CLIENT_ID = "client"; static final InetAddress DEFAULT_CLIENT_ADDRESS = InetAddress.getLoopbackAddress(); + static final String DEFAULT_PROCESS_ID = "process-id"; private static class GroupCoordinatorConfigContext extends GroupCoordinatorConfig { GroupCoordinatorConfigContext(AbstractConfig config) { @@ -465,6 +471,7 @@ public class GroupMetadataManagerTestContext { private final List shareGroupBuilders = new ArrayList<>(); private final Map config = new HashMap<>(); private Optional authorizer = Optional.empty(); + private List streamsGroupAssignors = Collections.singletonList(new MockTaskAssignor("mock")); public Builder withConfig(String key, Object value) { config.put(key, value); @@ -500,6 +507,11 @@ public class GroupMetadataManagerTestContext { this.authorizer = Optional.of(authorizer); return this; } + + public Builder withStreamsGroupTaskAssignors(List assignors) { + this.streamsGroupAssignors = assignors; + return this; + } public GroupMetadataManagerTestContext build() { if (metadataImage == null) metadataImage = MetadataImage.EMPTY; @@ -531,6 +543,7 @@ public class GroupMetadataManagerTestContext { .withShareGroupAssignor(shareGroupAssignor) .withGroupConfigManager(groupConfigManager) .withAuthorizer(authorizer) + .withStreamsGroupAssignors(streamsGroupAssignors) .build(), groupConfigManager ); @@ -607,6 +620,14 @@ public class GroupMetadataManagerTestContext { .state(); } + public StreamsGroup.StreamsGroupState streamsGroupState( + String groupId + ) { + return groupMetadataManager + .streamsGroup(groupId) + .state(); + } + public MemberState consumerGroupMemberState( String groupId, String memberId @@ -617,6 +638,16 @@ public class GroupMetadataManagerTestContext { .state(); } + public org.apache.kafka.coordinator.group.streams.MemberState streamsGroupMemberState( + String groupId, + String memberId + ) { + return groupMetadataManager + .streamsGroup(groupId) + .getMemberOrThrow(memberId) + .state(); + } + public CoordinatorResult consumerGroupHeartbeat( ConsumerGroupHeartbeatRequestData request ) { @@ -1746,4 +1777,22 @@ public class GroupMetadataManagerTestContext { public void updateGroupConfig(String groupId, Properties newGroupConfig) { groupConfigManager.updateGroupConfig(groupId, newGroupConfig); } + + public static StreamsGroupMember.Builder streamsGroupMemberBuilderWithDefaults(String memberId) { + return new StreamsGroupMember.Builder(memberId) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setAssignedTasks(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .setClientId(DEFAULT_CLIENT_ID) + .setClientHost(DEFAULT_CLIENT_ADDRESS.toString()) + .setRackId(null) + .setInstanceId(null) + .setRebalanceTimeoutMs(1500) + .setAssignedTasks(TasksTuple.EMPTY) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .setTopologyEpoch(0) + .setClientTags(Map.of()) + .setProcessId(DEFAULT_PROCESS_ID) + .setUserEndpoint(null); + } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/MockTaskAssignor.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/MockTaskAssignor.java new file mode 100644 index 00000000000..ee38d6b1300 --- /dev/null +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/MockTaskAssignor.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.streams; + +import org.apache.kafka.coordinator.group.streams.assignor.GroupAssignment; +import org.apache.kafka.coordinator.group.streams.assignor.GroupSpec; +import org.apache.kafka.coordinator.group.streams.assignor.MemberAssignment; +import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor; +import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignorException; +import org.apache.kafka.coordinator.group.streams.assignor.TopologyDescriber; + +import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Collectors; + +public class MockTaskAssignor implements TaskAssignor { + + private final String name; + private GroupAssignment preparedGroupAssignment = null; + + public MockTaskAssignor(String name) { + this.name = name; + } + + public void prepareGroupAssignment(GroupAssignment prepareGroupAssignment) { + this.preparedGroupAssignment = prepareGroupAssignment; + } + + public void prepareGroupAssignment(Map memberAssignments) { + this.preparedGroupAssignment = + new GroupAssignment(memberAssignments.entrySet().stream().collect( + Collectors.toMap( + Entry::getKey, + entry -> { + TasksTuple tasksTuple = entry.getValue(); + return new MemberAssignment( + tasksTuple.activeTasks(), tasksTuple.standbyTasks(), tasksTuple.warmupTasks()); + }))); + } + + @Override + public String name() { + return name; + } + + @Override + public GroupAssignment assign(final GroupSpec groupSpec, final TopologyDescriber topologyDescriber) + throws TaskAssignorException { + return preparedGroupAssignment; + } +} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java index e35d0baa210..b8b52267a3f 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java @@ -18,8 +18,10 @@ package org.apache.kafka.coordinator.group.streams; import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; +import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.Subtopology; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -97,7 +99,7 @@ public class StreamsGroupBuilder { groupId, new StreamsGroupTopologyValue() .setEpoch(topology.topologyEpoch()) - .setSubtopologies(topology.subtopologies().values().stream().sorted().toList())) + .setSubtopologies(topology.subtopologies().values().stream().sorted(Comparator.comparing(Subtopology::subtopologyId)).toList())) ); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java index a2d30c2db44..fe8161df615 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java @@ -74,6 +74,7 @@ import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil. import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -100,23 +101,53 @@ public class StreamsGroupTest { } @Test - public void testGetOrCreateMember() { + public void testGetOrCreateUninitializedMember() { + StreamsGroup streamsGroup = createStreamsGroup("foo"); + StreamsGroupMember uninitializedMember = new StreamsGroupMember.Builder("member-id").build(); + StreamsGroupMember member = streamsGroup.getOrCreateUninitializedMember("member-id"); + + assertEquals(uninitializedMember, member); + + StreamsGroupMember updatedMember = new StreamsGroupMember.Builder(member).setInstanceId("unique-new-id").build(); + streamsGroup.updateMember(updatedMember); + + assertEquals(updatedMember, streamsGroup.getOrCreateUninitializedMember("member-id")); + assertNotEquals(uninitializedMember, streamsGroup.getOrCreateUninitializedMember("member-id")); + } + + @Test + public void testGetOrCreateDefaultMember() { + StreamsGroup streamsGroup = createStreamsGroup("foo"); + StreamsGroupMember defaultMember = StreamsGroupMember.Builder.withDefaults("member-id").build(); + StreamsGroupMember member = streamsGroup.getOrCreateDefaultMember("member-id"); + + assertEquals(defaultMember, member); + + StreamsGroupMember updatedMember = new StreamsGroupMember.Builder(member).setInstanceId("unique-new-id").build(); + streamsGroup.updateMember(updatedMember); + + assertEquals(updatedMember, streamsGroup.getOrCreateDefaultMember("member-id")); + assertNotEquals(defaultMember, streamsGroup.getOrCreateDefaultMember("member-id")); + } + + @Test + public void testGetMemberOrThrow() { StreamsGroup streamsGroup = createStreamsGroup("foo"); StreamsGroupMember member; // Create a member. - member = streamsGroup.getOrMaybeCreateMember("member-id", true); + member = streamsGroup.getOrCreateDefaultMember("member-id"); assertEquals("member-id", member.memberId()); // Add member to the group. streamsGroup.updateMember(member); // Get that member back. - member = streamsGroup.getOrMaybeCreateMember("member-id", false); + member = streamsGroup.getMemberOrThrow("member-id"); assertEquals("member-id", member.memberId()); assertThrows(UnknownMemberIdException.class, () -> - streamsGroup.getOrMaybeCreateMember("does-not-exist", false)); + streamsGroup.getMemberOrThrow("does-not-exist")); } @Test @@ -124,13 +155,13 @@ public class StreamsGroupTest { StreamsGroup streamsGroup = createStreamsGroup("foo"); StreamsGroupMember member; - member = streamsGroup.getOrMaybeCreateMember("member", true); + member = streamsGroup.getOrCreateDefaultMember("member"); member = new StreamsGroupMember.Builder(member).build(); streamsGroup.updateMember(member); - assertEquals(member, streamsGroup.getOrMaybeCreateMember("member", false)); + assertEquals(member, streamsGroup.getMemberOrThrow("member")); } @Test @@ -138,7 +169,7 @@ public class StreamsGroupTest { StreamsGroup streamsGroup = createStreamsGroup("foo"); // Create a new member which is not static - streamsGroup.getOrMaybeCreateMember("member", true); + streamsGroup.getOrCreateDefaultMember("member"); assertNull(streamsGroup.staticMember("instance-id")); } @@ -147,7 +178,7 @@ public class StreamsGroupTest { StreamsGroup streamsGroup = createStreamsGroup("foo"); StreamsGroupMember member; - member = streamsGroup.getOrMaybeCreateMember("member", true); + member = streamsGroup.getOrCreateDefaultMember("member"); member = new StreamsGroupMember.Builder(member) .setInstanceId("instance") @@ -156,7 +187,7 @@ public class StreamsGroupTest { streamsGroup.updateMember(member); assertEquals(member, streamsGroup.staticMember("instance")); - assertEquals(member, streamsGroup.getOrMaybeCreateMember("member", false)); + assertEquals(member, streamsGroup.getMemberOrThrow("member")); assertEquals(member.memberId(), streamsGroup.staticMemberId("instance")); } @@ -164,13 +195,12 @@ public class StreamsGroupTest { public void testRemoveMember() { StreamsGroup streamsGroup = createStreamsGroup("foo"); - StreamsGroupMember member = streamsGroup.getOrMaybeCreateMember("member", true); + StreamsGroupMember member = streamsGroup.getOrCreateDefaultMember("member"); streamsGroup.updateMember(member); assertTrue(streamsGroup.hasMember("member")); streamsGroup.removeMember("member"); assertFalse(streamsGroup.hasMember("member")); - } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java index 89c785d633e..a9d269f2120 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsTopologyTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.coordinator.group.streams; +import org.apache.kafka.common.message.StreamsGroupHeartbeatRequestData; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.Subtopology; import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.TopicInfo; @@ -103,6 +104,22 @@ public class StreamsTopologyTest { assertEquals(mkSubtopology2(), topology.subtopologies().get(SUBTOPOLOGY_ID_2)); } + @Test + public void fromHeartbeatRequestShouldCreateCorrectTopology() { + StreamsGroupHeartbeatRequestData.Topology requestTopology = new StreamsGroupHeartbeatRequestData.Topology() + .setEpoch(1) + .setSubtopologies(List.of(mkRequestSubtopology1(), mkRequestSubtopology2())); + + StreamsTopology topology = StreamsTopology.fromHeartbeatRequest(requestTopology); + + assertEquals(1, topology.topologyEpoch()); + assertEquals(2, topology.subtopologies().size()); + assertTrue(topology.subtopologies().containsKey(SUBTOPOLOGY_ID_1)); + assertEquals(mkSubtopology1(), topology.subtopologies().get(SUBTOPOLOGY_ID_1)); + assertTrue(topology.subtopologies().containsKey(SUBTOPOLOGY_ID_2)); + assertEquals(mkSubtopology2(), topology.subtopologies().get(SUBTOPOLOGY_ID_2)); + } + private Subtopology mkSubtopology1() { return new Subtopology() .setSubtopologyId(SUBTOPOLOGY_ID_1) @@ -147,4 +164,49 @@ public class StreamsTopologyTest { new TopicInfo().setName(CHANGELOG_TOPIC_3) )); } + + private StreamsGroupHeartbeatRequestData.Subtopology mkRequestSubtopology1() { + return new StreamsGroupHeartbeatRequestData.Subtopology() + .setSubtopologyId(SUBTOPOLOGY_ID_1) + .setSourceTopics(List.of( + SOURCE_TOPIC_1, + SOURCE_TOPIC_2, + REPARTITION_TOPIC_1, + REPARTITION_TOPIC_2 + )) + .setRepartitionSourceTopics(List.of( + new StreamsGroupHeartbeatRequestData.TopicInfo().setName(REPARTITION_TOPIC_1), + new StreamsGroupHeartbeatRequestData.TopicInfo().setName(REPARTITION_TOPIC_2) + )) + .setRepartitionSinkTopics(List.of( + REPARTITION_TOPIC_3 + )) + .setStateChangelogTopics(List.of( + new StreamsGroupHeartbeatRequestData.TopicInfo().setName(CHANGELOG_TOPIC_1), + new StreamsGroupHeartbeatRequestData.TopicInfo().setName(CHANGELOG_TOPIC_2) + )) + .setCopartitionGroups(List.of( + new StreamsGroupHeartbeatRequestData.CopartitionGroup() + .setRepartitionSourceTopics(List.of((short) 0)) + .setSourceTopics(List.of((short) 0)), + new StreamsGroupHeartbeatRequestData.CopartitionGroup() + .setRepartitionSourceTopics(List.of((short) 1)) + .setSourceTopics(List.of((short) 1)) + )); + } + + private StreamsGroupHeartbeatRequestData.Subtopology mkRequestSubtopology2() { + return new StreamsGroupHeartbeatRequestData.Subtopology() + .setSubtopologyId(SUBTOPOLOGY_ID_2) + .setSourceTopics(List.of( + SOURCE_TOPIC_3, + REPARTITION_TOPIC_3 + )) + .setRepartitionSourceTopics(List.of( + new StreamsGroupHeartbeatRequestData.TopicInfo().setName(REPARTITION_TOPIC_3) + )) + .setStateChangelogTopics(List.of( + new StreamsGroupHeartbeatRequestData.TopicInfo().setName(CHANGELOG_TOPIC_3) + )); + } }