From 1e3a1facb31efee5e2242e5e817a44f7c9fdb9e8 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Wed, 5 Mar 2025 15:09:38 +0100 Subject: [PATCH 1/8] KAFKA-18613: Add StreamsGroupHeartbeat handler in the group coordinator Basic streams group heartbeat handling. - No support for static membership - No support for configurations (using constants instead) - No support for regular expressions --- .../group/GroupMetadataManager.java | 775 ++++++- .../streams/CurrentAssignmentBuilder.java | 2 +- .../group/streams/StreamsGroup.java | 26 + .../group/streams/StreamsGroupMember.java | 15 + .../coordinator/group/streams/TasksTuple.java | 71 +- .../streams/assignor/MemberAssignment.java | 4 +- .../group/GroupMetadataManagerTest.java | 2017 ++++++++++++++++- .../GroupMetadataManagerTestContext.java | 49 + .../group/streams/MockTaskAssignor.java | 65 + .../group/streams/StreamsGroupBuilder.java | 4 +- 10 files changed, 3007 insertions(+), 21 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/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 4985c01376013..dee448aa4108d 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.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.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,11 @@ 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.StreamsGroupHeartbeatResponseData; import org.apache.kafka.common.message.SyncGroupRequestData; import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.protocol.Errors; @@ -63,6 +70,7 @@ 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; @@ -140,6 +148,12 @@ 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; @@ -164,6 +178,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -171,6 +186,7 @@ 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; @@ -223,11 +239,18 @@ 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; /** @@ -270,6 +293,7 @@ public static class Builder { private ShareGroupPartitionAssignor shareGroupAssignor = null; private GroupCoordinatorMetricsShard metrics; private Optional authorizer = null; + private List streamsGroupAssignors = null; Builder withLogContext(LogContext logContext) { this.logContext = logContext; @@ -306,6 +330,11 @@ Builder withGroupConfigManager(GroupConfigManager groupConfigManager) { return this; } + Builder withStreamsGroupAssignors(List streamsGroupAssignors) { + this.streamsGroupAssignors = streamsGroupAssignors; + return this; + } + Builder withMetadataImage(MetadataImage metadataImage) { this.metadataImage = metadataImage; return this; @@ -345,6 +374,8 @@ GroupMetadataManager build() { 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, @@ -357,7 +388,8 @@ GroupMetadataManager build() { config, groupConfigManager, shareGroupAssignor, - authorizer + authorizer, + streamsGroupAssignors ); } } @@ -434,9 +466,9 @@ GroupMetadataManager build() { 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. @@ -479,7 +511,8 @@ private GroupMetadataManager( GroupCoordinatorConfig config, GroupConfigManager groupConfigManager, ShareGroupPartitionAssignor shareGroupAssignor, - Optional authorizer + Optional authorizer, + List streamsGroupAssignors ) { this.logContext = logContext; this.log = logContext.logger(GroupMetadataManager.class); @@ -499,8 +532,8 @@ private GroupMetadataManager( this.groupsByTopics = 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())); } /** @@ -1348,6 +1381,34 @@ private void throwIfEmptyString( } } + /** + * Throws an InvalidRequestException if the value is null or non-empty. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + private void throwIfNotEmptyCollection( + Collection value, + String error + ) throws InvalidRequestException { + if (value == null || !value.isEmpty()) { + throw new InvalidRequestException(error); + } + } + + private 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("Changelog topics must have an undefined partition count."); + } + } + } + } + /** * Throws an InvalidRequestException if the value is non-null. * @@ -1380,6 +1441,22 @@ private void throwIfNull( } } + /** + * Throws an InvalidRequestException if the value is null or empty. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + private void throwIfNullOrEmpty( + String value, + String error + ) throws InvalidRequestException { + if (value == null || value.trim().isEmpty()) { + throw new InvalidRequestException(error); + } + } + /** * Validates the request. * @@ -1452,6 +1529,49 @@ private void throwIfShareGroupHeartbeatRequestIsInvalid( } } + /** + * 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 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("MemberEpoch is invalid."); + } + + 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 @@ -1480,6 +1600,34 @@ private boolean isSubset( 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 + * 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 in the request. + * @param target The tasks that the member should have. + * + * @return A boolean indicating whether the owned partitions are a subset or not. + */ + private boolean isTaskSubset( + List ownedTasks, + Map> target + ) { + if (ownedTasks == null) return false; + + for (StreamsGroupHeartbeatRequestData.TaskIds topicPartitions : ownedTasks) { + Set partitions = target.get(topicPartitions.subtopologyId()); + if (partitions == null) return false; + for (Integer partitionId : topicPartitions.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. @@ -1633,6 +1781,43 @@ private void throwIfStaticMemberIsUnknown(ConsumerGroupMember staticMember, Stri } } + /** + * 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 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() + || !isTaskSubset(ownedActiveTasks, member.assignedTasks().activeTasks()) + || !isTaskSubset(ownedStandbyTasks, member.assignedTasks().standbyTasks()) + || !isTaskSubset(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. * @@ -1805,6 +1990,280 @@ private List fromShareGroupAssi .toList(); } + /** + * Handles a regular heartbeat from a Streams group member. + * It mainly consists of five parts: + * 1) Created or update the member. + * The group epoch is bumped if the member has been created or updated. + * 2) Initialized 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, + final StreamsGroupHeartbeatRequestData.Topology topology, + List ownedActiveTasks, + List ownedStandbyTasks, + List ownedWarmupTasks, + final String processId, + final Endpoint userEndpoint, + final List clientTags, + final List taskOffsets, + final List taskEndOffsets, + final 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 createIfNotExists = memberEpoch == 0; + final StreamsGroup group = getOrMaybeCreateStreamsGroup(groupId, createIfNotExists); + + // Get or create the member. + StreamsGroupMember member; + if (instanceId == null) { + member = getOrMaybeSubscribeDynamicStreamsGroupMember( + group, + memberId, + memberEpoch, + ownedActiveTasks, + ownedStandbyTasks, + ownedWarmupTasks, + createIfNotExists + ); + } else { + throw new UnsupportedOperationException("Static members are not supported yet."); + } + + // 1. Create or update the member. + // If the member is new or has changed, a StreamsMemberMetadataValue record is written to the __consumer_offsets partition to + // persist the change. If the subscriptions have changed, the subscription metadata is updated and persisted by writing a + // StreamsPartitionMetadataValue + // record to the __consumer_offsets partition. Finally, the group epoch is bumped if the subscriptions have + // changed, and persisted by writing a StreamsMetadataValue record to the partition. + 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 group is newly created, we must ensure that it moves away from + // epoch 0 and that it is fully initialized. + int groupEpoch = group.groupEpoch(); + boolean bumpGroupEpoch = false; + + bumpGroupEpoch |= hasStreamsMemberMetadataChanged(groupId, member, updatedMember, records); + + // 2. Initialize/Update the group topology. + // If the member 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; + boolean reconfigureTopology = false; + if (topology != null) { + StreamsGroupTopologyValue recordValue = convertToStreamsGroupTopologyRecord(topology); + + final Map subtopologyMap = recordValue.subtopologies().stream() + .collect(Collectors.toMap(StreamsGroupTopologyValue.Subtopology::subtopologyId, x -> x)); + + updatedTopology = new StreamsTopology(topology.epoch(), subtopologyMap); + + if (group.topology().isEmpty()) { + log.info("[GroupId {}][MemberId {}] Member initialized the topology with epoch {}", groupId, memberId, topology.epoch()); + + records.add(newStreamsGroupTopologyRecord(groupId, recordValue)); + + reconfigureTopology = true; + bumpGroupEpoch = true; + } 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."); + } + + if (group.topology().isPresent() && updatedMember.topologyEpoch() < group.topology().get().topologyEpoch()) { + returnedStatus.add( + new StreamsGroupHeartbeatResponseData.Status() + .setStatusCode(StreamsGroupHeartbeatResponse.Status.STALE_TOPOLOGY.code()) + .setStatusDetail( + String.format( + "The member's topology epoch %d is behind the group's topology epoch %d.", + updatedMember.topologyEpoch(), + group.topology().get().topologyEpoch() + ) + ) + ); + } + + // 3. Determine the partition metadata and any internal topics if needed. + ConfiguredTopology updatedConfiguredTopology; + Map updatedPartitionMetadata; + if (reconfigureTopology || group.hasMetadataExpired(currentTimeMs)) { + + // The partition metadata is updated when the refresh deadline has been reached. + 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 + 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 sent a full request. + // It does so when joining or rejoining the group with zero + // as the member epoch; or on any errors (e.g., timeout). + // We use all the non-optional fields to detect a full request as those must be set in a full request. + // 2. The member's assignment has been updated. + boolean isFullRequest = + rebalanceTimeoutMs != -1 + && ownedActiveTasks != null + && ownedStandbyTasks != null + && ownedWarmupTasks != null + && clientTags != null + && processId != null; + if (memberEpoch == 0 + || isFullRequest + || 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)); + } + + 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: @@ -2360,6 +2819,36 @@ private ConsumerGroupMember getOrMaybeSubscribeDynamicConsumerGroupMember( return member; } + /** + * Gets or subscribes 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 createIfNotExists Whether the member should be created or not. + * + * @return The existing streams group member or a new one. + */ + private StreamsGroupMember getOrMaybeSubscribeDynamicStreamsGroupMember( + StreamsGroup group, + String memberId, + int memberEpoch, + List ownedActiveTasks, + List ownedStandbyTasks, + List ownedWarmupTasks, + boolean createIfNotExists + ) { + StreamsGroupMember member = group.getOrMaybeCreateDefaultMember(memberId, createIfNotExists); + throwIfStreamsGroupMemberEpochIsInvalid(member, memberEpoch, ownedActiveTasks, ownedStandbyTasks, ownedWarmupTasks); + if (createIfNotExists) { + 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. @@ -2837,6 +3326,38 @@ private boolean hasMemberSubscriptionChanged( return false; } + /** + * Creates the member subscription record if the updatedMember is different from + * the old member. Returns true if the topologyEpoch of the member has changed, + * which is always true 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)); + + if (!Objects.equals(updatedMember.topologyEpoch(), member.topologyEpoch())) { + log.info("[GroupId {}] Member {} updated its topology epoch to: {}.", + groupId, memberId, updatedMember.topologyEpoch()); + return true; + } + } + return false; + } + + /** /** * Reconciles the current assignment of the member towards the target assignment if needed. * @@ -2893,7 +3414,7 @@ private ConsumerGroupMember maybeReconcile( updatedMember.rebalanceTimeoutMs() ); } else { - cancelConsumerGroupRebalanceTimeout(groupId, updatedMember.memberId()); + cancelGroupRebalanceTimeout(groupId, updatedMember.memberId()); } } } @@ -2939,6 +3460,87 @@ private ShareGroupMember maybeReconcile( 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. * @@ -3138,6 +3740,65 @@ private Assignment updateTargetAssignment( } } + /** + * 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. @@ -3178,6 +3839,35 @@ private CoordinatorResult } } + /** + * 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.getOrMaybeCreateMember(memberId, false); + 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 @@ -3233,8 +3923,7 @@ private CoordinatorResult sh 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. @@ -3483,7 +4172,7 @@ private void removeMember(List records, String groupId, Strin */ private void cancelTimers(String groupId, String memberId) { cancelGroupSessionTimeout(groupId, memberId); - cancelConsumerGroupRebalanceTimeout(groupId, memberId); + cancelGroupRebalanceTimeout(groupId, memberId); cancelConsumerGroupJoinTimeout(groupId, memberId); cancelConsumerGroupSyncTimeout(groupId, memberId); } @@ -3500,7 +4189,7 @@ private void scheduleStreamsGroupSessionTimeout( ) { timer.schedule( groupSessionTimeoutKey(groupId, memberId), - streamsGroupSessionTimeoutMs, + streamsGroupSessionTimeoutMs(groupId), TimeUnit.MILLISECONDS, true, () -> streamsGroupFenceMemberOperation(groupId, memberId, "the member session expired.") @@ -3778,7 +4467,7 @@ private void scheduleStreamsGroupRebalanceTimeout( * @param groupId The group id. * @param memberId The member id. */ - private void cancelConsumerGroupRebalanceTimeout( + private void cancelGroupRebalanceTimeout( String groupId, String memberId ) { @@ -3908,7 +4597,39 @@ public CoordinatorResult streams 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() + ); + } } /** @@ -7163,6 +7884,34 @@ private int shareGroupHeartbeatIntervalMs(String groupId) { .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 22f33eb82604b..2330497a7be39 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 CurrentAssignmentBuilder withCurrentWarmupTaskProcessIds(BiFunction> assignmentFromTaskIds( 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/TasksTuple.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java index 85c2238431153..aab08da812ae0 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 @@ * 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 @@ private static Map> merge(final Map> t /** * 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 static TasksTuple fromTargetAssignmentRecord(StreamsGroupTargetAssignment ) ); } + + 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 cf873992371e0..d56a8d0d71bf5 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 @@ /** * 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 f83bd1c267be4..eac3551a2552c 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.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.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; @@ -95,15 +100,19 @@ 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; @@ -115,7 +124,6 @@ import org.apache.kafka.server.share.persister.PartitionFactory; import org.apache.kafka.server.share.persister.PartitionIdData; import org.apache.kafka.server.share.persister.TopicData; - import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -163,6 +171,7 @@ 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; @@ -15066,6 +15075,2012 @@ public void testShareGroupStates() { 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()); + + // 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 invalid.", 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 topics must have an undefined partition count.", topoEx.getMessage()); + } + + @Test + public void testUnknownStreamsGroupId() { + String groupId = "fooup"; + String memberId = Uuid.randomUuid().toString(); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .build(); + + 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()))); + } + + + @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. + assertThrows(UnknownMemberIdException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setRebalanceTimeoutMs(1500) + .setActiveTasks(List.of()) + .setStandbyTasks(List.of()) + .setWarmupTasks(List.of()))); + } + + @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. + assertThrows(FencedMemberEpochException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(200) + .setRebalanceTimeoutMs(1500))); + + // Member epoch is smaller than the expected epoch. + assertThrows(FencedMemberEpochException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(50) + .setRebalanceTimeoutMs(1500))); + + // Member joins with previous epoch but without providing tasks. + assertThrows(FencedMemberEpochException.class, () -> + context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setRebalanceTimeoutMs(1500))); + + // 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 testStreamsUpdatingMetadataTriggersNewTargetAssignment() { + 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(asStreamsTopology(topology))) + .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.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, 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()); + } + + private StreamsTopology asStreamsTopology(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); + } + + @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(asStreamsTopology(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(context.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() + ); + + StreamsGroupMember expectedMember3 = streamsGroupMemberBuilderWithDefaults(memberId3) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS) + .setMemberEpoch(11) + .setPreviousMemberEpoch(0) + .build(); + + assertUnorderedRecordsEquals( + List.of( + List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, expectedMember3)), + List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11)), + List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId1, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), + TaskAssignmentTestUtil.mkTasks(subtopology2, 0) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId2, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 2, 3), + TaskAssignmentTestUtil.mkTasks(subtopology2, 1) + )), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId3, + TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, + TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5), + TaskAssignmentTestUtil.mkTasks(subtopology2, 2) + )) + ), + List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11)), + List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember3)) + ), + result.records() + ); + } + + @Test + public void testStreamsLeavingMemberBumpsGroupEpoch() { + 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(); + 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()) + .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 testStreamsGroupHeartbeatFullResponse() { + 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() + ); + + // A full response should be sent back when the member sends + // a full request again with topic names set. + result = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(result.response().data().memberEpoch()) + .setRebalanceTimeoutMs(1500) + .setProcessId(DEFAULT_PROCESS_ID) + .setClientTags(List.of()) + .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() + ); + } + + @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(asStreamsTopology(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)) + .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.NOT_READY, 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. + 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()))); + } + + @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(asStreamsTopology(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(asStreamsTopology(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 refr + 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(180000) + .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() { + 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(10000) // 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(10000) + .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(10000 + 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"; 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 b8ac2bfb9c2e7..3e31b7cf945d5 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.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; @@ -157,6 +162,7 @@ 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) { @@ -464,6 +470,7 @@ public static class Builder { 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); @@ -499,6 +506,11 @@ public Builder withAuthorizer(Authorizer authorizer) { 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; @@ -530,6 +542,7 @@ public GroupMetadataManagerTestContext build() { .withShareGroupAssignor(shareGroupAssignor) .withGroupConfigManager(groupConfigManager) .withAuthorizer(authorizer) + .withStreamsGroupAssignors(streamsGroupAssignors) .build(), groupConfigManager ); @@ -606,6 +619,14 @@ public ShareGroup.ShareGroupState shareGroupState( .state(); } + public StreamsGroup.StreamsGroupState streamsGroupState( + String groupId + ) { + return groupMetadataManager + .streamsGroup(groupId) + .state(); + } + public MemberState consumerGroupMemberState( String groupId, String memberId @@ -616,6 +637,16 @@ public MemberState consumerGroupMemberState( .state(); } + public org.apache.kafka.coordinator.group.streams.MemberState streamsGroupMemberState( + String groupId, + String memberId + ) { + return groupMetadataManager + .streamsGroup(groupId) + .getOrMaybeCreateMember(memberId, false) + .state(); + } + public CoordinatorResult consumerGroupHeartbeat( ConsumerGroupHeartbeatRequestData request ) { @@ -1744,4 +1775,22 @@ void onUnloaded() { 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 0000000000000..ee38d6b130045 --- /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 e35d0baa2106e..b8b52267a3fad 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 @@ 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 List build() { groupId, new StreamsGroupTopologyValue() .setEpoch(topology.topologyEpoch()) - .setSubtopologies(topology.subtopologies().values().stream().sorted().toList())) + .setSubtopologies(topology.subtopologies().values().stream().sorted(Comparator.comparing(Subtopology::subtopologyId)).toList())) ); } From fccd6e18dce2d4744eac8123157d4bce7f8473c3 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Wed, 5 Mar 2025 15:18:51 +0100 Subject: [PATCH 2/8] spotless --- .../apache/kafka/coordinator/group/GroupMetadataManagerTest.java | 1 + 1 file changed, 1 insertion(+) 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 eac3551a2552c..19729bdaf4dbb 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 @@ -124,6 +124,7 @@ import org.apache.kafka.server.share.persister.PartitionFactory; import org.apache.kafka.server.share.persister.PartitionIdData; import org.apache.kafka.server.share.persister.TopicData; + import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; From acb2b5f57dca68ef1de7853c12402fe03deec5f5 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 7 Mar 2025 14:41:11 +0100 Subject: [PATCH 3/8] fix tests --- .../coordinator/group/GroupMetadataManagerTest.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 19729bdaf4dbb..e01d520aa5471 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 @@ -3819,13 +3819,15 @@ private StreamsGroupMember.Builder streamsGroupMemberBuilderWithDefaults(String .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 From 787d8d4c113a38c387e77aec66f012434ba0d110 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Fri, 7 Mar 2025 15:19:12 +0100 Subject: [PATCH 4/8] refactoring to reuse StreamsTopology.fromRequest --- .../group/GroupMetadataManager.java | 5 +- .../group/streams/StreamsTopology.java | 14 +++++ .../group/GroupMetadataManagerTest.java | 23 ++----- .../group/streams/StreamsTopologyTest.java | 63 ++++++++++++++++++- 4 files changed, 83 insertions(+), 22 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index dee448aa4108d..2202bc7781823 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 @@ -2098,10 +2098,7 @@ private CoordinatorResult stream if (topology != null) { StreamsGroupTopologyValue recordValue = convertToStreamsGroupTopologyRecord(topology); - final Map subtopologyMap = recordValue.subtopologies().stream() - .collect(Collectors.toMap(StreamsGroupTopologyValue.Subtopology::subtopologyId, x -> x)); - - updatedTopology = new StreamsTopology(topology.epoch(), subtopologyMap); + updatedTopology = StreamsTopology.fromRequest(topology); if (group.topology().isEmpty()) { log.info("[GroupId {}][MemberId {}] Member initialized the topology with epoch {}", groupId, memberId, topology.epoch()); 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 49ce9f9b4fd99..e48b10bade9ff 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 static StreamsTopology fromRecord(StreamsGroupTopologyValue record) { 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 fromRequest(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/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index e01d520aa5471..ceab7168d3f50 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 @@ -15507,7 +15507,7 @@ public void testStreamsUpdatingMetadataTriggersNewTargetAssignment() { .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) .withTargetAssignmentEpoch(10) - .withTopology(asStreamsTopology(topology))) + .withTopology(StreamsTopology.fromRequest(topology))) .build(); assignor.prepareGroupAssignment( @@ -15572,13 +15572,6 @@ public void testStreamsUpdatingMetadataTriggersNewTargetAssignment() { assertRecordsEquals(expectedRecords, result.records()); } - private StreamsTopology asStreamsTopology(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); - } - @Test public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { String groupId = "fooup"; @@ -15627,7 +15620,7 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) .withTargetAssignmentEpoch(10) - .withTopology(asStreamsTopology(topology)) + .withTopology(StreamsTopology.fromRequest(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) @@ -15658,7 +15651,7 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { .setMemberEpoch(0) .setRebalanceTimeoutMs(1500) .setTopology(topology) - .setProcessId(context.DEFAULT_PROCESS_ID) + .setProcessId(DEFAULT_PROCESS_ID) .setActiveTasks(List.of()) .setStandbyTasks(List.of()) .setWarmupTasks(List.of())); @@ -15720,10 +15713,6 @@ public void testStreamsLeavingMemberBumpsGroupEpoch() { 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() @@ -15920,7 +15909,7 @@ public void testStreamsReconciliationProcess() { TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) .build()) - .withTopology(asStreamsTopology(topology)) + .withTopology(StreamsTopology.fromRequest(topology)) .withTargetAssignment(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) @@ -16457,7 +16446,7 @@ public void testStreamsPartitionMetadataRefreshedAfterGroupIsLoaded() { .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) - .withTopology(asStreamsTopology(topology)) + .withTopology(StreamsTopology.fromRequest(topology)) .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .withTargetAssignmentEpoch(10) @@ -16554,7 +16543,7 @@ public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() { .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) - .withTopology(asStreamsTopology(topology)) + .withTopology(StreamsTopology.fromRequest(topology)) .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .withTargetAssignmentEpoch(10) 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 89c785d633e3c..768a190393285 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,10 +16,10 @@ */ 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; - import org.junit.jupiter.api.Test; import java.util.Arrays; @@ -103,6 +103,22 @@ public void fromRecordShouldCreateCorrectTopology() { assertEquals(mkSubtopology2(), topology.subtopologies().get(SUBTOPOLOGY_ID_2)); } + @Test + public void fromRequestShouldCreateCorrectTopology() { + StreamsGroupHeartbeatRequestData.Topology requestTopology = new StreamsGroupHeartbeatRequestData.Topology() + .setEpoch(1) + .setSubtopologies(List.of(mkRequestSubtopology1(), mkRequestSubtopology2())); + + StreamsTopology topology = StreamsTopology.fromRequest(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 +163,49 @@ private Subtopology mkSubtopology2() { 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) + )); + } } From c2914050977966240996fa341575cdfa76e9f13d Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 10 Mar 2025 14:47:32 +0100 Subject: [PATCH 5/8] comments --- .../group/GroupCoordinatorShard.java | 2 +- .../group/GroupMetadataManager.java | 308 +++++++++--------- .../group/streams/StreamsGroup.java | 54 +-- .../group/streams/StreamsTopology.java | 2 +- .../group/GroupMetadataManagerTest.java | 52 +-- .../GroupMetadataManagerTestContext.java | 2 +- .../group/streams/StreamsGroupTest.java | 52 ++- .../group/streams/StreamsTopologyTest.java | 5 +- 8 files changed, 250 insertions(+), 227 deletions(-) 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 88a65ca0ee402..1e5f08227af5f 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 @@ -396,7 +396,7 @@ public CoordinatorResult * @param context The request context. * @param request The actual StreamsGroupHeartbeat request. * - * @return A Result containing the StreamsGroupHeartbeat response, a list of internal topics to be created and + * @return A result containing the StreamsGroupHeartbeat response, a list of internal topics to be created and * a list of records to update the state machine. */ public CoordinatorResult 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 2202bc7781823..a30f77cd51c1e 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 @@ -62,7 +62,9 @@ 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; @@ -812,40 +814,58 @@ ConsumerGroup getOrMaybeCreateConsumerGroup( } /** - * 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 + StreamsGroup getOrCreateStreamsGroup( + String groupId + ) { + Group group = groups.get(groupId); + + if (group == null) { + return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); + } else { + 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 && !createIfNotExists) { + if (group == null) { throw new GroupIdNotFoundException(String.format("Streams group %s not found.", groupId)); + } else { + return castToStreamsGroup(group); } + } - if (group == null) { - return new StreamsGroup(logContext, snapshotRegistry, groupId, metrics); + private StreamsGroup castToStreamsGroup(final Group group) { + if (group.type() == STREAMS) { + return (StreamsGroup) group; } else { - if (group.type() == STREAMS) { - return (StreamsGroup) group; - } else { - throw new GroupIdNotFoundException(String.format("Group %s is not a streams group.", groupId)); - } + throw new GroupIdNotFoundException(String.format("Group %s is not a streams group.", group.groupId())); } } - + /** * Gets a streams group by committed offset. * @@ -1372,7 +1392,7 @@ private void removeGroup( * @param error The error message. * @throws InvalidRequestException */ - private void throwIfEmptyString( + private static void throwIfEmptyString( String value, String error ) throws InvalidRequestException { @@ -1388,7 +1408,7 @@ private void throwIfEmptyString( * @param error The error message. * @throws InvalidRequestException */ - private void throwIfNotEmptyCollection( + private static void throwIfNotEmptyCollection( Collection value, String error ) throws InvalidRequestException { @@ -1397,7 +1417,7 @@ private void throwIfNotEmptyCollection( } } - private void throwIfInvalidTopology( + private static void throwIfInvalidTopology( StreamsGroupHeartbeatRequestData.Topology topology ) throws StreamsInvalidTopologyException { for (StreamsGroupHeartbeatRequestData.Subtopology subtopology: topology.subtopologies()) { @@ -1416,7 +1436,7 @@ private void throwIfInvalidTopology( * @param error The error message. * @throws InvalidRequestException */ - private void throwIfNotNull( + private static void throwIfNotNull( Object value, String error ) throws InvalidRequestException { @@ -1432,7 +1452,7 @@ private void throwIfNotNull( * @param error The error message. * @throws InvalidRequestException */ - private void throwIfNull( + private static void throwIfNull( Object value, String error ) throws InvalidRequestException { @@ -1441,22 +1461,6 @@ private void throwIfNull( } } - /** - * Throws an InvalidRequestException if the value is null or empty. - * - * @param value The value. - * @param error The error message. - * @throws InvalidRequestException - */ - private void throwIfNullOrEmpty( - String value, - String error - ) throws InvalidRequestException { - if (value == null || value.trim().isEmpty()) { - throw new InvalidRequestException(error); - } - } - /** * Validates the request. * @@ -1536,7 +1540,7 @@ private void throwIfShareGroupHeartbeatRequestIsInvalid( * @throws InvalidRequestException if the request is not valid. * @throws UnsupportedAssignorException if the assignor is not supported. */ - private void throwIfStreamsGroupHeartbeatRequestIsInvalid( + private static void throwIfStreamsGroupHeartbeatRequestIsInvalid( StreamsGroupHeartbeatRequestData request ) throws InvalidRequestException { throwIfEmptyString(request.memberId(), "MemberId can't be empty."); @@ -1583,7 +1587,7 @@ private void throwIfStreamsGroupHeartbeatRequestIsInvalid( * * @return A boolean indicating whether the owned partitions are a subset or not. */ - private boolean isSubset( + private static boolean isSubset( List ownedTopicPartitions, Map> target ) { @@ -1603,24 +1607,24 @@ private boolean isSubset( /** * 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 - * only owns tasks which are in the assigned tasks. It does not match if + * 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 in the request. - * @param target The tasks that the member should have. + * @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 boolean isTaskSubset( + private static boolean areOwnedTasksContainedInAssignedTasks( List ownedTasks, - Map> target + Map> assignedTasks ) { if (ownedTasks == null) return false; - for (StreamsGroupHeartbeatRequestData.TaskIds topicPartitions : ownedTasks) { - Set partitions = target.get(topicPartitions.subtopologyId()); + for (StreamsGroupHeartbeatRequestData.TaskIds ownedTasksOfSubtopology : ownedTasks) { + Set partitions = assignedTasks.get(ownedTasksOfSubtopology.subtopologyId()); if (partitions == null) return false; - for (Integer partitionId : topicPartitions.partitions()) { + for (Integer partitionId : ownedTasksOfSubtopology.partitions()) { if (!partitions.contains(partitionId)) return false; } } @@ -1793,7 +1797,7 @@ private void throwIfStaticMemberIsUnknown(ConsumerGroupMember staticMember, Stri * @throws FencedMemberEpochException if the provided epoch is ahead or behind the epoch known * by this coordinator. */ - private void throwIfStreamsGroupMemberEpochIsInvalid( + private static void throwIfStreamsGroupMemberEpochIsInvalid( StreamsGroupMember member, int receivedMemberEpoch, List ownedActiveTasks, @@ -1808,9 +1812,9 @@ private void throwIfStreamsGroupMemberEpochIsInvalid( // 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() - || !isTaskSubset(ownedActiveTasks, member.assignedTasks().activeTasks()) - || !isTaskSubset(ownedStandbyTasks, member.assignedTasks().standbyTasks()) - || !isTaskSubset(ownedWarmupTasks, member.assignedTasks().warmupTasks())) { + || !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."); @@ -1993,16 +1997,16 @@ private List fromShareGroupAssi /** * Handles a regular heartbeat from a Streams group member. * It mainly consists of five parts: - * 1) Created or update the member. + * 1) Create or update the member. * The group epoch is bumped if the member has been created or updated. - * 2) Initialized or update the topology. The group epoch is bumped if the topology - * 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 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. @@ -2030,47 +2034,42 @@ private CoordinatorResult stream int rebalanceTimeoutMs, String clientId, String clientHost, - final StreamsGroupHeartbeatRequestData.Topology topology, + StreamsGroupHeartbeatRequestData.Topology topology, List ownedActiveTasks, List ownedStandbyTasks, List ownedWarmupTasks, - final String processId, - final Endpoint userEndpoint, - final List clientTags, - final List taskOffsets, - final List taskEndOffsets, - final boolean shutdownApplication + 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 createIfNotExists = memberEpoch == 0; - final StreamsGroup group = getOrMaybeCreateStreamsGroup(groupId, createIfNotExists); + boolean isJoining = memberEpoch == 0; + final StreamsGroup group = isJoining ? getOrCreateStreamsGroup(groupId) : getStreamsGroupOrThrow(groupId); // Get or create the member. StreamsGroupMember member; if (instanceId == null) { - member = getOrMaybeSubscribeDynamicStreamsGroupMember( + member = getOrMaybeCreateDynamicStreamsGroupMember( group, memberId, memberEpoch, ownedActiveTasks, ownedStandbyTasks, ownedWarmupTasks, - createIfNotExists + isJoining ); } else { throw new UnsupportedOperationException("Static members are not supported yet."); } // 1. Create or update the member. - // If the member is new or has changed, a StreamsMemberMetadataValue record is written to the __consumer_offsets partition to - // persist the change. If the subscriptions have changed, the subscription metadata is updated and persisted by writing a - // StreamsPartitionMetadataValue - // record to the __consumer_offsets partition. Finally, the group epoch is bumped if the subscriptions have - // changed, and persisted by writing a StreamsMetadataValue record to the partition. StreamsGroupMember updatedMember = new StreamsGroupMember.Builder(member) .maybeUpdateInstanceId(Optional.empty()) .maybeUpdateRackId(Optional.ofNullable(rackId)) @@ -2083,59 +2082,22 @@ private CoordinatorResult stream .maybeUpdateUserEndpoint(Optional.ofNullable(userEndpoint).map(x -> new StreamsGroupMemberMetadataValue.Endpoint().setHost(x.host()).setPort(x.port()))) .build(); - // If the group is newly created, we must ensure that it moves away from - // epoch 0 and that it is fully initialized. - int groupEpoch = group.groupEpoch(); - boolean bumpGroupEpoch = false; - - bumpGroupEpoch |= hasStreamsMemberMetadataChanged(groupId, member, updatedMember, records); + // 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 member is new or has changed, a StreamsGroupTopologyValue record is written to the __consumer_offsets partition to persist + // 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; - boolean reconfigureTopology = false; - if (topology != null) { - StreamsGroupTopologyValue recordValue = convertToStreamsGroupTopologyRecord(topology); - - updatedTopology = StreamsTopology.fromRequest(topology); - - if (group.topology().isEmpty()) { - log.info("[GroupId {}][MemberId {}] Member initialized the topology with epoch {}", groupId, memberId, topology.epoch()); - - records.add(newStreamsGroupTopologyRecord(groupId, recordValue)); - - reconfigureTopology = true; - bumpGroupEpoch = true; - } 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."); - } - - if (group.topology().isPresent() && updatedMember.topologyEpoch() < group.topology().get().topologyEpoch()) { - returnedStatus.add( - new StreamsGroupHeartbeatResponseData.Status() - .setStatusCode(StreamsGroupHeartbeatResponse.Status.STALE_TOPOLOGY.code()) - .setStatusDetail( - String.format( - "The member's topology epoch %d is behind the group's topology epoch %d.", - updatedMember.topologyEpoch(), - group.topology().get().topologyEpoch() - ) - ) - ); - } + 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)) { - // The partition metadata is updated when the refresh deadline has been reached. updatedPartitionMetadata = group.computePartitionMetadata( metadataImage.topics(), updatedTopology @@ -2162,6 +2124,7 @@ private CoordinatorResult stream } // Actually bump the group epoch + int groupEpoch = group.groupEpoch(); if (bumpGroupEpoch) { groupEpoch += 1; records.add(newStreamsGroupEpochRecord(groupId, groupEpoch)); @@ -2215,22 +2178,9 @@ private CoordinatorResult stream .setHeartbeatIntervalMs(streamsGroupHeartbeatIntervalMs(groupId)); // The assignment is only provided in the following cases: - // 1. The member sent a full request. - // It does so when joining or rejoining the group with zero - // as the member epoch; or on any errors (e.g., timeout). - // We use all the non-optional fields to detect a full request as those must be set in a full request. + // 1. The member is joining. // 2. The member's assignment has been updated. - boolean isFullRequest = - rebalanceTimeoutMs != -1 - && ownedActiveTasks != null - && ownedStandbyTasks != null - && ownedWarmupTasks != null - && clientTags != null - && processId != null; - if (memberEpoch == 0 - || isFullRequest - || hasAssignedTasksChanged(member, updatedMember) - ) { + if (memberEpoch == 0 || hasAssignedTasksChanged(member, updatedMember)) { response.setActiveTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().activeTasks())); response.setStandbyTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().standbyTasks())); response.setWarmupTasks(createStreamsGroupHeartbeatResponseTaskIds(updatedMember.assignedTasks().warmupTasks())); @@ -2253,6 +2203,70 @@ private CoordinatorResult stream 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() @@ -2817,7 +2831,7 @@ private ConsumerGroupMember getOrMaybeSubscribeDynamicConsumerGroupMember( } /** - * Gets or subscribes a new dynamic streams group member. + * Gets or creates a new dynamic streams group member. * * @param group The streams group. * @param memberId The member id. @@ -2825,22 +2839,22 @@ private ConsumerGroupMember getOrMaybeSubscribeDynamicConsumerGroupMember( * @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 createIfNotExists Whether the member should be created or not. + * @param memberIsJoining Whether the member should be created or not. * * @return The existing streams group member or a new one. */ - private StreamsGroupMember getOrMaybeSubscribeDynamicStreamsGroupMember( + private StreamsGroupMember getOrMaybeCreateDynamicStreamsGroupMember( StreamsGroup group, String memberId, int memberEpoch, List ownedActiveTasks, List ownedStandbyTasks, List ownedWarmupTasks, - boolean createIfNotExists + boolean memberIsJoining ) { - StreamsGroupMember member = group.getOrMaybeCreateDefaultMember(memberId, createIfNotExists); + StreamsGroupMember member = memberIsJoining ? group.getOrCreateDefaultMember(memberId) : group.getMemberOrThrow(memberId); throwIfStreamsGroupMemberEpochIsInvalid(member, memberEpoch, ownedActiveTasks, ownedStandbyTasks, ownedWarmupTasks); - if (createIfNotExists) { + if (memberIsJoining) { log.info("[GroupId {}][MemberId {}] Member joins the streams group.", group.groupId(), memberId); } return member; @@ -3842,7 +3856,7 @@ private CoordinatorResult * @param memberId The member id from the request. * @param memberEpoch The member epoch from the request. * - * @return A Result containing the StreamsGroupHeartbeat response and + * @return A result containing the StreamsGroupHeartbeat response and * a list of records to update the state machine. */ private CoordinatorResult streamsGroupLeave( @@ -3857,7 +3871,7 @@ private CoordinatorResult stream .setMemberEpoch(memberEpoch); if (instanceId == null) { - StreamsGroupMember member = group.getOrMaybeCreateMember(memberId, false); + 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 { @@ -4302,7 +4316,7 @@ private CoordinatorResult streamsGroupFenceMemberOpera ) { 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); @@ -4433,7 +4447,7 @@ private void scheduleStreamsGroupRebalanceTimeout( 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 " + @@ -4587,7 +4601,7 @@ public CoordinatorResult * @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( @@ -5222,14 +5236,14 @@ public void replay( } 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; @@ -5327,7 +5341,7 @@ public void replay( 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(); @@ -5343,7 +5357,7 @@ public void replay( 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; 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 14dfac1932722..dacb041d93b0b 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,54 +330,60 @@ public String staticMemberId(String groupInstanceId) { } /** - * 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 but without adding it to the group. Adding a member is done via the + * 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. - * @param createIfNotExists Booleans indicating whether the member must be created if it does not exist. * @return A StreamsGroupMember. */ - public StreamsGroupMember getOrMaybeCreateDefaultMember( - String memberId, - boolean createIfNotExists + public StreamsGroupMember getOrCreateDefaultMember( + 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) - ); - } - return StreamsGroupMember.Builder.withDefaults(memberId).build(); } @@ -389,7 +395,7 @@ public StreamsGroupMember getOrMaybeCreateDefaultMember( */ public StreamsGroupMember staticMember(String instanceId) { String existingMemberId = staticMemberId(instanceId); - return existingMemberId == null ? null : getOrMaybeCreateMember(existingMemberId, false); + return existingMemberId == null ? null : getMemberOrThrow(existingMemberId); } /** @@ -682,7 +688,7 @@ public void validateOffsetCommit( 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/StreamsTopology.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsTopology.java index e48b10bade9ff..498ff00a595cb 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 @@ -89,7 +89,7 @@ public static StreamsTopology fromRecord(StreamsGroupTopologyValue record) { * @param topology The topology supplied in the request. * @return The instance of StreamsTopology created from the request. */ - public static StreamsTopology fromRequest(StreamsGroupHeartbeatRequestData.Topology topology) { + 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)); 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 ceab7168d3f50..f872e6006ed77 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 @@ -3798,9 +3798,9 @@ public void testUpdateStreamsGroupSizeCounter() { 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(); @@ -15507,7 +15507,7 @@ public void testStreamsUpdatingMetadataTriggersNewTargetAssignment() { .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) .withTargetAssignmentEpoch(10) - .withTopology(StreamsTopology.fromRequest(topology))) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology))) .build(); assignor.prepareGroupAssignment( @@ -15620,7 +15620,7 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) .withTargetAssignmentEpoch(10) - .withTopology(StreamsTopology.fromRequest(topology)) + .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) @@ -15839,34 +15839,6 @@ public void testStreamsGroupHeartbeatFullResponse() { .setHeartbeatIntervalMs(5000), result.response().data() ); - - // A full response should be sent back when the member sends - // a full request again with topic names set. - result = context.streamsGroupHeartbeat( - new StreamsGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId) - .setMemberEpoch(result.response().data().memberEpoch()) - .setRebalanceTimeoutMs(1500) - .setProcessId(DEFAULT_PROCESS_ID) - .setClientTags(List.of()) - .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() - ); } @Test @@ -15909,7 +15881,7 @@ public void testStreamsReconciliationProcess() { TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) .build()) - .withTopology(StreamsTopology.fromRequest(topology)) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) .withTargetAssignment(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2), TaskAssignmentTestUtil.mkTasks(subtopology2, 0, 1))) @@ -16446,7 +16418,7 @@ public void testStreamsPartitionMetadataRefreshedAfterGroupIsLoaded() { .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) - .withTopology(StreamsTopology.fromRequest(topology)) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .withTargetAssignmentEpoch(10) @@ -16543,7 +16515,7 @@ public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() { .setAssignedTasks(TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .build()) - .withTopology(StreamsTopology.fromRequest(topology)) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2))) .withTargetAssignmentEpoch(10) @@ -17427,7 +17399,7 @@ public void testReplayStreamsGroupMemberMetadata() { // 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 @@ -17439,7 +17411,7 @@ public void testReplayStreamsGroupMemberMetadataTombstoneNotExisting() { // 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. @@ -17714,7 +17686,7 @@ public void testReplayStreamsGroupCurrentMemberAssignment() { // 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 @@ -17726,7 +17698,7 @@ public void testReplayStreamsGroupCurrentMemberAssignmentTombstoneNotExisting() // 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. @@ -17758,7 +17730,7 @@ public void testReplayStreamsGroupCurrentMemberAssignmentTombstoneExisting() { 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 3e31b7cf945d5..038f261257cdf 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 @@ -643,7 +643,7 @@ public org.apache.kafka.coordinator.group.streams.MemberState streamsGroupMember ) { return groupMetadataManager .streamsGroup(groupId) - .getOrMaybeCreateMember(memberId, false) + .getMemberOrThrow(memberId) .state(); } 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 a2d30c2db449f..fe8161df61533 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.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 @@ private StreamsGroup createStreamsGroup(String groupId) { } @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 void testUpdateMember() { 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 void testNoStaticMember() { 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 void testGetStaticMemberByInstanceId() { 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 void testGetStaticMemberByInstanceId() { 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 void testGetStaticMemberByInstanceId() { 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 768a190393285..a9d269f2120fb 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 @@ -20,6 +20,7 @@ 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; + import org.junit.jupiter.api.Test; import java.util.Arrays; @@ -104,12 +105,12 @@ public void fromRecordShouldCreateCorrectTopology() { } @Test - public void fromRequestShouldCreateCorrectTopology() { + public void fromHeartbeatRequestShouldCreateCorrectTopology() { StreamsGroupHeartbeatRequestData.Topology requestTopology = new StreamsGroupHeartbeatRequestData.Topology() .setEpoch(1) .setSubtopologies(List.of(mkRequestSubtopology1(), mkRequestSubtopology2())); - StreamsTopology topology = StreamsTopology.fromRequest(requestTopology); + StreamsTopology topology = StreamsTopology.fromHeartbeatRequest(requestTopology); assertEquals(1, topology.topologyEpoch()); assertEquals(2, topology.subtopologies().size()); From 537d4fe4e98dfc93c4a44f857068bf7001eb063d Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Wed, 12 Mar 2025 21:44:28 +0100 Subject: [PATCH 6/8] comments addressed --- .../group/GroupMetadataManager.java | 24 +-- .../group/GroupMetadataManagerTest.java | 172 +++++++++++++----- 2 files changed, 140 insertions(+), 56 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index a30f77cd51c1e..2124b15346370 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 @@ -1423,7 +1423,10 @@ private static void throwIfInvalidTopology( for (StreamsGroupHeartbeatRequestData.Subtopology subtopology: topology.subtopologies()) { for (StreamsGroupHeartbeatRequestData.TopicInfo topicInfo: subtopology.stateChangelogTopics()) { if (topicInfo.partitions() != 0) { - throw new StreamsInvalidTopologyException("Changelog topics must have an undefined partition count."); + throw new StreamsInvalidTopologyException(String.format( + "Changelog topic %s must have an undefined partition count, but it is set to %d.", + topicInfo.name(), topicInfo.partitions() + )); } } } @@ -1562,7 +1565,8 @@ private static void throwIfStreamsGroupHeartbeatRequestIsInvalid( } 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("MemberEpoch is invalid."); + 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) { @@ -2023,7 +2027,7 @@ private List fromShareGroupAssi * @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. + * @return A result containing the StreamsGroupHeartbeat response and a list of records to update the state machine. */ private CoordinatorResult streamsGroupHeartbeat( String groupId, @@ -3338,9 +3342,9 @@ private boolean hasMemberSubscriptionChanged( } /** - * Creates the member subscription record if the updatedMember is different from - * the old member. Returns true if the topologyEpoch of the member has changed, - * which is always true when a member is first created. + * Creates the member metadatarecord record if the updatedMember is different from + * the old member. Returns true if the metadata has changed, which is always true + * when a member is first created. * * @param groupId The group id. * @param member The old member. @@ -3358,12 +3362,10 @@ private boolean hasStreamsMemberMetadataChanged( 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); - if (!Objects.equals(updatedMember.topologyEpoch(), member.topologyEpoch())) { - log.info("[GroupId {}] Member {} updated its topology epoch to: {}.", - groupId, memberId, updatedMember.topologyEpoch()); - return true; - } + return true; } return false; } 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 f872e6006ed77..00f915c38e1d9 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 @@ -15090,6 +15090,12 @@ public void testStreamsHeartbeatRequestValidation() { 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() @@ -15191,7 +15197,7 @@ public void testStreamsHeartbeatRequestValidation() { .setMemberEpoch(-3) .setRebalanceTimeoutMs(1500) )); - assertEquals("MemberEpoch is invalid.", ex.getMessage()); + 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( @@ -15230,7 +15236,8 @@ public void testStreamsHeartbeatRequestValidation() { ) )) )); - assertEquals("Changelog topics must have an undefined partition count.", topoEx.getMessage()); + assertEquals("Changelog topic changelog_topic_with_fixed_partition must have an undefined partition count, but it is set to 3.", + topoEx.getMessage()); } @Test @@ -15241,7 +15248,7 @@ public void testUnknownStreamsGroupId() { GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() .build(); - assertThrows(GroupIdNotFoundException.class, () -> + GroupIdNotFoundException e = assertThrows(GroupIdNotFoundException.class, () -> context.streamsGroupHeartbeat( new StreamsGroupHeartbeatRequestData() .setGroupId(groupId) @@ -15251,9 +15258,9 @@ public void testUnknownStreamsGroupId() { .setActiveTasks(List.of()) .setStandbyTasks(List.of()) .setWarmupTasks(List.of()))); + assertEquals("Streams group fooup not found.", e.getMessage()); } - @Test public void testUnknownMemberIdJoinsStreamsGroup() { String groupId = "fooup"; @@ -15281,16 +15288,18 @@ public void testUnknownMemberIdJoinsStreamsGroup() { // The second member is rejected because the member id is unknown and // the member epoch is not zero. - assertThrows(UnknownMemberIdException.class, () -> + final String memberId2 = Uuid.randomUuid().toString(); + UnknownMemberIdException e = assertThrows(UnknownMemberIdException.class, () -> context.streamsGroupHeartbeat( new StreamsGroupHeartbeatRequestData() .setGroupId(groupId) - .setMemberId(Uuid.randomUuid().toString()) + .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 @@ -15335,31 +15344,37 @@ public void testStreamsGroupMemberEpochValidation() { context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, member)); // Member epoch is greater than the expected epoch. - assertThrows(FencedMemberEpochException.class, () -> + 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. - assertThrows(FencedMemberEpochException.class, () -> + 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. - assertThrows(FencedMemberEpochException.class, () -> + 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. @@ -15474,7 +15489,7 @@ public void testMemberJoinsEmptyStreamsGroup() { } @Test - public void testStreamsUpdatingMetadataTriggersNewTargetAssignment() { + public void testStreamsUpdatingMemberMetadataTriggersNewTargetAssignment() { String groupId = "fooup"; String memberId = Uuid.randomUuid().toString(); @@ -15507,7 +15522,12 @@ public void testStreamsUpdatingMetadataTriggersNewTargetAssignment() { .withTargetAssignment(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5))) .withTargetAssignmentEpoch(10) - .withTopology(StreamsTopology.fromHeartbeatRequest(topology))) + .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( @@ -15555,6 +15575,100 @@ public void testStreamsUpdatingMetadataTriggersNewTargetAssignment() { 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 testStreamsUpdatingPartitonMetadataTriggersNewTargetAssignment() { + 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)) + ) + .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, 3) @@ -15667,38 +15781,6 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { result.response().data() ); - StreamsGroupMember expectedMember3 = streamsGroupMemberBuilderWithDefaults(memberId3) - .setState(org.apache.kafka.coordinator.group.streams.MemberState.UNRELEASED_TASKS) - .setMemberEpoch(11) - .setPreviousMemberEpoch(0) - .build(); - - assertUnorderedRecordsEquals( - List.of( - List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, expectedMember3)), - List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11)), - List.of( - StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId1, - TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1), - TaskAssignmentTestUtil.mkTasks(subtopology2, 0) - )), - StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId2, - TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 2, 3), - TaskAssignmentTestUtil.mkTasks(subtopology2, 1) - )), - StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId3, - TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, - TaskAssignmentTestUtil.mkTasks(subtopology1, 4, 5), - TaskAssignmentTestUtil.mkTasks(subtopology2, 2) - )) - ), - List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11)), - List.of(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember3)) - ), - result.records() - ); } @Test @@ -15774,7 +15856,7 @@ public void testStreamsLeavingMemberBumpsGroupEpoch() { } @Test - public void testStreamsGroupHeartbeatFullResponse() { + public void testStreamsGroupHeartbeatPartialResponseWhenNothingChanges() { String groupId = "fooup"; String memberId = Uuid.randomUuid().toString(); @@ -16781,7 +16863,7 @@ public void testStreamsRebalanceTimeoutLifecycle() { .setGroupId(groupId) .setMemberId(memberId1) .setMemberEpoch(0) - .setRebalanceTimeoutMs(180000) + .setRebalanceTimeoutMs(12000) .setTopology(topology) .setActiveTasks(List.of()) .setStandbyTasks(List.of()) From da10d3f31e3dc1479d90ea987dd075e2a26d4b6a Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 17 Mar 2025 09:49:45 +0100 Subject: [PATCH 7/8] comments --- .../apache/kafka/coordinator/group/GroupMetadataManager.java | 4 ++-- .../kafka/coordinator/group/GroupMetadataManagerTest.java | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index efbccb86b32be..16f3c4e93e64d 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 @@ -3342,8 +3342,8 @@ private boolean hasMemberSubscriptionChanged( } /** - * Creates the member metadatarecord record if the updatedMember is different from - * the old member. Returns true if the metadata has changed, which is always true + * 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. 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 33415c8524710..99f9a40ec071d 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 @@ -15816,10 +15816,9 @@ public void testStreamsUpdatingMemberMetadataTriggersNewTargetAssignment() { } @Test - public void testStreamsUpdatingPartitonMetadataTriggersNewTargetAssignment() { + public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() { String groupId = "fooup"; String memberId = Uuid.randomUuid().toString(); - String subtopology1 = "subtopology1"; String fooTopicName = "foo"; Uuid fooTopicId = Uuid.randomUuid(); From 88e494aa3f8cb9ad2288aaefe9cdc113040b72ad Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Mon, 31 Mar 2025 10:58:00 +0200 Subject: [PATCH 8/8] comments --- .../group/GroupMetadataManagerTest.java | 32 +++++++++++++------ 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index 99f9a40ec071d..5d3ffceab03d8 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 @@ -15817,6 +15817,7 @@ public void testStreamsUpdatingMemberMetadataTriggersNewTargetAssignment() { @Test public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() { + int changedPartitionCount = 6; // New partition count for the topic. String groupId = "fooup"; String memberId = Uuid.randomUuid().toString(); String subtopology1 = "subtopology1"; @@ -15835,7 +15836,7 @@ public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() { .withStreamsGroupTaskAssignors(List.of(assignor)) .withMetadataImage(new MetadataImageBuilder() .addTopic(fooTopicId, fooTopicName, 6) - .addTopic(barTopicId, barTopicName, 3) + .addTopic(barTopicId, barTopicName, changedPartitionCount) .build()) .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) .withMember(streamsGroupMemberBuilderWithDefaults(memberId) @@ -15849,6 +15850,10 @@ public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() { 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(); @@ -15897,7 +15902,7 @@ public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() { 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, 3) + barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, changedPartitionCount) )), StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, @@ -16010,7 +16015,7 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() { } @Test - public void testStreamsLeavingMemberBumpsGroupEpoch() { + public void testStreamsLeavingMemberRemovesMemberAndBumpsGroupEpoch() { String groupId = "fooup"; String memberId1 = Uuid.randomUuid().toString(); String memberId2 = Uuid.randomUuid().toString(); @@ -16196,7 +16201,12 @@ public void testStreamsReconciliationProcess() { .withTargetAssignment(memberId2, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 3, 4, 5), TaskAssignmentTestUtil.mkTasks(subtopology2, 2))) - .withTargetAssignmentEpoch(10)) + .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. @@ -16220,7 +16230,7 @@ public void testStreamsReconciliationProcess() { // 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.NOT_READY, context.streamsGroupState(groupId)); + 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 @@ -16688,7 +16698,7 @@ public void testStreamsTaskAssignorExceptionOnRegularHeartbeat() { // Member 1 joins the streams group. The request fails because the // target assignment computation failed. - assertThrows(UnknownServerException.class, () -> + UnknownServerException e = assertThrows(UnknownServerException.class, () -> context.streamsGroupHeartbeat( new StreamsGroupHeartbeatRequestData() .setGroupId(groupId) @@ -16699,6 +16709,7 @@ public void testStreamsTaskAssignorExceptionOnRegularHeartbeat() { .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 @@ -16863,7 +16874,7 @@ public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() { context.rollback(); // However, the next heartbeat should detect the divergence based on the epoch and trigger - // a metadata refr + // a metadata refresh. CoordinatorResult result = context.streamsGroupHeartbeat( new StreamsGroupHeartbeatRequestData() .setGroupId(groupId) @@ -17215,6 +17226,7 @@ public void testStreamsRebalanceTimeoutLifecycle() { @Test public void testStreamsRebalanceTimeoutExpiration() { + final int rebalanceTimeoutMs = 10000; String groupId = "fooup"; String memberId1 = Uuid.randomUuid().toString(); String memberId2 = Uuid.randomUuid().toString(); @@ -17244,7 +17256,7 @@ public void testStreamsRebalanceTimeoutExpiration() { .setGroupId(groupId) .setMemberId(memberId1) .setMemberEpoch(0) - .setRebalanceTimeoutMs(10000) // Use timeout smaller than session timeout. + .setRebalanceTimeoutMs(rebalanceTimeoutMs) // Use timeout smaller than session timeout. .setTopology(topology) .setActiveTasks(List.of()) .setStandbyTasks(List.of()) @@ -17285,7 +17297,7 @@ public void testStreamsRebalanceTimeoutExpiration() { .setGroupId(groupId) .setMemberId(memberId2) .setMemberEpoch(0) - .setRebalanceTimeoutMs(10000) + .setRebalanceTimeoutMs(rebalanceTimeoutMs) .setTopology(topology) .setActiveTasks(List.of()) .setStandbyTasks(List.of()) @@ -17330,7 +17342,7 @@ public void testStreamsRebalanceTimeoutExpiration() { ); // Advance time past the revocation timeout. - List> timeouts = context.sleep(10000 + 1); + List> timeouts = context.sleep(rebalanceTimeoutMs + 1); // Verify the expired timeout. assertEquals(