From 205fd25e58b00a13de666dab03e087f6a8d8f6b8 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Tue, 11 Apr 2023 10:18:57 +0200 Subject: [PATCH 01/16] Add GroupMetadataManager and ConsumerGroup --- checkstyle/import-control.xml | 3 + checkstyle/suppressions.xml | 6 +- .../group/GroupMetadataManager.java | 865 +++++++ .../group/consumer/ConsumerGroup.java | 535 +++++ .../group/GroupMetadataManagerTest.java | 2076 +++++++++++++++++ .../group/consumer/ConsumerGroupTest.java | 326 +++ 6 files changed, 3810 insertions(+), 1 deletion(-) create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java create mode 100644 group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java create mode 100644 group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 255537dd8d041..0c5d102596242 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -223,12 +223,15 @@ + + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 9cbb54b5cf3ee..a7d7e453c627d 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -320,7 +320,11 @@ + files="(ConsumerGroupMember|GroupMetadataManager).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 new file mode 100644 index 0000000000000..976e381bdf85a --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -0,0 +1,865 @@ +/* + * 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; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.ApiException; +import org.apache.kafka.common.errors.FencedMemberEpochException; +import org.apache.kafka.common.errors.GroupIdNotFoundException; +import org.apache.kafka.common.errors.GroupMaxSizeReachedException; +import org.apache.kafka.common.errors.InvalidRequestException; +import org.apache.kafka.common.errors.NotCoordinatorException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnsupportedAssignorException; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.requests.RequestContext; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; +import org.apache.kafka.coordinator.group.consumer.Assignment; +import org.apache.kafka.coordinator.group.consumer.ConsumerGroup; +import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.consumer.CurrentAssignmentBuilder; +import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder; +import org.apache.kafka.coordinator.group.consumer.TopicMetadata; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.timeline.SnapshotRegistry; +import org.apache.kafka.timeline.TimelineHashMap; +import org.slf4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.kafka.coordinator.group.RecordHelpers.newCurrentAssignmentRecord; +import static org.apache.kafka.coordinator.group.RecordHelpers.newCurrentAssignmentTombstoneRecord; +import static org.apache.kafka.coordinator.group.RecordHelpers.newGroupEpochRecord; +import static org.apache.kafka.coordinator.group.RecordHelpers.newGroupSubscriptionMetadataRecord; +import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionRecord; +import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionTombstoneRecord; +import static org.apache.kafka.coordinator.group.RecordHelpers.newTargetAssignmentTombstoneRecord; + +/** + * The GroupMetadataManager manages the metadata of all generic and consumer groups. It holds + * the hard and the soft state of the groups. This class has two kinds of methods: + * 1) The request handlers which handle the requests and generate a response and records to + * mutate the hard state. Those records will be written by the runtime and applied to the + * hard state via the replay methods. + * 2) The replay methods which apply records to the hard state. Those are used in the request + * handling as well as during the initial loading of the records from the partitions. + */ +public class GroupMetadataManager { + + public static class Builder { + private LogContext logContext = null; + private SnapshotRegistry snapshotRegistry = null; + private List assignors = null; + private TopicsImage topicsImage = null; + private int consumerGroupMaxSize = Integer.MAX_VALUE; + private int consumerGroupHeartbeatIntervalMs = 5000; + + Builder withLogContext(LogContext logContext) { + this.logContext = logContext; + return this; + } + + Builder withSnapshotRegistry(SnapshotRegistry snapshotRegistry) { + this.snapshotRegistry = snapshotRegistry; + return this; + } + + Builder withAssignors(List assignors) { + this.assignors = assignors; + return this; + } + + Builder withConsumerGroupMaxSize(int consumerGroupMaxSize) { + this.consumerGroupMaxSize = consumerGroupMaxSize; + return this; + } + + Builder withConsumerGroupHeartbeatInterval(int consumerGroupHeartbeatIntervalMs) { + this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs; + return this; + } + + Builder withTopicsImage(TopicsImage topicsImage) { + this.topicsImage = topicsImage; + return this; + } + + GroupMetadataManager build() { + if (logContext == null) logContext = new LogContext(); + if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); + if (topicsImage == null) topicsImage = TopicsImage.EMPTY; + + if (assignors == null || assignors.isEmpty()) { + throw new IllegalStateException("Assignors must be set before building."); + } + + return new GroupMetadataManager( + snapshotRegistry, + logContext, + assignors, + topicsImage, + consumerGroupMaxSize, + consumerGroupHeartbeatIntervalMs + ); + } + } + + /** + * The logger. + */ + private final Logger log; + + /** + * The snapshot registry. + */ + private final SnapshotRegistry snapshotRegistry; + + /** + * The list of supported assignors. + */ + private final Map assignors; + + /** + * The default assignor used. + */ + private final PartitionAssignor defaultAssignor; + + /** + * The generic and consumer groups keyed by their name. + */ + private final TimelineHashMap groups; + + /** + * The maximum number of members allowed in a single consumer group. + */ + private final int consumerGroupMaxSize; + + /** + * The heartbeat interval for consumer groups. + */ + private final int consumerGroupHeartbeatIntervalMs; + + /** + * The topics metadata (or image). + */ + private TopicsImage topicsImage; + + private GroupMetadataManager( + SnapshotRegistry snapshotRegistry, + LogContext logContext, + List assignors, + TopicsImage topicsImage, + int consumerGroupMaxSize, + int consumerGroupHeartbeatIntervalMs + ) { + this.log = logContext.logger(GroupMetadataManager.class); + this.snapshotRegistry = snapshotRegistry; + this.topicsImage = topicsImage; + this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity())); + this.defaultAssignor = assignors.get(0); + this.groups = new TimelineHashMap<>(snapshotRegistry, 0); + this.consumerGroupMaxSize = consumerGroupMaxSize; + this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs; + } + + /** + * Gets or maybe creates a consumer group. + * + * @param groupId The group id. + * @param createIfNotExists A boolean indicating whether the group should be + * created if it does not exist. + * + * @return A ConsumerGroup. + * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or + * if the group is not a consumer group. + */ + // Package private for testing. + ConsumerGroup getOrMaybeCreateConsumerGroup( + String groupId, + boolean createIfNotExists + ) throws GroupIdNotFoundException { + Group group = groups.get(groupId); + + if (group == null && !createIfNotExists) { + throw new GroupIdNotFoundException(String.format("Consumer group %s not found.", groupId)); + } + + if (group == null) { + ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId); + groups.put(groupId, consumerGroup); + return consumerGroup; + } else { + if (group.type() == Group.GroupType.CONSUMER) { + return (ConsumerGroup) group; + } else { + // We don't support upgrading/downgrading between protocols at the moment so + // we throw an exception if a group exists with the wrong type. + throw new GroupIdNotFoundException(String.format("Group %s is not a consumer group.", groupId)); + } + } + } + + /** + * Removes the group. + * + * @param groupId The group id. + */ + private void removeGroup( + String groupId + ) { + groups.remove(groupId); + } + + /** + * 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 throwIfConsumerGroupHeartbeatRequestIsInvalid( + ConsumerGroupHeartbeatRequestData request + ) throws InvalidRequestException, UnsupportedAssignorException { + if (request.groupId().isEmpty()) { + throw new InvalidRequestException("GroupId can't be empty."); + } + + if (request.memberEpoch() > 0 || request.memberEpoch() == -1) { + if (request.memberId().isEmpty()) { + throw new InvalidRequestException("MemberId can't be empty."); + } + if (request.instanceId() != null) { + throw new InvalidRequestException("InstanceId should only be provided in first request."); + } + if (request.rackId() != null) { + throw new InvalidRequestException("RackId should only be provided in first request."); + } + } else if (request.memberEpoch() == 0) { + if (request.rebalanceTimeoutMs() == -1) { + throw new InvalidRequestException("RebalanceTimeoutMs must be provided in first request."); + } + if (request.topicPartitions() == null || !request.topicPartitions().isEmpty()) { + throw new InvalidRequestException("TopicPartitions must be empty when (re-)joining."); + } + if (request.subscribedTopicNames() == null || request.subscribedTopicNames().isEmpty()) { + throw new InvalidRequestException("SubscribedTopicNames must be set in first request."); + } + if (request.serverAssignor() != null && !assignors.containsKey(request.serverAssignor())) { + throw new UnsupportedAssignorException("ServerAssignor " + request.serverAssignor() + + " is not supported. Supported assignors: " + String.join(", ", assignors.keySet()) + + "."); + } + } else { + throw new InvalidRequestException("MemberEpoch is invalid."); + } + + if (request.subscribedTopicRegex() != null) { + throw new InvalidRequestException("SubscribedTopicRegex is not supported yet."); + } + + if (request.clientAssignors() != null) { + throw new InvalidRequestException("Client side assignors are not supported yet."); + } + } + + /** + * 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 client + * has at least of subset of them. + * + * @param ownedTopicPartitions The partitions provided by the consumer in the request. + * @param target The partitions that they member should have. + * + * @return A boolean indicating whether the owned partitions are a subset or not. + */ + private boolean isSubset( + List ownedTopicPartitions, + Map> target + ) { + if (ownedTopicPartitions == null) return false; + + for (ConsumerGroupHeartbeatRequestData.TopicPartitions topicPartitions : ownedTopicPartitions) { + Set partitions = target.get(topicPartitions.topicId()); + 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 side defined. + * + * @param group The consumer group. + * @param memberId The member id. + * + * @throws GroupMaxSizeReachedException if the maximum capacity has been reached. + */ + private void throwIfConsumerGroupIsFull( + ConsumerGroup group, + String memberId + ) throws GroupMaxSizeReachedException { + // If the consumer group has reached its maximum capacity, the member is rejected if it is not + // already a member of the consumer group. + if (group.numMembers() >= consumerGroupMaxSize && (memberId.isEmpty() || !group.hasMember(memberId))) { + throw new GroupMaxSizeReachedException("The consumer group has reached its maximum capacity of " + + consumerGroupMaxSize + " members."); + } + } + + /** + * Validates the member epoch provided in the heartbeat request. + * + * @param member The consumer group member. + * @param memberEpoch The member epoch. + * @param ownedTopicPartitions The owned partitions. + * + * @throws NotCoordinatorException if the provided epoch is ahead of the epoch known + * by this coordinator. This suggests that the member + * got a higher epoch from another coordinator. + * @throws FencedMemberEpochException if the provided epoch is behind the epoch known + * by this coordinator. + */ + private void throwIfMemberEpochIsInvalid( + ConsumerGroupMember member, + int memberEpoch, + List ownedTopicPartitions + ) { + if (memberEpoch > member.memberEpoch()) { + // The member has likely got a bump from another coordinator and this coordinator + // is stale. Return NOT_COORDINATOR to force the member to refresh its coordinator. + throw new NotCoordinatorException("The consumer group member has got a larger member " + + "epoch (" + memberEpoch + ") than the one known by this group coordinator (" + + member.memberEpoch() + ")."); + } else if (memberEpoch < 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 (memberEpoch != member.previousMemberEpoch() || !isSubset(ownedTopicPartitions, member.assignedPartitions())) { + throw new FencedMemberEpochException("The consumer group member has an old member " + + "epoch. The member must abandon all its partitions and rejoin."); + } + } + } + + private ConsumerGroupHeartbeatResponseData.Assignment createResponseAssignment( + ConsumerGroupMember member + ) { + ConsumerGroupHeartbeatResponseData.Assignment assignment = new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(fromAssignmentMap(member.assignedPartitions())); + + if (member.state() == ConsumerGroupMember.MemberState.ASSIGNING) { + assignment.setPendingTopicPartitions(fromAssignmentMap(member.partitionsPendingAssignment())); + } + + return assignment; + } + + private List fromAssignmentMap( + Map> assignment + ) { + return assignment.entrySet().stream() + .map(keyValue -> new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(keyValue.getKey()) + .setPartitions(new ArrayList<>(keyValue.getValue()))) + .collect(Collectors.toList()); + } + + private OptionalInt ofSentinel(int value) { + return value != -1 ? OptionalInt.of(value) : OptionalInt.empty(); + } + + /** + * Handles a regular heartbeat from a consumer 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. + * @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 subscribedTopicNames The list of subscribed topic names from the request + * of null. + * @param subscribedTopicRegex The regular expression based subscription from the + * request or null. + * @param assignorName The assignor name from the request or null. + * @param ownedTopicPartitions The list of owned partitions from the request or null. + * + * @return A Result containing the ConsumerGroupHeartbeat response and + * a list of records to update the state machine. + */ + private Result consumerGroupHeartbeat( + String groupId, + String memberId, + int memberEpoch, + String instanceId, + String rackId, + int rebalanceTimeoutMs, + String clientId, + String clientHost, + List subscribedTopicNames, + String subscribedTopicRegex, + String assignorName, + List ownedTopicPartitions + ) throws ApiException { + List records = new ArrayList<>(); + boolean createIfNotExists = memberEpoch == 0; + + ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, createIfNotExists); + throwIfConsumerGroupIsFull(group, memberId); + + if (memberId.isEmpty()) memberId = Uuid.randomUuid().toString(); + ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, createIfNotExists); + throwIfMemberEpochIsInvalid(member, memberEpoch, ownedTopicPartitions); + + if (memberEpoch == 0) { + log.info("[GroupId " + groupId + "] Member " + memberId + " re-joins the consumer group."); + } + + // Update the subscription part of the member if we received new values. If the member has + // changed, we write it to the log. If the subscribed topics have changed, we also recompute + // the subscription metadata. + int groupEpoch = group.groupEpoch(); + Map subscriptionMetadata = group.subscriptionMetadata(); + ConsumerGroupMember updatedMember = new ConsumerGroupMember.Builder(member) + .maybeUpdateInstanceId(Optional.ofNullable(instanceId)) + .maybeUpdateRackId(Optional.ofNullable(rackId)) + .maybeUpdateRebalanceTimeoutMs(ofSentinel(rebalanceTimeoutMs)) + .maybeUpdateServerAssignorName(Optional.ofNullable(assignorName)) + .maybeUpdateSubscribedTopicNames(Optional.ofNullable(subscribedTopicNames)) + .maybeUpdateSubscribedTopicRegex(Optional.ofNullable(subscribedTopicRegex)) + .setClientId(clientId) + .setClientHost(clientHost) + .build(); + + if (!updatedMember.equals(member)) { + records.add(newMemberSubscriptionRecord(groupId, updatedMember)); + + if (!updatedMember.subscribedTopicNames().equals(member.subscribedTopicNames())) { + log.info("[GroupId " + groupId + "] Member " + memberId + " updated its subscribed topics to: " + + updatedMember.subscribedTopicNames()); + + subscriptionMetadata = group.computeSubscriptionMetadata( + updatedMember.memberId(), + updatedMember.subscribedTopicNames(), + topicsImage + ); + + if (!subscriptionMetadata.equals(group.subscriptionMetadata())) { + log.info("[GroupId " + groupId + "] Computed new subscription metadata: " + + subscriptionMetadata + "."); + records.add(newGroupSubscriptionMetadataRecord(groupId, subscriptionMetadata)); + } + + groupEpoch += 1; + records.add(newGroupEpochRecord(groupId, groupEpoch)); + + log.info("[GroupId " + groupId + "] Bumped group epoch to " + groupEpoch + "."); + } + + member = updatedMember; + } + + // Update target assignment if needed. If the new target has any changes, we write the + // changes to the log. + int targetAssignmentEpoch = group.assignmentEpoch(); + Assignment targetAssignment = group.targetAssignment(memberId); + if (groupEpoch > targetAssignmentEpoch) { + String preferredServerAssignor = group.preferredServerAssignor( + member.memberId(), + member.serverAssignorName() + ).orElse(defaultAssignor.name()); + + try { + TargetAssignmentBuilder.TargetAssignmentResult assignmentResult = + new TargetAssignmentBuilder(groupId, groupEpoch, assignors.get(preferredServerAssignor)) + .withMembers(group.members()) + .withSubscriptionMetadata(subscriptionMetadata) + .withTargetAssignment(group.targetAssignments()) + .addOrUpdateMember(member.memberId(), member) + .build(); + + log.info("[GroupId " + groupId + "] Computed a new target assignment for epoch " + groupEpoch + ": " + + assignmentResult.targetAssignment() + "."); + + records.addAll(assignmentResult.records()); + targetAssignment = assignmentResult.targetAssignment().get(member.memberId()); + targetAssignmentEpoch = groupEpoch; + } catch (PartitionAssignorException ex) { + String msg = "Failed to compute a new target assignment for epoch " + groupEpoch + ": " + ex + "."; + log.error("[GroupId " + groupId + "] " + msg); + throw new UnknownServerException(msg, ex); + } + } + + // If the member is stable and its next epoch matches the current target epoch + // of the assignment, we can skip this reconciliation. + boolean assignmentUpdated = false; + if (member.state() != ConsumerGroupMember.MemberState.STABLE + || member.nextMemberEpoch() != targetAssignmentEpoch) { + updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(targetAssignmentEpoch, targetAssignment) + .withCurrentPartitionEpoch(group::currentPartitionEpoch) + .withOwnedTopicPartitions(ownedTopicPartitions) + .build(); + + // Checking the reference is enough here because a new instance + // is created only when the state has changed. + if (updatedMember != member) { + assignmentUpdated = true; + records.add(newCurrentAssignmentRecord(groupId, updatedMember)); + + log.info("[GroupId " + groupId + "] Member " + memberId + " transitioned from " + + member.currentAssignmentSummary() + " to " + updatedMember.currentAssignmentSummary() + "."); + + // TODO(dajac) Starts or restarts the timer for the revocation timeout. + + member = updatedMember; + } + } + + // TODO(dajac) Starts or restarts the timer for the session timeout. + + ConsumerGroupHeartbeatResponseData response = new ConsumerGroupHeartbeatResponseData() + .setMemberId(member.memberId()) + .setMemberEpoch(member.memberEpoch()) + .setHeartbeatIntervalMs(consumerGroupHeartbeatIntervalMs); + + if (ownedTopicPartitions != null || memberEpoch == 0 || assignmentUpdated) { + response.setAssignment(createResponseAssignment(member)); + } + + return new Result<>(records, response); + } + + /** + * Handles leave request from a consumer group member. + * @param groupId The group id from the request. + * @param memberId The member id from the request. + * + * @return A Result containing the ConsumerGroupHeartbeat response and + * a list of records to update the state machine. + */ + private Result consumerGroupLeave( + String groupId, + String memberId + ) throws ApiException { + List records = new ArrayList<>(); + + ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); + + log.info("[GroupId " + groupId + "] Member " + memberId + " left the consumer group."); + + // Write tombstones for the member. The order matters here. + records.add(newCurrentAssignmentTombstoneRecord(groupId, memberId)); + records.add(newTargetAssignmentTombstoneRecord(groupId, memberId)); + records.add(newMemberSubscriptionTombstoneRecord(groupId, memberId)); + + // We update the subscription metadata without the leaving member. + Map subscriptionMetadata = group.computeSubscriptionMetadata( + memberId, + null, + topicsImage + ); + + if (!subscriptionMetadata.equals(group.subscriptionMetadata())) { + log.info("[GroupId " + groupId + "] Computed new subscription metadata: " + + subscriptionMetadata + "."); + records.add(newGroupSubscriptionMetadataRecord(groupId, subscriptionMetadata)); + } + + // We bump the group epoch. + int groupEpoch = group.groupEpoch() + 1; + records.add(newGroupEpochRecord(groupId, groupEpoch)); + + // We update the target assignment for the group and write it to + // the log. + String assignorName = group.preferredServerAssignor( + member.memberId(), + Optional.empty() + ).orElse(defaultAssignor.name()); + + try { + TargetAssignmentBuilder.TargetAssignmentResult assignmentResult = + new TargetAssignmentBuilder(groupId, groupEpoch, assignors.get(assignorName)) + .withMembers(group.members()) + .withSubscriptionMetadata(subscriptionMetadata) + .removeMember(member.memberId()) + .build(); + + log.info("[GroupId " + groupId + "] Computed a new target assignment for epoch " + groupEpoch + ": " + + assignmentResult.targetAssignment() + "."); + + records.addAll(assignmentResult.records()); + } catch (PartitionAssignorException ex) { + String msg = "Failed to compute a new target assignment for epoch " + groupEpoch + ": " + ex + "."; + log.error("[GroupId " + groupId + "] " + msg); + throw new UnknownServerException(msg, ex); + } + + return new Result<>(records, new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(-1) + ); + } + + /** + * Handles a ConsumerGroupHeartbeat request. + * + * @param context The request context. + * @param request The actual ConsumerGroupHeartbeat request. + * + * @return A Result containing the ConsumerGroupHeartbeat response and + * a list of records to update the state machine. + */ + public Result consumerGroupHeartbeat( + RequestContext context, + ConsumerGroupHeartbeatRequestData request + ) throws ApiException { + throwIfConsumerGroupHeartbeatRequestIsInvalid(request); + + if (request.memberEpoch() == -1) { + // -1 means that the member wants to leave the group. + return consumerGroupLeave( + request.groupId(), + request.memberId() + ); + } else { + // Otherwise, it is a regular heartbeat. + return consumerGroupHeartbeat( + request.groupId(), + request.memberId(), + request.memberEpoch(), + request.instanceId(), + request.rackId(), + request.rebalanceTimeoutMs(), + context.clientId(), + context.clientAddress.toString(), + request.subscribedTopicNames(), + request.subscribedTopicRegex(), + request.serverAssignor(), + request.topicPartitions() + ); + } + } + + /** + * Replays ConsumerGroupMemberMetadataKey/Value to update the hard state of + * the consumer group. + * + * @param key A ConsumerGroupMemberMetadataKey key. + * @param value A ConsumerGroupMemberMetadataValue record. + */ + public void replay( + ConsumerGroupMemberMetadataKey key, + ConsumerGroupMemberMetadataValue value + ) { + String groupId = key.groupId(); + String memberId = key.memberId(); + + if (value != null) { + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, true); + ConsumerGroupMember oldMember = consumerGroup.getOrMaybeCreateMember(memberId, true); + consumerGroup.updateMember(new ConsumerGroupMember.Builder(oldMember) + .updateWith(value) + .build()); + } else { + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroupMember oldMember = consumerGroup.getOrMaybeCreateMember(memberId, false); + if (oldMember.memberEpoch() != -1) { + throw new IllegalStateException("Received a tombstone record to delete member " + memberId + + " but did not receive ConsumerGroupCurrentMemberAssignmentValue tombstone."); + } + if (consumerGroup.targetAssignments().containsKey(memberId)) { + throw new IllegalStateException("Received a tombstone record to delete member " + memberId + + " but did not receive ConsumerGroupTargetAssignmentMetadataValue tombstone."); + } + consumerGroup.removeMember(memberId); + } + } + + /** + * Replays ConsumerGroupMetadataKey/Value to update the hard state of + * the consumer group. + * + * @param key A ConsumerGroupMetadataKey key. + * @param value A ConsumerGroupMetadataValue record. + */ + public void replay( + ConsumerGroupMetadataKey key, + ConsumerGroupMetadataValue value + ) { + String groupId = key.groupId(); + + if (value != null) { + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, true); + consumerGroup.setGroupEpoch(value.epoch()); + } else { + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + if (!consumerGroup.members().isEmpty()) { + throw new IllegalStateException("Received a tombstone record to delete group " + groupId + + " but the group still has " + consumerGroup.members().size() + " members."); + } + if (!consumerGroup.targetAssignments().isEmpty()) { + throw new IllegalStateException("Received a tombstone record to delete group " + groupId + + " but the group still has " + consumerGroup.targetAssignments().size() + " members."); + } + if (consumerGroup.assignmentEpoch() != -1) { + throw new IllegalStateException("Received a tombstone record to delete group " + groupId + + " but did not receive ConsumerGroupTargetAssignmentMetadataValue tombstone."); + } + removeGroup(groupId); + } + + } + + /** + * Replays ConsumerGroupPartitionMetadataKey/Value to update the hard state of + * the consumer group. + * + * @param key A ConsumerGroupPartitionMetadataKey key. + * @param value A ConsumerGroupPartitionMetadataValue record. + */ + public void replay( + ConsumerGroupPartitionMetadataKey key, + ConsumerGroupPartitionMetadataValue value + ) { + String groupId = key.groupId(); + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + + if (value != null) { + Map subscriptionMetadata = new HashMap<>(); + value.topics().forEach(topicMetadata -> { + subscriptionMetadata.put(topicMetadata.topicName(), TopicMetadata.fromRecord(topicMetadata)); + }); + consumerGroup.setSubscriptionMetadata(subscriptionMetadata); + } else { + consumerGroup.setSubscriptionMetadata(Collections.emptyMap()); + } + } + + /** + * Replays ConsumerGroupTargetAssignmentMemberKey/Value to update the hard state of + * the consumer group. + * + * @param key A ConsumerGroupTargetAssignmentMemberKey key. + * @param value A ConsumerGroupTargetAssignmentMemberValue record. + */ + public void replay( + ConsumerGroupTargetAssignmentMemberKey key, + ConsumerGroupTargetAssignmentMemberValue value + ) { + String groupId = key.groupId(); + String memberId = key.memberId(); + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + + if (value != null) { + consumerGroup.updateTargetAssignment(memberId, Assignment.fromRecord(value)); + } else { + consumerGroup.removeTargetAssignment(memberId); + } + } + + /** + * Replays ConsumerGroupTargetAssignmentMetadataKey/Value to update the hard state of + * the consumer group. + * + * @param key A ConsumerGroupTargetAssignmentMetadataKey key. + * @param value A ConsumerGroupTargetAssignmentMetadataValue record. + */ + public void replay( + ConsumerGroupTargetAssignmentMetadataKey key, + ConsumerGroupTargetAssignmentMetadataValue value + ) { + String groupId = key.groupId(); + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + + if (value != null) { + consumerGroup.setAssignmentEpoch(value.assignmentEpoch()); + } else { + if (!consumerGroup.targetAssignments().isEmpty()) { + throw new IllegalStateException("Received a tombstone record to delete target assignment of " + groupId + + " but the assignment still has " + consumerGroup.targetAssignments().size() + " members."); + } + consumerGroup.setAssignmentEpoch(-1); + } + } + + /** + * Replays ConsumerGroupCurrentMemberAssignmentKey/Value to update the hard state of + * the consumer group. + * + * @param key A ConsumerGroupCurrentMemberAssignmentKey key. + * @param value A ConsumerGroupCurrentMemberAssignmentValue record. + */ + public void replay( + ConsumerGroupCurrentMemberAssignmentKey key, + ConsumerGroupCurrentMemberAssignmentValue value + ) { + String groupId = key.groupId(); + String memberId = key.memberId(); + ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); + ConsumerGroupMember oldMember = consumerGroup.getOrMaybeCreateMember(memberId, false); + + if (value != null) { + ConsumerGroupMember newMember = new ConsumerGroupMember.Builder(oldMember) + .updateWith(value) + .build(); + consumerGroup.updateMember(newMember); + } else { + ConsumerGroupMember newMember = new ConsumerGroupMember.Builder(oldMember) + .setMemberEpoch(-1) + .setPreviousMemberEpoch(-1) + .setNextMemberEpoch(-1) + .setAssignedPartitions(Collections.emptyMap()) + .setPartitionsPendingRevocation(Collections.emptyMap()) + .setPartitionsPendingAssignment(Collections.emptyMap()) + .build(); + consumerGroup.updateMember(newMember); + } + } +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java new file mode 100644 index 0000000000000..29b427859b975 --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -0,0 +1,535 @@ +/* + * 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.consumer; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.coordinator.group.Group; +import org.apache.kafka.image.TopicImage; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.timeline.SnapshotRegistry; +import org.apache.kafka.timeline.TimelineHashMap; +import org.apache.kafka.timeline.TimelineInteger; +import org.apache.kafka.timeline.TimelineObject; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Consumer; + +/** + * A Consumer Group. All the metadata in this class are backed by + * records in the __consumer_offsets partitions. + */ +public class ConsumerGroup implements Group { + + public enum ConsumerGroupState { + EMPTY("empty"), + ASSIGNING("assigning"), + RECONCILING("reconciling"), + STABLE("stable"), + DEAD("dead"); + + private final String name; + + ConsumerGroupState(String name) { + this.name = name; + } + + @Override + public String toString() { + return name; + } + } + + /** + * The snapshot registry. + */ + private final SnapshotRegistry snapshotRegistry; + + /** + * The group id. + */ + private final String groupId; + + /** + * The group state. + */ + private final TimelineObject state; + + /** + * The group epoch. The epoch is incremented whenever the subscriptions + * are updated and it will trigger the computation of a new assignment + * for the group. + */ + private final TimelineInteger groupEpoch; + + /** + * The group members. + */ + private final TimelineHashMap members; + + /** + * The metadata of the subscribed topics. + */ + private final TimelineHashMap subscribedTopicMetadata; + + /** + * The assignment epoch. An assignment epoch smaller than the group epoch means + * that a new assignment is required. The assignment epoch is updated when a new + * assignment is installed. + */ + private final TimelineInteger assignmentEpoch; + + /** + * The target assignment. + */ + private final TimelineHashMap assignments; + + /** + * The current partition epoch maps each topic-partitions to their current epoch where + * the epoch is the epoch of their owners. When a member revokes a partition, it removes + * itself from this map. When a member gets a partition, it adds itself to this map. + */ + private final TimelineHashMap> currentPartitionEpoch; + + public ConsumerGroup( + SnapshotRegistry snapshotRegistry, + String groupId + ) { + this.snapshotRegistry = Objects.requireNonNull(snapshotRegistry); + this.groupId = Objects.requireNonNull(groupId); + this.state = new TimelineObject<>(snapshotRegistry, ConsumerGroupState.EMPTY); + this.groupEpoch = new TimelineInteger(snapshotRegistry); + this.members = new TimelineHashMap<>(snapshotRegistry, 0); + this.subscribedTopicMetadata = new TimelineHashMap<>(snapshotRegistry, 0); + this.assignmentEpoch = new TimelineInteger(snapshotRegistry); + this.assignments = new TimelineHashMap<>(snapshotRegistry, 0); + this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 0); + } + + /** + * The type of this group. + * + * @return The group type (Consumer). + */ + @Override + public GroupType type() { + return GroupType.CONSUMER; + } + + /** + * The state of this group. + * + * @return The current state as a String. + */ + @Override + public String stateAsString() { + return state.get().toString(); + } + + /** + * The group id. + * + * @return The group id. + */ + @Override + public String groupId() { + return groupId; + } + + /** + * The state of this group. + * + * @return The current state. + */ + public ConsumerGroupState state() { + return state.get(); + } + + /** + * Returns the current group epoch. + * + * @return The group epoch. + */ + public int groupEpoch() { + return groupEpoch.get(); + } + + /** + * Sets the group epoch. + * + * @param groupEpoch The new group epoch. + */ + public void setGroupEpoch(int groupEpoch) { + this.groupEpoch.set(groupEpoch); + maybeUpdateGroupState(); + } + + /** + * Returns the current assignment epoch. + * + * @return The current assignment epoch. + */ + public int assignmentEpoch() { + return assignmentEpoch.get(); + } + + /** + * Sets the assignment epoch. + * + * @param assignmentEpoch The new assignment epoch. + */ + public void setAssignmentEpoch(int assignmentEpoch) { + this.assignmentEpoch.set(assignmentEpoch); + maybeUpdateGroupState(); + } + + /** + * Gets or creates a member. + * + * @param memberId The member id. + * @param createIfNotExists Booleans indicating whether the member must be + * created if it does not exist. + * + * @return A ConsumerGroupMember. + */ + public ConsumerGroupMember getOrMaybeCreateMember( + String memberId, + boolean createIfNotExists + ) { + ConsumerGroupMember member = members.get(memberId); + if (member == null) { + if (!createIfNotExists) { + throw new UnknownMemberIdException(String.format("Member %s is not a member of group %s.", + memberId, groupId)); + } + member = new ConsumerGroupMember.Builder(memberId).build(); + members.put(memberId, member); + } + + return member; + } + + /** + * Updates the member. + * + * @param newMember The new member state. + */ + public void updateMember(ConsumerGroupMember newMember) { + ConsumerGroupMember oldMember = members.put(newMember.memberId(), newMember); + maybeUpdatePartitionEpoch(oldMember, newMember); + maybeUpdateGroupState(); + } + + /** + * Remove the member from the group. + * + * @param memberId The member id to remove. + */ + public void removeMember(String memberId) { + ConsumerGroupMember member = members.remove(memberId); + maybeRemovePartitionEpoch(member); + maybeUpdateGroupState(); + } + + /** + * Returns true if the member exists. + * + * @param memberId The member id. + * + * @return A boolean indicating whether the member exists or not. + */ + public boolean hasMember(String memberId) { + return members.containsKey(memberId); + } + + /** + * Returns the number of members in the group. + * + * @return The number of members. + */ + public int numMembers() { + return members.size(); + } + + /** + * Returns the members keyed by their id. + * + * @return A immutable Map containing all the members. + */ + public Map members() { + return Collections.unmodifiableMap(members); + } + + /** + * Returns the current target assignment of the member. + * + * @return The ConsumerGroupMemberAssignment or an EMPTY one if it does not + * exist. + */ + public Assignment targetAssignment(String memberId) { + return assignments.getOrDefault(memberId, Assignment.EMPTY); + } + + /** + * Updates target assignment of a member. + * + * @param memberId The member id. + * @param newTargetAssignment The new target assignment. + */ + public void updateTargetAssignment(String memberId, Assignment newTargetAssignment) { + assignments.put(memberId, newTargetAssignment); + } + + /** + * Removes the target assignment of a member. + * + * @param memberId The member id. + */ + public void removeTargetAssignment(String memberId) { + assignments.remove(memberId); + } + + /** + * Returns the target assignments for the entire group. + * + * @return A immutable Map containing all the target assignments. + */ + public Map targetAssignments() { + return Collections.unmodifiableMap(assignments); + } + + /** + * Returns the current epoch of a partition or -1 if the partition + * does not have one. + * + * @param topicId The topic id. + * @param partitionId The partition id. + * + * @return The epoch or -1. + */ + public int currentPartitionEpoch( + Uuid topicId, int partitionId + ) { + Map partitions = currentPartitionEpoch.get(topicId); + if (partitions == null) { + return -1; + } else { + return partitions.getOrDefault(partitionId, -1); + } + } + + /** + * Compute the preferred (server side) assignor for the group while + * using the provided assignor for the member. + * + * @param updatedMemberId The member id. + * @param serverAssignorNameOpt The assignor name. + * + * @return An Optional containing the preferred assignor. + */ + public Optional preferredServerAssignor( + String updatedMemberId, + Optional serverAssignorNameOpt + ) { + Map counts = new HashMap<>(); + + serverAssignorNameOpt.ifPresent(serverAssignorName -> + counts.put(serverAssignorName, 1) + ); + + members.forEach((memberId, member) -> { + if (!memberId.equals(updatedMemberId) && member.serverAssignorName().isPresent()) { + counts.compute(member.serverAssignorName().get(), (k, v) -> v == null ? 1 : v + 1); + } + }); + + return counts.entrySet().stream() + .max(Map.Entry.comparingByValue()) + .map(Map.Entry::getKey); + } + + /** + * Returns the subscription metadata for all the topics whose + * members are subscribed to. + * + * @return An immutable Map containing the subscription metadata. + */ + public Map subscriptionMetadata() { + return Collections.unmodifiableMap(subscribedTopicMetadata); + } + + /** + * Updates the subscription metadata. This replace the previous one. + * + * @param subscriptionMetadata The new subscription metadata. + */ + public void setSubscriptionMetadata( + Map subscriptionMetadata + ) { + this.subscribedTopicMetadata.clear(); + this.subscribedTopicMetadata.putAll(subscriptionMetadata); + } + + /** + * Computes new subscription metadata but with specific information for + * a member. + * + * @param memberId The member id. + * @param updatedMemberSubscriptions The member's updated topic subscriptions. + * @param topicsImage The topic metadata. + * + * @return The new subscription metadata as an immutable Map. + */ + public Map computeSubscriptionMetadata( + String memberId, + List updatedMemberSubscriptions, + TopicsImage topicsImage + ) { + Map newSubscriptionMetadata = new HashMap<>(subscriptionMetadata().size()); + + Consumer> updateSubscription = subscribedTopicNames -> { + subscribedTopicNames.forEach(topicName -> + newSubscriptionMetadata.computeIfAbsent(topicName, __ -> { + TopicImage topicImage = topicsImage.getTopic(topicName); + if (topicImage == null) { + return null; + } else { + return new TopicMetadata( + topicImage.id(), + topicImage.name(), + topicImage.partitions().size() + ); + } + }) + ); + }; + + if (updatedMemberSubscriptions != null) { + updateSubscription.accept(updatedMemberSubscriptions); + } + + members.forEach((mid, member) -> { + if (!mid.equals(memberId)) { + updateSubscription.accept(member.subscribedTopicNames()); + } + }); + + return Collections.unmodifiableMap(newSubscriptionMetadata); + } + + /** + * Updates the current state of the group. + */ + private void maybeUpdateGroupState() { + if (members.isEmpty()) { + state.set(ConsumerGroupState.EMPTY); + } else if (groupEpoch.get() > assignmentEpoch.get()) { + state.set(ConsumerGroupState.ASSIGNING); + } else { + for (Map.Entry keyValue : members.entrySet()) { + ConsumerGroupMember member = keyValue.getValue(); + if (member.nextMemberEpoch() != assignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) { + state.set(ConsumerGroupState.RECONCILING); + return; + } + } + + state.set(ConsumerGroupState.STABLE); + } + } + + /** + * Updates the partition epochs based on the old and the new member. + */ + private void maybeUpdatePartitionEpoch( + ConsumerGroupMember oldMember, + ConsumerGroupMember newMember + ) { + if (oldMember == null) { + addPartitionEpochs(newMember.assignedPartitions(), newMember.memberEpoch()); + addPartitionEpochs(newMember.partitionsPendingRevocation(), newMember.memberEpoch()); + } else { + if (!oldMember.assignedPartitions().equals(newMember.assignedPartitions())) { + removePartitionEpochs(oldMember.assignedPartitions()); + addPartitionEpochs(newMember.assignedPartitions(), newMember.memberEpoch()); + } + if (!oldMember.partitionsPendingRevocation().equals(newMember.partitionsPendingRevocation())) { + removePartitionEpochs(oldMember.partitionsPendingRevocation()); + addPartitionEpochs(newMember.partitionsPendingRevocation(), newMember.memberEpoch()); + } + } + } + + /** + * Removes the partition epochs for the provided member. + */ + private void maybeRemovePartitionEpoch( + ConsumerGroupMember oldMember + ) { + if (oldMember != null) { + removePartitionEpochs(oldMember.assignedPartitions()); + removePartitionEpochs(oldMember.partitionsPendingRevocation()); + } + } + + /** + * Removes the partition epochs based on the provided assignment. + */ + private void removePartitionEpochs( + Map> assignment + ) { + assignment.forEach((topicId, assignedPartitions) -> { + currentPartitionEpoch.compute(topicId, (__, partitionsOrNull) -> { + if (partitionsOrNull != null) { + assignedPartitions.forEach(partitionsOrNull::remove); + if (partitionsOrNull.isEmpty()) { + return null; + } else { + return partitionsOrNull; + } + } else { + return null; + } + }); + }); + } + + /** + * Adds the partitions epoch based on the provided assignment. + */ + private void addPartitionEpochs( + Map> assignment, + int epoch + ) { + assignment.forEach((topicId, assignedPartitions) -> { + currentPartitionEpoch.compute(topicId, (__, partitionsOrNull) -> { + if (partitionsOrNull == null) partitionsOrNull = new TimelineHashMap<>(snapshotRegistry, 1); + for (Integer partitionId : assignedPartitions) { + partitionsOrNull.put(partitionId, epoch); + } + return partitionsOrNull; + }); + }); + } + +} 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 new file mode 100644 index 0000000000000..de1ad2d1d5dac --- /dev/null +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -0,0 +1,2076 @@ +/* + * 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; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.FencedMemberEpochException; +import org.apache.kafka.common.errors.GroupIdNotFoundException; +import org.apache.kafka.common.errors.GroupMaxSizeReachedException; +import org.apache.kafka.common.errors.InvalidRequestException; +import org.apache.kafka.common.errors.NotCoordinatorException; +import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnsupportedAssignorException; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; +import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; +import org.apache.kafka.common.metadata.PartitionRecord; +import org.apache.kafka.common.metadata.TopicRecord; +import org.apache.kafka.common.network.ClientInformation; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.requests.RequestContext; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; +import org.apache.kafka.coordinator.group.assignor.GroupAssignment; +import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; +import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; +import org.apache.kafka.coordinator.group.consumer.Assignment; +import org.apache.kafka.coordinator.group.consumer.ConsumerGroup; +import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember; +import org.apache.kafka.coordinator.group.consumer.TopicMetadata; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupPartitionMetadataValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMemberValue; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataKey; +import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue; +import org.apache.kafka.image.TopicImage; +import org.apache.kafka.image.TopicsDelta; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.server.common.ApiMessageAndVersion; +import org.apache.kafka.timeline.SnapshotRegistry; +import org.junit.jupiter.api.Test; + +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment; +import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment; +import static org.junit.jupiter.api.AssertionFailureBuilder.assertionFailure; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class GroupMetadataManagerTest { + static class MockPartitionAssignor implements PartitionAssignor { + private final String name; + private AssignmentSpec lastSpecReceived = null; + private GroupAssignment prepareGroupAssignment = null; + + MockPartitionAssignor(String name) { + this.name = name; + } + + public AssignmentSpec lastSpecReceived() { + return lastSpecReceived; + } + + public void prepareGroupAssignment(GroupAssignment prepareGroupAssignment) { + this.prepareGroupAssignment = prepareGroupAssignment; + } + + @Override + public String name() { + return name; + } + + @Override + public GroupAssignment assign(AssignmentSpec assignmentSpec) throws PartitionAssignorException { + lastSpecReceived = assignmentSpec; + return prepareGroupAssignment; + } + } + + public static class TopicsImageBuilder { + private TopicsDelta delta = new TopicsDelta(TopicsImage.EMPTY); + + public TopicsImageBuilder addTopic( + Uuid topicId, + String topicName, + int numPartitions + ) { + delta.replay(new TopicRecord().setTopicId(topicId).setName(topicName)); + for (int i = 0; i < numPartitions; i++) { + delta.replay(new PartitionRecord() + .setTopicId(topicId) + .setPartitionId(i)); + } + return this; + } + + public TopicsImage build() { + return delta.apply(); + } + } + + static class ConsumerGroupBuilder { + private final String groupId; + private final int groupEpoch; + private int assignmentEpoch; + private final Map members = new HashMap<>(); + private final Map assignments = new HashMap<>(); + + public ConsumerGroupBuilder(String groupId, int groupEpoch) { + this.groupId = groupId; + this.groupEpoch = groupEpoch; + this.assignmentEpoch = 0; + } + + public ConsumerGroupBuilder withMember(ConsumerGroupMember member) { + this.members.put(member.memberId(), member); + return this; + } + + public ConsumerGroupBuilder withAssignment(String memberId, Map> assignment) { + this.assignments.put(memberId, new Assignment(assignment)); + return this; + } + + public ConsumerGroupBuilder withAssignment(String memberId, Assignment assignment) { + this.assignments.put(memberId, assignment); + return this; + } + + public ConsumerGroupBuilder withAssignmentEpoch(int assignmentEpoch) { + this.assignmentEpoch = assignmentEpoch; + return this; + } + + public List build(TopicsImage topicsImage) { + List records = new ArrayList<>(); + + // Add subscription records for members. + members.forEach((memberId, member) -> { + records.add(RecordHelpers.newMemberSubscriptionRecord(groupId, member)); + }); + + // Add subscription metadata. + Map subscriptionMetadata = new HashMap<>(); + members.forEach((memberId, member) -> { + member.subscribedTopicNames().forEach(topicName -> + subscriptionMetadata.computeIfAbsent(topicName, __ -> { + TopicImage topicImage = topicsImage.getTopic(topicName); + if (topicImage == null) { + return null; + } else { + return new TopicMetadata( + topicImage.id(), + topicImage.name(), + topicImage.partitions().size() + ); + } + }) + ); + }); + if (!subscriptionMetadata.isEmpty()) { + records.add(RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, subscriptionMetadata)); + } + + // Add group epoch record. + records.add(RecordHelpers.newGroupEpochRecord(groupId, groupEpoch)); + + // Add target assignment records. + assignments.forEach((memberId, assignment) -> { + records.add(RecordHelpers.newTargetAssignmentRecord(groupId, memberId, assignment.partitions())); + }); + + // Add target assignment epoch. + records.add(RecordHelpers.newTargetAssignmentEpochRecord(groupId, assignmentEpoch)); + + // Add current assignment records for members. + members.forEach((memberId, member) -> { + records.add(RecordHelpers.newCurrentAssignmentRecord(groupId, member)); + }); + + return records; + } + } + + static class GroupMetadataManagerTestContext { + static class Builder { + private LogContext logContext; + private SnapshotRegistry snapshotRegistry; + private TopicsImage topicsImage; + private List assignors; + private List consumerGroupBuilders = new ArrayList<>(); + private int consumerGroupMaxSize = Integer.MAX_VALUE; + + public Builder withLogContext(LogContext logContext) { + this.logContext = logContext; + return this; + } + + public Builder withSnapshotRegistry(SnapshotRegistry snapshotRegistry) { + this.snapshotRegistry = snapshotRegistry; + return this; + } + + public Builder withTopicsImage(TopicsImage topicsImage) { + this.topicsImage = topicsImage; + return this; + } + + public Builder withAssignors(List assignors) { + this.assignors = assignors; + return this; + } + + public Builder withConsumerGroup(ConsumerGroupBuilder builder) { + this.consumerGroupBuilders.add(builder); + return this; + } + + public Builder withConsumerGroupMaxSize(int consumerGroupMaxSize) { + this.consumerGroupMaxSize = consumerGroupMaxSize; + return this; + } + + public GroupMetadataManagerTestContext build() { + if (logContext == null) logContext = new LogContext(); + if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); + if (topicsImage == null) topicsImage = TopicsImage.EMPTY; + if (assignors == null) assignors = Collections.emptyList(); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext( + snapshotRegistry, + new GroupMetadataManager.Builder() + .withSnapshotRegistry(snapshotRegistry) + .withLogContext(logContext) + .withTopicsImage(topicsImage) + .withConsumerGroupHeartbeatInterval(5000) + .withConsumerGroupMaxSize(consumerGroupMaxSize) + .withAssignors(assignors) + .build() + ); + + consumerGroupBuilders.forEach(builder -> { + builder.build(topicsImage).forEach(context::replay); + }); + + context.commit(); + + return context; + } + } + + final SnapshotRegistry snapshotRegistry; + final GroupMetadataManager groupMetadataManager; + + long lastCommittedOffset = 0L; + long lastWrittenOffset = 0L; + + public GroupMetadataManagerTestContext( + SnapshotRegistry snapshotRegistry, + GroupMetadataManager groupMetadataManager + ) { + this.snapshotRegistry = snapshotRegistry; + this.groupMetadataManager = groupMetadataManager; + } + + public void commit() { + long lastCommittedOffset = this.lastCommittedOffset; + this.lastCommittedOffset = lastWrittenOffset; + snapshotRegistry.deleteSnapshotsUpTo(lastCommittedOffset); + } + + public void rollback() { + lastWrittenOffset = lastCommittedOffset; + snapshotRegistry.revertToSnapshot(lastCommittedOffset); + } + + public ConsumerGroup.ConsumerGroupState consumerGroupState( + String groupId + ) { + return groupMetadataManager + .getOrMaybeCreateConsumerGroup(groupId, false) + .state(); + } + + public ConsumerGroupMember.MemberState consumerGroupMemberState( + String groupId, + String memberId + ) { + return groupMetadataManager + .getOrMaybeCreateConsumerGroup(groupId, false) + .getOrMaybeCreateMember(memberId, false) + .state(); + } + + public Result consumerGroupHeartbeat( + ConsumerGroupHeartbeatRequestData request + ) { + snapshotRegistry.getOrCreateSnapshot(lastCommittedOffset); + + RequestContext context = new RequestContext( + new RequestHeader( + ApiKeys.CONSUMER_GROUP_HEARTBEAT, + ApiKeys.CONSUMER_GROUP_HEARTBEAT.latestVersion(), + "client", + 0 + ), + "1", + InetAddress.getLoopbackAddress(), + KafkaPrincipal.ANONYMOUS, + ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), + SecurityProtocol.PLAINTEXT, + ClientInformation.EMPTY, + false + ); + + Result result = groupMetadataManager.consumerGroupHeartbeat( + context, + request + ); + + result.records().forEach(this::replay); + return result; + } + + private ApiMessage messageOrNull(ApiMessageAndVersion apiMessageAndVersion) { + if (apiMessageAndVersion == null) { + return null; + } else { + return apiMessageAndVersion.message(); + } + } + + private void replay( + Record record + ) { + ApiMessageAndVersion key = record.key(); + ApiMessageAndVersion value = record.value(); + + if (key == null) { + throw new IllegalStateException("Received a null key in " + record); + } + + switch (key.version()) { + case ConsumerGroupMemberMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupMemberMetadataKey) key.message(), + (ConsumerGroupMemberMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupMetadataKey) key.message(), + (ConsumerGroupMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupPartitionMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupPartitionMetadataKey) key.message(), + (ConsumerGroupPartitionMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupTargetAssignmentMemberKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupTargetAssignmentMemberKey) key.message(), + (ConsumerGroupTargetAssignmentMemberValue) messageOrNull(value) + ); + break; + + case ConsumerGroupTargetAssignmentMetadataKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupTargetAssignmentMetadataKey) key.message(), + (ConsumerGroupTargetAssignmentMetadataValue) messageOrNull(value) + ); + break; + + case ConsumerGroupCurrentMemberAssignmentKey.HIGHEST_SUPPORTED_VERSION: + groupMetadataManager.replay( + (ConsumerGroupCurrentMemberAssignmentKey) key.message(), + (ConsumerGroupCurrentMemberAssignmentValue) messageOrNull(value) + ); + break; + + default: + throw new IllegalStateException("Received an unknown record type " + key.version() + + " in " + record); + } + + lastWrittenOffset++; + } + } + + @Test + public void testConsumerHeartbeatRequestValidation() { + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + Exception ex; + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData())); + assertEquals("GroupId can't be empty.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0))); + assertEquals("RebalanceTimeoutMs must be provided in first request.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000))); + assertEquals("TopicPartitions must be empty when (re-)joining.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setTopicPartitions(Collections.emptyList()))); + assertEquals("SubscribedTopicNames must be set in first request.", ex.getMessage()); + + ex = assertThrows(UnsupportedAssignorException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setTopicPartitions(Collections.emptyList()) + .setSubscribedTopicNames(Collections.singletonList("foo")) + .setServerAssignor("bar"))); + assertEquals("ServerAssignor bar is not supported. Supported assignors: range.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberEpoch(1))); + assertEquals("MemberId can't be empty.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setInstanceId("instance-id"))); + assertEquals("InstanceId should only be provided in first request.", ex.getMessage()); + + ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setRackId("rack-id"))); + assertEquals("RackId should only be provided in first request.", ex.getMessage()); + } + + @Test + public void testMemberIdGeneration() { + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(TopicsImage.EMPTY) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + Collections.emptyMap() + )); + + Result result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("group-foo") + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + // Verify that a member id was generated for the new member. + String memberId = result.response().memberId(); + assertNotNull(memberId); + assertNotEquals("", memberId); + + // The response should get a bumped epoch and should not + // contain any assignment because we did not provide + // topics metadata. + assertEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment()), + result.response() + ); + } + + @Test + public void testUnknownGroupId() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + + assertThrows(GroupIdNotFoundException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(100) // Epoch must be > 0. + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList()))); + } + + @Test + public void testUnknownMemberIdJoinsConsumerGroup() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment(Collections.emptyMap())); + + // A first member joins to create the group. + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + // The second member is rejected because the member id is unknown and + // the member epoch is not zero. + assertThrows(UnknownMemberIdException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList()))); + } + + @Test + public void testConsumerGroupMemberEpochValidation() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + Uuid fooTopicId = Uuid.randomUuid(); + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .build(); + + ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(100) + .setPreviousMemberEpoch(99) + .setNextMemberEpoch(100) + .setRebalanceTimeoutMs(5000) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2, 3))) + .build(); + + context.replay(RecordHelpers.newMemberSubscriptionRecord(groupId, member)); + + context.replay(RecordHelpers.newGroupEpochRecord(groupId, 100)); + + context.replay(RecordHelpers.newTargetAssignmentRecord(groupId, memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3) + ))); + + context.replay(RecordHelpers.newTargetAssignmentEpochRecord(groupId, 100)); + + context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, member)); + + // Member epoch is greater than the expected epoch. + assertThrows(NotCoordinatorException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(200) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); + + // Member epoch is smaller than the expected epoch. + assertThrows(FencedMemberEpochException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(50) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); + + // Member joins with previous epoch but without providing partitions. + assertThrows(FencedMemberEpochException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); + + // Member joins with previous epoch and has a subset of the owned partitions. 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. + Result result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.singletonList(new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(1, 2))))); + assertEquals(100, result.response().memberEpoch()); + } + + @Test + public void testMemberJoinsEmptyConsumerGroup() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + Collections.singletonMap(memberId, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + ))) + )); + + assertThrows(GroupIdNotFoundException.class, () -> + context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false)); + + Result result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(1) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1, 2, 3, 4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(0, 1, 2)) + ))), + result.response() + ); + + ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(1) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(1) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2))) + .build(); + + List expectedRecords = Arrays.asList( + RecordHelpers.newMemberSubscriptionRecord(groupId, expectedMember), + RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() {{ + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)); + put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)); + }}), + RecordHelpers.newGroupEpochRecord(groupId, 1), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + )), + RecordHelpers.newTargetAssignmentEpochRecord(groupId, 1), + RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertEquals(expectedRecords, result.records()); + } + + @Test + public void testUpdatingSubscriptionTriggersNewTargetAssignment() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5))) + .build()) + .withAssignment(memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5))) + .withAssignmentEpoch(10)) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + Collections.singletonMap(memberId, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + ))) + )); + + Result result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(10) + .setSubscribedTopicNames(Arrays.asList("foo", "bar"))); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1, 2, 3, 4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(0, 1, 2)) + ))), + result.response() + ); + + ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setNextMemberEpoch(11) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2))) + .build(); + + List expectedRecords = Arrays.asList( + RecordHelpers.newMemberSubscriptionRecord(groupId, expectedMember), + RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() {{ + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)); + put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)); + }}), + RecordHelpers.newGroupEpochRecord(groupId, 11), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + )), + RecordHelpers.newTargetAssignmentEpochRecord(groupId, 11), + RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember) + ); + + assertEquals(expectedRecords, result.records()); + } + + @Test + public void testNewJoiningMemberTriggersNewTargetAssignment() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .build()) + .withMember(new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .withAssignment(memberId2, mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .withAssignmentEpoch(10)) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + new HashMap() {{ + put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1), + mkTopicAssignment(barTopicId, 0) + ))); + put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2, 3), + mkTopicAssignment(barTopicId, 1) + ))); + put(memberId3, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5), + mkTopicAssignment(barTopicId, 2) + ))); + }} + )); + + // Member 3 joins the consumer group. + Result result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setPendingTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(2)) + ))), + result.response() + ); + + ConsumerGroupMember expectedMember3 = new ConsumerGroupMember.Builder(memberId3) + .setMemberEpoch(11) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(11) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .build(); + + List expectedRecords = Arrays.asList( + RecordHelpers.newMemberSubscriptionRecord(groupId, expectedMember3), + RecordHelpers.newGroupEpochRecord(groupId, 11), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1), + mkTopicAssignment(barTopicId, 0) + )), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId2, mkAssignment( + mkTopicAssignment(fooTopicId, 2, 3), + mkTopicAssignment(barTopicId, 1) + )), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId3, mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5), + mkTopicAssignment(barTopicId, 2) + )), + RecordHelpers.newTargetAssignmentEpochRecord(groupId, 11), + RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember3) + ); + + assertEquals(expectedRecords.subList(0, 2), result.records().subList(0, 2)); + assertUnorderedListEquals(expectedRecords.subList(2, 5), result.records().subList(2, 5)); + assertEquals(expectedRecords.subList(5, 7), result.records().subList(5, 7)); + } + + @Test + public void testLeavingMemberTriggersNewTargetAssignment() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + Uuid zarTopicId = Uuid.randomUuid(); + String zarTopicName = "zar"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + + // Consumer group with two members. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .addTopic(zarTopicId, zarTopicName, 1) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .build()) + .withMember(new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + // Use zar only here to ensure that metadata needs to be recomputed. + .setSubscribedTopicNames(Arrays.asList("foo", "bar", "zar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .withAssignment(memberId2, mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .withAssignmentEpoch(10)) + .build(); + + assignor.prepareGroupAssignment(new GroupAssignment( + Collections.singletonMap(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + ))) + )); + + // Member 3 leaves the consumer group. + Result result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(-1) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(-1), + result.response() + ); + + List expectedRecords = Arrays.asList( + RecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId2), + RecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId2), + RecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId2), + // Subscription metadata is recomputed because zar is no longer there. + RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() { + { + put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)); + put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)); + } + }), + RecordHelpers.newGroupEpochRecord(groupId, 11), + RecordHelpers.newTargetAssignmentRecord(groupId, memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), + mkTopicAssignment(barTopicId, 0, 1, 2) + )), + RecordHelpers.newTargetAssignmentEpochRecord(groupId, 11) + ); + + assertEquals(expectedRecords, result.records()); + } + + @Test + public void testReconciliationProcess() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + // Create a context with one consumer group containing two members. + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .build()) + .withMember(new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .withAssignment(memberId2, mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .withAssignmentEpoch(10)) + .build(); + + // Prepare new assignment for the group. + assignor.prepareGroupAssignment(new GroupAssignment( + new HashMap() { + { + put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1), + mkTopicAssignment(barTopicId, 0) + ))); + put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2, 3), + mkTopicAssignment(barTopicId, 2) + ))); + put(memberId3, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5), + mkTopicAssignment(barTopicId, 1) + ))); + } + } + )); + + Result result; + + // Members in the group are in Stable state. + assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId1)); + assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, context.consumerGroupState(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 partitions yet because they are + // all owned by other members. However, it transitions to epoch 11 / Assigning state. + result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setPendingTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(1)) + ))), + result.response() + ); + + // We only check the last record as the subscription/target assignment updates are + // already covered by other tests. + assertEquals( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setMemberEpoch(11) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(11) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5), + mkTopicAssignment(barTopicId, 1))) + .build()), + result.records().get(result.records().size() - 1) + ); + + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 1 heartbeats. It remains at epoch 10 but transitions to Revoking state until + // it acknowledges the revocation of its partitions. The response contains the new + // assignment without the partitions that must be revoked. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(10)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(0)) + ))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1), + mkTopicAssignment(barTopicId, 0))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(fooTopicId, 2), + mkTopicAssignment(barTopicId, 1))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId1)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 2 heartbeats. It remains at epoch 10 but transitions to Revoking state until + // it acknowledges the revocation of its partitions. The response contains the new + // assignment without the partitions that must be revoked. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(3)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(2)) + ))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3), + mkTopicAssignment(barTopicId, 2))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5))) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 3 heartbeats. The response does not contain any assignment + // because the member is still waiting on other members to revoke partitions. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000), + result.response() + ); + + assertEquals(Collections.emptyList(), result.records()); + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 1 acknowledges the revocation of the partitions. It does so by providing the + // partitions that it still owns in the request. This allows him to transition to epoch 11 + // and to the Stable state. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(10) + .setTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1)), + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(0)) + ))); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(0)) + ))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1), + mkTopicAssignment(barTopicId, 0))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId1)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 2 heartbeats but without acknowledging the revocation yet. This is basically a no-op. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000), + result.response() + ); + + assertEquals(Collections.emptyList(), result.records()); + assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 3 heartbeats. It receives the partitions revoked by member 1 but remains + // in Assigning state because it still waits on other partitions. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(1)))) + .setPendingTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(4, 5))))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(barTopicId, 1))) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 4, 5))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 3 heartbeats but without acknowledging the revocation yet. This is basically a no-op. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000), + result.response() + ); + + assertEquals(Collections.emptyList(), result.records()); + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 2 acknowledges the revocation of the partitions. It does so by providing the + // partitions that it still owns in the request. This allows him to transition to epoch 11 + // and to the Stable state. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10) + .setTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(3)), + new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(2)) + ))); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(2, 3)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(2)) + ))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 2, 3), + mkTopicAssignment(barTopicId, 2))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId2)); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + // Member 3 heartbeats. It receives all its partitions and transitions to Stable. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(11)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(1)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(4, 5))))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setMemberEpoch(11) + .setPreviousMemberEpoch(11) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(barTopicId, 1), + mkTopicAssignment(fooTopicId, 4, 5))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.STABLE, context.consumerGroupMemberState(groupId, memberId3)); + assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, context.consumerGroupState(groupId)); + } + + @Test + public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + + // Create a context with one consumer group containing one member. + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2))) + .withAssignmentEpoch(10)) + .build(); + + Result result; + + // Prepare new assignment for the group. + assignor.prepareGroupAssignment(new GroupAssignment( + new HashMap() { + { + put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1) + ))); + put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2) + ))); + } + } + )); + + // Member 2 joins. + result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(11) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setPendingTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(2)) + ))), + result.response() + ); + + assertEquals( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(11) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(11) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2))) + .build()), + result.records().get(result.records().size() - 1) + ); + + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId2)); + + // Member 1 heartbeats and transitions to Revoking. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(10)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0, 1))))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(fooTopicId, 2))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId1)); + + // Prepare new assignment for the group. + assignor.prepareGroupAssignment(new GroupAssignment( + new HashMap() { + { + put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0) + ))); + put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2) + ))); + put(memberId3, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 1) + ))); + } + } + )); + + // Member 3 joins. + result = context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList())); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId3) + .setMemberEpoch(12) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setPendingTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(1)) + ))), + result.response() + ); + + assertEquals( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) + .setMemberEpoch(12) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(12) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 1))) + .build()), + result.records().get(result.records().size() - 1) + ); + + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); + + // When member 1 heartbeats, it transitions to Revoke again but an updated state. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(10)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(10) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setAssignedTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(0))))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(12) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.REVOKING, context.consumerGroupMemberState(groupId, memberId1)); + + // When member 2 heartbeats, it transitions to Assign again but with an updated state. + result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(11)); + + assertResponseEquals( + new ConsumerGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(12) + .setHeartbeatIntervalMs(5000) + .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() + .setPendingTopicPartitions(Arrays.asList( + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(2))))), + result.response() + ); + + assertEquals(Collections.singletonList( + RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(12) + .setPreviousMemberEpoch(11) + .setNextMemberEpoch(12) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 2))) + .build())), + result.records() + ); + + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId2)); + } + + @Test + public void testNewMemberIsRejectedWithMaximumMembersIsReached() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String memberId3 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + // Create a context with one consumer group containing two members. + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withConsumerGroupMaxSize(2) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .build()) + .withMember(new ConsumerGroupMember.Builder(memberId2) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .withAssignment(memberId2, mkAssignment( + mkTopicAssignment(fooTopicId, 3, 4, 5), + mkTopicAssignment(barTopicId, 2))) + .withAssignmentEpoch(10)) + .build(); + + assertThrows(GroupMaxSizeReachedException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId3) + .setMemberEpoch(0) + .setServerAssignor("range") + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList()))); + } + + @Test + public void testConsumerGroupStates() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + + MockPartitionAssignor assignor = new MockPartitionAssignor("range"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)) + .build(); + + assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, context.consumerGroupState(groupId)); + + context.replay(RecordHelpers.newMemberSubscriptionRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setSubscribedTopicNames(Collections.singletonList(fooTopicName)) + .build())); + context.replay(RecordHelpers.newGroupEpochRecord(groupId, 11)); + + assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, context.consumerGroupState(groupId)); + + context.replay(RecordHelpers.newTargetAssignmentRecord(groupId, memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3)))); + context.replay(RecordHelpers.newTargetAssignmentEpochRecord(groupId, 11)); + + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2))) + .setPartitionsPendingAssignment(mkAssignment(mkTopicAssignment(fooTopicId, 3))) + .build())); + + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); + + context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(11) + .setPreviousMemberEpoch(10) + .setNextMemberEpoch(11) + .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2, 3))) + .build())); + + assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, context.consumerGroupState(groupId)); + } + + @Test + public void testPartitionAssignorExceptionOnRegularHeartbeat() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + PartitionAssignor assignor = mock(PartitionAssignor.class); + when(assignor.name()).thenReturn("range"); + when(assignor.assign(any())).thenThrow(new PartitionAssignorException("Assignment failed.")); + + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .build(); + + // Member 1 joins the consumer group. The request fails because the + // target assignment computation failed. + assertThrows(UnknownServerException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(0) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignor("range") + .setTopicPartitions(Collections.emptyList()))); + } + + @Test + public void testPartitionAssignorExceptionOnLeaveHeatbeat() { + String groupId = "fooup"; + // Use a static member id as it makes the test easier. + String memberId1 = Uuid.randomUuid().toString(); + + Uuid fooTopicId = Uuid.randomUuid(); + String fooTopicName = "foo"; + Uuid barTopicId = Uuid.randomUuid(); + String barTopicName = "bar"; + + PartitionAssignor assignor = mock(PartitionAssignor.class); + when(assignor.assign(any())).thenThrow(new PartitionAssignorException("Assignment failed.")); + + // Consumer group with two members. + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withAssignors(Collections.singletonList(assignor)) + .withTopicsImage(new TopicsImageBuilder() + .addTopic(fooTopicId, fooTopicName, 6) + .addTopic(barTopicId, barTopicName, 3) + .build()) + .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) + .withMember(new ConsumerGroupMember.Builder(memberId1) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .setNextMemberEpoch(10) + .setClientId("client") + .setClientHost("localhost/127.0.0.1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setServerAssignorName("range") + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .build()) + .withAssignment(memberId1, mkAssignment( + mkTopicAssignment(fooTopicId, 0, 1, 2), + mkTopicAssignment(barTopicId, 0, 1))) + .withAssignmentEpoch(10)) + .build(); + + // Member 3 leaves the consumer group. + assertThrows(UnknownServerException.class, () -> + context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(-1) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.emptyList()))); + } + + private void assertUnorderedListEquals( + List expected, + List actual + ) { + assertEquals(new HashSet<>(expected), new HashSet<>(actual)); + } + + private void assertResponseEquals( + ConsumerGroupHeartbeatResponseData expected, + ConsumerGroupHeartbeatResponseData actual + ) { + if (!responseEquals(expected, actual)) { + assertionFailure() + .expected(expected) + .actual(actual) + .buildAndThrow(); + } + } + + private boolean responseEquals( + ConsumerGroupHeartbeatResponseData expected, + ConsumerGroupHeartbeatResponseData actual + ) { + if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false; + if (expected.errorCode() != actual.errorCode()) return false; + if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false; + if (!Objects.equals(expected.memberId(), actual.memberId())) return false; + if (expected.memberEpoch() != actual.memberEpoch()) return false; + if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false; + if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false; + // Unordered comparison of the assignments. + return responseAssignmentEquals(expected.assignment(), actual.assignment()); + } + + private boolean responseAssignmentEquals( + ConsumerGroupHeartbeatResponseData.Assignment expected, + ConsumerGroupHeartbeatResponseData.Assignment actual + ) { + if (expected == actual) return true; + if (expected == null) return false; + if (actual == null) return false; + + if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions()))) + return false; + + return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions())); + } + + private Map> fromAssignment( + List assignment + ) { + if (assignment == null) return null; + + Map> assigmentMap = new HashMap<>(); + assignment.forEach(topicPartitions -> { + assigmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions())); + }); + return assigmentMap; + } +} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java new file mode 100644 index 0000000000000..287095c770fd1 --- /dev/null +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -0,0 +1,326 @@ +/* + * 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.consumer; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.UnknownMemberIdException; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.coordinator.group.GroupMetadataManagerTest; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.timeline.SnapshotRegistry; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment; +import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ConsumerGroupTest { + + private ConsumerGroup createConsumerGroup(String groupId) { + SnapshotRegistry snapshotRegistry = new SnapshotRegistry(new LogContext()); + return new ConsumerGroup(snapshotRegistry, groupId); + } + + @Test + public void testGetOrCreateMember() { + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + ConsumerGroupMember member; + + // Create a group. + member = consumerGroup.getOrMaybeCreateMember("member-id", true); + assertEquals("member-id", member.memberId()); + + // Get that group back. + member = consumerGroup.getOrMaybeCreateMember("member-id", false); + assertEquals("member-id", member.memberId()); + + assertThrows(UnknownMemberIdException.class, () -> + consumerGroup.getOrMaybeCreateMember("does-not-exist", false)); + } + + @Test + public void testUpdateMember() { + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + ConsumerGroupMember member; + + member = consumerGroup.getOrMaybeCreateMember("member", true); + + member = new ConsumerGroupMember.Builder(member) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .build(); + + consumerGroup.updateMember(member); + + assertEquals(member, consumerGroup.getOrMaybeCreateMember("member", false)); + } + + @Test + public void testRemoveMember() { + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + + consumerGroup.getOrMaybeCreateMember("member", true); + assertTrue(consumerGroup.hasMember("member")); + + consumerGroup.removeMember("member"); + assertFalse(consumerGroup.hasMember("member")); + + } + + @Test + public void testUpdatingMemberUpdatesPartitionEpoch() { + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + Uuid zarTopicId = Uuid.randomUuid(); + + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + ConsumerGroupMember member; + + member = new ConsumerGroupMember.Builder("member") + .setMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(barTopicId, 4, 5, 6))) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(zarTopicId, 7, 8, 9))) + .build(); + + consumerGroup.updateMember(member); + + assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 1)); + assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 2)); + assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 3)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 4)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 5)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 6)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 7)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 8)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 9)); + + member = new ConsumerGroupMember.Builder(member) + .setMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(barTopicId, 1, 2, 3))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(zarTopicId, 4, 5, 6))) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 7, 8, 9))) + .build(); + + consumerGroup.updateMember(member); + + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 1)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 2)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 3)); + assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 4)); + assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 5)); + assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 6)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 7)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 8)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 9)); + } + + @Test + public void testDeletingMemberRemovesPartitionEpoch() { + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + Uuid zarTopicId = Uuid.randomUuid(); + + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + ConsumerGroupMember member; + + member = new ConsumerGroupMember.Builder("member") + .setMemberEpoch(10) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 1, 2, 3))) + .setPartitionsPendingRevocation(mkAssignment( + mkTopicAssignment(barTopicId, 4, 5, 6))) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(zarTopicId, 7, 8, 9))) + .build(); + + consumerGroup.updateMember(member); + + assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 1)); + assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 2)); + assertEquals(10, consumerGroup.currentPartitionEpoch(fooTopicId, 3)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 4)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 5)); + assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 6)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 7)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 8)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 9)); + + consumerGroup.removeMember(member.memberId()); + + assertEquals(-1, consumerGroup.currentPartitionEpoch(barTopicId, 1)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(barTopicId, 2)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(barTopicId, 3)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 4)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 5)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 6)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 7)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 8)); + assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 9)); + } + + @Test + public void testGroupState() { + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, consumerGroup.state()); + + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") + .setMemberEpoch(1) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(1) + .build(); + + consumerGroup.updateMember(member1); + consumerGroup.setGroupEpoch(1); + + assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state()); + + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") + .setMemberEpoch(1) + .setPreviousMemberEpoch(0) + .setNextMemberEpoch(1) + .build(); + + consumerGroup.updateMember(member2); + consumerGroup.setGroupEpoch(2); + + assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state()); + + consumerGroup.setAssignmentEpoch(2); + + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); + + member1 = new ConsumerGroupMember.Builder(member1) + .setMemberEpoch(2) + .setPreviousMemberEpoch(1) + .setNextMemberEpoch(2) + .build(); + consumerGroup.updateMember(member1); + + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); + + member2 = new ConsumerGroupMember.Builder(member2) + .setMemberEpoch(2) + .setPreviousMemberEpoch(1) + .setNextMemberEpoch(2) + .build(); + consumerGroup.updateMember(member2); + + assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, consumerGroup.state()); + + consumerGroup.removeMember("member1"); + consumerGroup.removeMember("member2"); + + assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, consumerGroup.state()); + } + + @Test + public void testPreferredServerAssignor() { + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member1") + .setServerAssignorName("range") + .build()); + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member2") + .setServerAssignorName("range") + .build()); + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member3") + .setServerAssignorName("uniform") + .build()); + + assertEquals(Optional.of("range"), consumerGroup.preferredServerAssignor( + null, + Optional.empty()) + ); + + assertEquals(Optional.of("uniform"), consumerGroup.preferredServerAssignor( + "member2", + Optional.of("uniform")) + ); + + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member1") + .setServerAssignorName(null) + .build()); + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member2") + .setServerAssignorName(null) + .build()); + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member3") + .setServerAssignorName(null) + .build()); + + assertEquals(Optional.empty(), consumerGroup.preferredServerAssignor( + null, + Optional.empty()) + ); + } + + @Test + public void testUpdateSubscriptionMetadata() { + Uuid fooTopicId = Uuid.randomUuid(); + Uuid barTopicId = Uuid.randomUuid(); + Uuid zarTopicId = Uuid.randomUuid(); + + TopicsImage image = new GroupMetadataManagerTest.TopicsImageBuilder() + .addTopic(fooTopicId, "foo", 1) + .addTopic(barTopicId, "bar", 2) + .addTopic(zarTopicId, "zar", 3) + .build(); + + ConsumerGroup consumerGroup = createConsumerGroup("foo"); + + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member1") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .build()); + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member2") + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .build()); + consumerGroup.updateMember(new ConsumerGroupMember.Builder("member3") + .setSubscribedTopicNames(Arrays.asList("bar", "zar")) + .build()); + + Map expectedSubscriptionMetadata = new HashMap<>(); + expectedSubscriptionMetadata.put("foo", new TopicMetadata(fooTopicId, "foo", 1)); + expectedSubscriptionMetadata.put("bar", new TopicMetadata(barTopicId, "bar", 2)); + expectedSubscriptionMetadata.put("zar", new TopicMetadata(zarTopicId, "zar", 3)); + assertEquals(expectedSubscriptionMetadata, consumerGroup.computeSubscriptionMetadata( + null, + Collections.emptyList(), + image + )); + + expectedSubscriptionMetadata.remove("zar"); + assertEquals(expectedSubscriptionMetadata, consumerGroup.computeSubscriptionMetadata( + "member3", + Collections.emptyList(), + image + )); + } +} From 61079c20852a28b2364a9efb0b34008302ad54a1 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 12 May 2023 13:43:32 +0200 Subject: [PATCH 02/16] address minor comments --- .../group/GroupMetadataManager.java | 25 +++++++++++-------- .../group/GroupMetadataManagerTest.java | 22 +++++++++------- 2 files changed, 27 insertions(+), 20 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 976e381bdf85a..e61a0aef68688 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 @@ -153,7 +153,7 @@ GroupMetadataManager build() { private final SnapshotRegistry snapshotRegistry; /** - * The list of supported assignors. + * The supported partition assignors keyed by their name. */ private final Map assignors; @@ -283,15 +283,16 @@ private void throwIfConsumerGroupHeartbeatRequestIsInvalid( if (request.subscribedTopicNames() == null || request.subscribedTopicNames().isEmpty()) { throw new InvalidRequestException("SubscribedTopicNames must be set in first request."); } - if (request.serverAssignor() != null && !assignors.containsKey(request.serverAssignor())) { - throw new UnsupportedAssignorException("ServerAssignor " + request.serverAssignor() - + " is not supported. Supported assignors: " + String.join(", ", assignors.keySet()) - + "."); - } } else { throw new InvalidRequestException("MemberEpoch is invalid."); } + if (request.serverAssignor() != null && !assignors.containsKey(request.serverAssignor())) { + throw new UnsupportedAssignorException("ServerAssignor " + request.serverAssignor() + + " is not supported. Supported assignors: " + String.join(", ", assignors.keySet()) + + "."); + } + if (request.subscribedTopicRegex() != null) { throw new InvalidRequestException("SubscribedTopicRegex is not supported yet."); } @@ -303,11 +304,12 @@ private void throwIfConsumerGroupHeartbeatRequestIsInvalid( /** * 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 client - * has at least of subset of them. + * request) matches the ones that the member should own. It matches if the consumer + * only owns partitions which are in the assigned partitions. If does not match if + * it owns any other partitions. * * @param ownedTopicPartitions The partitions provided by the consumer in the request. - * @param target The partitions that they member should have. + * @param target The partitions that the member should have. * * @return A boolean indicating whether the owned partitions are a subset or not. */ @@ -370,7 +372,7 @@ private void throwIfMemberEpochIsInvalid( if (memberEpoch > member.memberEpoch()) { // The member has likely got a bump from another coordinator and this coordinator // is stale. Return NOT_COORDINATOR to force the member to refresh its coordinator. - throw new NotCoordinatorException("The consumer group member has got a larger member " + throw new NotCoordinatorException("The consumer group member has a larger member " + "epoch (" + memberEpoch + ") than the one known by this group coordinator (" + member.memberEpoch() + ")."); } else if (memberEpoch < member.memberEpoch()) { @@ -536,7 +538,8 @@ private Result consumerGroupHeartbeat( } // If the member is stable and its next epoch matches the current target epoch - // of the assignment, we can skip this reconciliation. + // of the assignment, we don't have to update its current assignment. Otherwise, + // we reconcile its current state based on the target assignment. boolean assignmentUpdated = false; if (member.state() != ConsumerGroupMember.MemberState.STABLE || member.nextMemberEpoch() != targetAssignmentEpoch) { 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 de1ad2d1d5dac..2c233de630282 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 @@ -496,6 +496,14 @@ public void testConsumerHeartbeatRequestValidation() { .setMemberEpoch(1) .setRackId("rack-id"))); assertEquals("RackId should only be provided in first request.", ex.getMessage()); + + ex = assertThrows(UnsupportedAssignorException.class, () -> context.consumerGroupHeartbeat( + new ConsumerGroupHeartbeatRequestData() + .setGroupId("foo") + .setMemberId(Uuid.randomUuid().toString()) + .setMemberEpoch(1) + .setServerAssignor("bar"))); + assertEquals("ServerAssignor bar is not supported. Supported assignors: range.", ex.getMessage()); } @Test @@ -554,7 +562,6 @@ public void testUnknownGroupId() { .setGroupId(groupId) .setMemberId(memberId) .setMemberEpoch(100) // Epoch must be > 0. - .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setTopicPartitions(Collections.emptyList()))); @@ -592,7 +599,6 @@ public void testUnknownMemberIdJoinsConsumerGroup() { .setGroupId(groupId) .setMemberId(Uuid.randomUuid().toString()) .setMemberEpoch(1) - .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setTopicPartitions(Collections.emptyList()))); @@ -641,7 +647,6 @@ public void testConsumerGroupMemberEpochValidation() { .setGroupId(groupId) .setMemberId(memberId) .setMemberEpoch(200) - .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); @@ -652,7 +657,6 @@ public void testConsumerGroupMemberEpochValidation() { .setGroupId(groupId) .setMemberId(memberId) .setMemberEpoch(50) - .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); @@ -663,7 +667,6 @@ public void testConsumerGroupMemberEpochValidation() { .setGroupId(groupId) .setMemberId(memberId) .setMemberEpoch(99) - .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")))); @@ -675,7 +678,6 @@ public void testConsumerGroupMemberEpochValidation() { .setGroupId(groupId) .setMemberId(memberId) .setMemberEpoch(99) - .setServerAssignor("range") .setRebalanceTimeoutMs(5000) .setSubscribedTopicNames(Arrays.asList("foo", "bar")) .setTopicPartitions(Collections.singletonList(new ConsumerGroupHeartbeatRequestData.TopicPartitions() @@ -927,7 +929,8 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() {{ + new HashMap() { + { put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), mkTopicAssignment(barTopicId, 0) @@ -940,7 +943,8 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { mkTopicAssignment(fooTopicId, 4, 5), mkTopicAssignment(barTopicId, 2) ))); - }} + } + } )); // Member 3 joins the consumer group. @@ -1963,7 +1967,7 @@ public void testPartitionAssignorExceptionOnRegularHeartbeat() { } @Test - public void testPartitionAssignorExceptionOnLeaveHeatbeat() { + public void testPartitionAssignorExceptionOnLeaveHeartbeat() { String groupId = "fooup"; // Use a static member id as it makes the test easier. String memberId1 = Uuid.randomUuid().toString(); From eec323d92cd36585bcceffa936d0346c926ec64c Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 22 May 2023 11:26:24 +0200 Subject: [PATCH 03/16] address minor comments --- .../group/GroupMetadataManager.java | 73 ++++++++++++------- .../group/consumer/ConsumerGroup.java | 32 +++++--- 2 files changed, 68 insertions(+), 37 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 e61a0aef68688..b48c53a91b9da 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 @@ -210,8 +210,9 @@ private GroupMetadataManager( * @return A ConsumerGroup. * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or * if the group is not a consumer group. + * + * Package private for testing. */ - // Package private for testing. ConsumerGroup getOrMaybeCreateConsumerGroup( String groupId, boolean createIfNotExists @@ -305,7 +306,7 @@ private void throwIfConsumerGroupHeartbeatRequestIsInvalid( /** * 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 - * only owns partitions which are in the assigned partitions. If does not match if + * only owns partitions which are in the assigned partitions. It does not match if * it owns any other partitions. * * @param ownedTopicPartitions The partitions provided by the consumer in the request. @@ -355,7 +356,7 @@ private void throwIfConsumerGroupIsFull( * Validates the member epoch provided in the heartbeat request. * * @param member The consumer group member. - * @param memberEpoch The member epoch. + * @param receivedMemberEpoch The member epoch. * @param ownedTopicPartitions The owned partitions. * * @throws NotCoordinatorException if the provided epoch is ahead of the epoch known @@ -366,19 +367,19 @@ private void throwIfConsumerGroupIsFull( */ private void throwIfMemberEpochIsInvalid( ConsumerGroupMember member, - int memberEpoch, + int receivedMemberEpoch, List ownedTopicPartitions ) { - if (memberEpoch > member.memberEpoch()) { + if (receivedMemberEpoch > member.memberEpoch()) { // The member has likely got a bump from another coordinator and this coordinator // is stale. Return NOT_COORDINATOR to force the member to refresh its coordinator. throw new NotCoordinatorException("The consumer group member has a larger member " - + "epoch (" + memberEpoch + ") than the one known by this group coordinator (" + + "epoch (" + receivedMemberEpoch + ") than the one known by this group coordinator (" + member.memberEpoch() + ")."); - } else if (memberEpoch < member.memberEpoch()) { + } 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 (memberEpoch != member.previousMemberEpoch() || !isSubset(ownedTopicPartitions, member.assignedPartitions())) { + if (receivedMemberEpoch != member.previousMemberEpoch() || !isSubset(ownedTopicPartitions, member.assignedPartitions())) { throw new FencedMemberEpochException("The consumer group member has an old member " + "epoch. The member must abandon all its partitions and rejoin."); } @@ -413,7 +414,13 @@ private OptionalInt ofSentinel(int value) { } /** - * Handles a regular heartbeat from a consumer group member. + * Handles a regular heartbeat from a consumer group member. It mainly consists of + * three parts: + * 1) The member is created or updated. The group epoch is bumped if the member + * has been created or updated. + * 2) The target assignment for the consumer group is updated if the group epoch + * is larger than the current target assignment epoch. + * 3) The member's assignment is reconciled with the target assignment. * * @param groupId The group id from the request. * @param memberId The member id from the request. @@ -448,11 +455,13 @@ private Result consumerGroupHeartbeat( List ownedTopicPartitions ) throws ApiException { List records = new ArrayList<>(); - boolean createIfNotExists = memberEpoch == 0; + // Get or create the consumer group. + boolean createIfNotExists = memberEpoch == 0; ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, createIfNotExists); throwIfConsumerGroupIsFull(group, memberId); + // Get or create the member. if (memberId.isEmpty()) memberId = Uuid.randomUuid().toString(); ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, createIfNotExists); throwIfMemberEpochIsInvalid(member, memberEpoch, ownedTopicPartitions); @@ -461,9 +470,11 @@ private Result consumerGroupHeartbeat( log.info("[GroupId " + groupId + "] Member " + memberId + " re-joins the consumer group."); } - // Update the subscription part of the member if we received new values. If the member has - // changed, we write it to the log. If the subscribed topics have changed, we also recompute - // the subscription metadata. + // 1. Create or update the member. If the member is new or has changed, a ConsumerGroupMemberMetadataValue + // 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 ConsumerGroupPartitionMetadataValue + // record to the __consumer_offsets partition. Finally, the group epoch is bumped if the subscriptions have + // changed, and persisted by writing a ConsumerGroupMetadataValue record to the partition. int groupEpoch = group.groupEpoch(); Map subscriptionMetadata = group.subscriptionMetadata(); ConsumerGroupMember updatedMember = new ConsumerGroupMember.Builder(member) @@ -505,8 +516,8 @@ private Result consumerGroupHeartbeat( member = updatedMember; } - // Update target assignment if needed. If the new target has any changes, we write the - // changes to the log. + // 2. Update the target assignment if the group epoch is larger than the target assignment epoch. The + // delta between the current and the new target assignment is persisted to the partition. int targetAssignmentEpoch = group.assignmentEpoch(); Assignment targetAssignment = group.targetAssignment(memberId); if (groupEpoch > targetAssignmentEpoch) { @@ -537,12 +548,10 @@ private Result consumerGroupHeartbeat( } } - // If the member is stable and its next epoch matches the current target epoch - // of the assignment, we don't have to update its current assignment. Otherwise, - // we reconcile its current state based on the target assignment. + // 3. Reconcile the member's assignment with the target assignment. This is only required if + // the member is not stable or if a new target assignment has been installed. boolean assignmentUpdated = false; - if (member.state() != ConsumerGroupMember.MemberState.STABLE - || member.nextMemberEpoch() != targetAssignmentEpoch) { + if (member.state() != ConsumerGroupMember.MemberState.STABLE || member.nextMemberEpoch() != targetAssignmentEpoch) { updatedMember = new CurrentAssignmentBuilder(member) .withTargetAssignment(targetAssignmentEpoch, targetAssignment) .withCurrentPartitionEpoch(group::currentPartitionEpoch) @@ -566,11 +575,16 @@ private Result consumerGroupHeartbeat( // TODO(dajac) Starts or restarts the timer for the session timeout. + // Prepare the response. ConsumerGroupHeartbeatResponseData response = new ConsumerGroupHeartbeatResponseData() .setMemberId(member.memberId()) .setMemberEpoch(member.memberEpoch()) .setHeartbeatIntervalMs(consumerGroupHeartbeatIntervalMs); + // The assignment is only provided in the following cases: + // 1. The member reported its owned partitions; + // 2. The member just joined or rejoined to group. This is signaled with epoch equals to zero; + // 3. The member's assignment has been updated. if (ownedTopicPartitions != null || memberEpoch == 0 || assignmentUpdated) { response.setAssignment(createResponseAssignment(member)); } @@ -692,7 +706,8 @@ public Result consumerGroupHeartbeat( /** * Replays ConsumerGroupMemberMetadataKey/Value to update the hard state of - * the consumer group. + * the consumer group. It updates the subscription part of the member or + * delete the member. * * @param key A ConsumerGroupMemberMetadataKey key. * @param value A ConsumerGroupMemberMetadataValue record. @@ -727,7 +742,8 @@ public void replay( /** * Replays ConsumerGroupMetadataKey/Value to update the hard state of - * the consumer group. + * the consumer group. It updates the group epoch of the consumer + * group or deletes the consumer group. * * @param key A ConsumerGroupMetadataKey key. * @param value A ConsumerGroupMetadataValue record. @@ -749,7 +765,8 @@ public void replay( } if (!consumerGroup.targetAssignments().isEmpty()) { throw new IllegalStateException("Received a tombstone record to delete group " + groupId - + " but the group still has " + consumerGroup.targetAssignments().size() + " members."); + + " but the target assignment still has " + consumerGroup.targetAssignments().size() + + " members."); } if (consumerGroup.assignmentEpoch() != -1) { throw new IllegalStateException("Received a tombstone record to delete group " + groupId @@ -762,7 +779,8 @@ public void replay( /** * Replays ConsumerGroupPartitionMetadataKey/Value to update the hard state of - * the consumer group. + * the consumer group. It updates the subscription metadata of the consumer + * group. * * @param key A ConsumerGroupPartitionMetadataKey key. * @param value A ConsumerGroupPartitionMetadataValue record. @@ -787,7 +805,7 @@ public void replay( /** * Replays ConsumerGroupTargetAssignmentMemberKey/Value to update the hard state of - * the consumer group. + * the consumer group. It updates the target assignment of the member or deletes it. * * @param key A ConsumerGroupTargetAssignmentMemberKey key. * @param value A ConsumerGroupTargetAssignmentMemberValue record. @@ -809,7 +827,8 @@ public void replay( /** * Replays ConsumerGroupTargetAssignmentMetadataKey/Value to update the hard state of - * the consumer group. + * the consumer group. It updates the target assignment epoch or set it to -1 to signal + * that it has been deleted. * * @param key A ConsumerGroupTargetAssignmentMetadataKey key. * @param value A ConsumerGroupTargetAssignmentMetadataValue record. @@ -834,7 +853,7 @@ public void replay( /** * Replays ConsumerGroupCurrentMemberAssignmentKey/Value to update the hard state of - * the consumer group. + * the consumer group. It updates the assignment of a member or deletes it. * * @param key A ConsumerGroupCurrentMemberAssignmentKey key. * @param value A ConsumerGroupCurrentMemberAssignmentValue record. diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index 29b427859b975..f6cfb5fa2ffa5 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -235,6 +235,9 @@ public ConsumerGroupMember getOrMaybeCreateMember( * @param newMember The new member state. */ public void updateMember(ConsumerGroupMember newMember) { + if (newMember == null) { + throw new IllegalArgumentException("newMember cannot be null."); + } ConsumerGroupMember oldMember = members.put(newMember.memberId(), newMember); maybeUpdatePartitionEpoch(oldMember, newMember); maybeUpdateGroupState(); @@ -274,7 +277,7 @@ public int numMembers() { /** * Returns the members keyed by their id. * - * @return A immutable Map containing all the members. + * @return An immutable Map containing all the members. */ public Map members() { return Collections.unmodifiableMap(members); @@ -310,9 +313,9 @@ public void removeTargetAssignment(String memberId) { } /** - * Returns the target assignments for the entire group. + * Returns the target assignments for the entire group keyed by member id. * - * @return A immutable Map containing all the target assignments. + * @return An immutable Map containing all the target assignments. */ public Map targetAssignments() { return Collections.unmodifiableMap(assignments); @@ -379,7 +382,7 @@ public Map subscriptionMetadata() { } /** - * Updates the subscription metadata. This replace the previous one. + * Updates the subscription metadata. This replaces the previous one. * * @param subscriptionMetadata The new subscription metadata. */ @@ -391,8 +394,7 @@ public void setSubscriptionMetadata( } /** - * Computes new subscription metadata but with specific information for - * a member. + * Computes a new subscription metadata with a member's updated topic subscriptions. * * @param memberId The member id. * @param updatedMemberSubscriptions The member's updated topic subscriptions. @@ -446,8 +448,7 @@ private void maybeUpdateGroupState() { } else if (groupEpoch.get() > assignmentEpoch.get()) { state.set(ConsumerGroupState.ASSIGNING); } else { - for (Map.Entry keyValue : members.entrySet()) { - ConsumerGroupMember member = keyValue.getValue(); + for (ConsumerGroupMember member : members.values()) { if (member.nextMemberEpoch() != assignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) { state.set(ConsumerGroupState.RECONCILING); return; @@ -460,6 +461,9 @@ private void maybeUpdateGroupState() { /** * Updates the partition epochs based on the old and the new member. + * + * @param oldMember The old member. + * @param newMember The new member. */ private void maybeUpdatePartitionEpoch( ConsumerGroupMember oldMember, @@ -482,6 +486,8 @@ private void maybeUpdatePartitionEpoch( /** * Removes the partition epochs for the provided member. + * + * @param oldMember The old member. */ private void maybeRemovePartitionEpoch( ConsumerGroupMember oldMember @@ -494,6 +500,8 @@ private void maybeRemovePartitionEpoch( /** * Removes the partition epochs based on the provided assignment. + * + * @param assignment The assignment. */ private void removePartitionEpochs( Map> assignment @@ -516,6 +524,9 @@ private void removePartitionEpochs( /** * Adds the partitions epoch based on the provided assignment. + * + * @param assignment The assignment. + * @param epoch The new epoch. */ private void addPartitionEpochs( Map> assignment, @@ -523,7 +534,9 @@ private void addPartitionEpochs( ) { assignment.forEach((topicId, assignedPartitions) -> { currentPartitionEpoch.compute(topicId, (__, partitionsOrNull) -> { - if (partitionsOrNull == null) partitionsOrNull = new TimelineHashMap<>(snapshotRegistry, 1); + if (partitionsOrNull == null) { + partitionsOrNull = new TimelineHashMap<>(snapshotRegistry, assignedPartitions.size()); + } for (Integer partitionId : assignedPartitions) { partitionsOrNull.put(partitionId, epoch); } @@ -531,5 +544,4 @@ private void addPartitionEpochs( }); }); } - } From 628fb6ac4ab1589e205580dcf96b45799b0a987d Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 22 May 2023 11:50:08 +0200 Subject: [PATCH 04/16] don't recompute target assignment when member leaves --- .../group/GroupMetadataManager.java | 27 +------- .../group/GroupMetadataManagerTest.java | 68 +------------------ 2 files changed, 3 insertions(+), 92 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 b48c53a91b9da..c5b3309fce4c2 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 @@ -607,7 +607,7 @@ private Result consumerGroupLeave( List records = new ArrayList<>(); ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false); - ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); + group.getOrMaybeCreateMember(memberId, false); log.info("[GroupId " + groupId + "] Member " + memberId + " left the consumer group."); @@ -633,31 +633,6 @@ private Result consumerGroupLeave( int groupEpoch = group.groupEpoch() + 1; records.add(newGroupEpochRecord(groupId, groupEpoch)); - // We update the target assignment for the group and write it to - // the log. - String assignorName = group.preferredServerAssignor( - member.memberId(), - Optional.empty() - ).orElse(defaultAssignor.name()); - - try { - TargetAssignmentBuilder.TargetAssignmentResult assignmentResult = - new TargetAssignmentBuilder(groupId, groupEpoch, assignors.get(assignorName)) - .withMembers(group.members()) - .withSubscriptionMetadata(subscriptionMetadata) - .removeMember(member.memberId()) - .build(); - - log.info("[GroupId " + groupId + "] Computed a new target assignment for epoch " + groupEpoch + ": " - + assignmentResult.targetAssignment() + "."); - - records.addAll(assignmentResult.records()); - } catch (PartitionAssignorException ex) { - String msg = "Failed to compute a new target assignment for epoch " + groupEpoch + ": " + ex + "."; - log.error("[GroupId " + groupId + "] " + msg); - throw new UnknownServerException(msg, ex); - } - return new Result<>(records, new ConsumerGroupHeartbeatResponseData() .setMemberId(memberId) .setMemberEpoch(-1) 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 2c233de630282..617467ff199f8 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 @@ -1014,7 +1014,7 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { } @Test - public void testLeavingMemberTriggersNewTargetAssignment() { + public void testLeavingMemberBumpsGroupEpoch() { String groupId = "fooup"; // Use a static member id as it makes the test easier. String memberId1 = Uuid.randomUuid().toString(); @@ -1072,13 +1072,6 @@ public void testLeavingMemberTriggersNewTargetAssignment() { .withAssignmentEpoch(10)) .build(); - assignor.prepareGroupAssignment(new GroupAssignment( - Collections.singletonMap(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), - mkTopicAssignment(barTopicId, 0, 1, 2) - ))) - )); - // Member 3 leaves the consumer group. Result result = context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() @@ -1107,12 +1100,7 @@ public void testLeavingMemberTriggersNewTargetAssignment() { put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)); } }), - RecordHelpers.newGroupEpochRecord(groupId, 11), - RecordHelpers.newTargetAssignmentRecord(groupId, memberId1, mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), - mkTopicAssignment(barTopicId, 0, 1, 2) - )), - RecordHelpers.newTargetAssignmentEpochRecord(groupId, 11) + RecordHelpers.newGroupEpochRecord(groupId, 11) ); assertEquals(expectedRecords, result.records()); @@ -1966,58 +1954,6 @@ public void testPartitionAssignorExceptionOnRegularHeartbeat() { .setTopicPartitions(Collections.emptyList()))); } - @Test - public void testPartitionAssignorExceptionOnLeaveHeartbeat() { - String groupId = "fooup"; - // Use a static member id as it makes the test easier. - String memberId1 = Uuid.randomUuid().toString(); - - Uuid fooTopicId = Uuid.randomUuid(); - String fooTopicName = "foo"; - Uuid barTopicId = Uuid.randomUuid(); - String barTopicName = "bar"; - - PartitionAssignor assignor = mock(PartitionAssignor.class); - when(assignor.assign(any())).thenThrow(new PartitionAssignorException("Assignment failed.")); - - // Consumer group with two members. - GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() - .withAssignors(Collections.singletonList(assignor)) - .withTopicsImage(new TopicsImageBuilder() - .addTopic(fooTopicId, fooTopicName, 6) - .addTopic(barTopicId, barTopicName, 3) - .build()) - .withConsumerGroup(new ConsumerGroupBuilder(groupId, 10) - .withMember(new ConsumerGroupMember.Builder(memberId1) - .setMemberEpoch(10) - .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) - .setClientId("client") - .setClientHost("localhost/127.0.0.1") - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .setServerAssignorName("range") - .setAssignedPartitions(mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1, 2), - mkTopicAssignment(barTopicId, 0, 1))) - .build()) - .withAssignment(memberId1, mkAssignment( - mkTopicAssignment(fooTopicId, 0, 1, 2), - mkTopicAssignment(barTopicId, 0, 1))) - .withAssignmentEpoch(10)) - .build(); - - // Member 3 leaves the consumer group. - assertThrows(UnknownServerException.class, () -> - context.consumerGroupHeartbeat( - new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId1) - .setMemberEpoch(-1) - .setRebalanceTimeoutMs(5000) - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .setTopicPartitions(Collections.emptyList()))); - } - private void assertUnorderedListEquals( List expected, List actual From 190fe726ea483c27bd1f22a96e0689acc28fa133 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 22 May 2023 12:06:06 +0200 Subject: [PATCH 05/16] cleanup --- .../kafka/coordinator/group/GroupMetadataManagerTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 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 617467ff199f8..b4c40a9f3b06e 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 @@ -856,10 +856,12 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { List expectedRecords = Arrays.asList( RecordHelpers.newMemberSubscriptionRecord(groupId, expectedMember), - RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() {{ + RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, new HashMap() { + { put(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)); put(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)); - }}), + } + }), RecordHelpers.newGroupEpochRecord(groupId, 11), RecordHelpers.newTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), From 859e7709772a3269f20c4e3c290dca4cfb7d0586 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 22 May 2023 13:45:47 +0200 Subject: [PATCH 06/16] Make computeSubscriptionMetadata more efficient --- checkstyle/suppressions.xml | 2 +- .../group/GroupMetadataManager.java | 8 +- .../group/consumer/ConsumerGroup.java | 111 ++++++++---- .../group/consumer/ConsumerGroupTest.java | 170 +++++++++++++++--- 4 files changed, 223 insertions(+), 68 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index a7d7e453c627d..3876b9f519a3e 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -322,7 +322,7 @@ + files="(ConsumerGroupTest|GroupMetadataManagerTest).java"/> consumerGroupHeartbeat( updatedMember.subscribedTopicNames()); subscriptionMetadata = group.computeSubscriptionMetadata( - updatedMember.memberId(), - updatedMember.subscribedTopicNames(), + member, + updatedMember, topicsImage ); @@ -607,7 +607,7 @@ private Result consumerGroupLeave( List records = new ArrayList<>(); ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false); - group.getOrMaybeCreateMember(memberId, false); + ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false); log.info("[GroupId " + groupId + "] Member " + memberId + " left the consumer group."); @@ -618,7 +618,7 @@ private Result consumerGroupLeave( // We update the subscription metadata without the leaving member. Map subscriptionMetadata = group.computeSubscriptionMetadata( - memberId, + member, null, topicsImage ); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index f6cfb5fa2ffa5..1585f4d57e25d 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -28,12 +28,10 @@ import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.function.Consumer; /** * A Consumer Group. All the metadata in this class are backed by @@ -87,6 +85,11 @@ public String toString() { */ private final TimelineHashMap members; + /** + * The number of subscribers per topic. + */ + private final TimelineHashMap subscribedTopicNames; + /** * The metadata of the subscribed topics. */ @@ -120,6 +123,7 @@ public ConsumerGroup( this.state = new TimelineObject<>(snapshotRegistry, ConsumerGroupState.EMPTY); this.groupEpoch = new TimelineInteger(snapshotRegistry); this.members = new TimelineHashMap<>(snapshotRegistry, 0); + this.subscribedTopicNames = new TimelineHashMap<>(snapshotRegistry, 0); this.subscribedTopicMetadata = new TimelineHashMap<>(snapshotRegistry, 0); this.assignmentEpoch = new TimelineInteger(snapshotRegistry); this.assignments = new TimelineHashMap<>(snapshotRegistry, 0); @@ -239,6 +243,7 @@ public void updateMember(ConsumerGroupMember newMember) { throw new IllegalArgumentException("newMember cannot be null."); } ConsumerGroupMember oldMember = members.put(newMember.memberId(), newMember); + maybeUpdateSubscribedTopicNames(oldMember, newMember); maybeUpdatePartitionEpoch(oldMember, newMember); maybeUpdateGroupState(); } @@ -394,46 +399,39 @@ public void setSubscriptionMetadata( } /** - * Computes a new subscription metadata with a member's updated topic subscriptions. + * Computes the subscription metadata based on the current subscription and + * an updated member. * - * @param memberId The member id. - * @param updatedMemberSubscriptions The member's updated topic subscriptions. - * @param topicsImage The topic metadata. + * @param oldMember The old member. + * @param newMember The new member. + * @param topicsImage The topic metadata. * * @return The new subscription metadata as an immutable Map. */ public Map computeSubscriptionMetadata( - String memberId, - List updatedMemberSubscriptions, + ConsumerGroupMember oldMember, + ConsumerGroupMember newMember, TopicsImage topicsImage ) { - Map newSubscriptionMetadata = new HashMap<>(subscriptionMetadata().size()); - - Consumer> updateSubscription = subscribedTopicNames -> { - subscribedTopicNames.forEach(topicName -> - newSubscriptionMetadata.computeIfAbsent(topicName, __ -> { - TopicImage topicImage = topicsImage.getTopic(topicName); - if (topicImage == null) { - return null; - } else { - return new TopicMetadata( - topicImage.id(), - topicImage.name(), - topicImage.partitions().size() - ); - } - }) - ); - }; - - if (updatedMemberSubscriptions != null) { - updateSubscription.accept(updatedMemberSubscriptions); - } - - members.forEach((mid, member) -> { - if (!mid.equals(memberId)) { - updateSubscription.accept(member.subscribedTopicNames()); - } + // Copy and update the current subscriptions. + Map subscribedTopicNames = new HashMap<>(this.subscribedTopicNames); + maybeUpdateSubscribedTopicNames(subscribedTopicNames, oldMember, newMember); + + // Create the topic metadata for each subscribed topic. + Map newSubscriptionMetadata = new HashMap<>(subscribedTopicNames.size()); + subscribedTopicNames.forEach((topicName, count) -> { + newSubscriptionMetadata.computeIfAbsent(topicName, __ -> { + TopicImage topicImage = topicsImage.getTopic(topicName); + if (topicImage == null) { + return null; + } else { + return new TopicMetadata( + topicImage.id(), + topicImage.name(), + topicImage.partitions().size() + ); + } + }); }); return Collections.unmodifiableMap(newSubscriptionMetadata); @@ -459,6 +457,49 @@ private void maybeUpdateGroupState() { } } + /** + * Updates the subscribed topic names count. + * + * @param oldMember The old member. + * @param newMember The new member. + */ + private void maybeUpdateSubscribedTopicNames( + ConsumerGroupMember oldMember, + ConsumerGroupMember newMember + ) { + maybeUpdateSubscribedTopicNames(subscribedTopicNames, oldMember, newMember); + } + + /** + * Updates the subscription count. + * + * @param subscribedTopicCount The map to update. + * @param oldMember The old member. + * @param newMember The new member. + */ + private static void maybeUpdateSubscribedTopicNames( + Map subscribedTopicCount, + ConsumerGroupMember oldMember, + ConsumerGroupMember newMember + ) { + if (oldMember != null) { + oldMember.subscribedTopicNames().forEach(topicName -> { + subscribedTopicCount.compute(topicName, (__, value) -> { + if (value == null) return null; + return value == 1 ? null : value - 1; + }); + }); + } + + if (newMember != null) { + newMember.subscribedTopicNames().forEach(topicName -> { + subscribedTopicCount.compute(topicName, (__, value) -> { + return value == null ? 1 : value + 1; + }); + }); + } + } + /** * Updates the partition epochs based on the old and the new member. * diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index 287095c770fd1..22e37e102b735 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -26,10 +26,10 @@ import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.Map; import java.util.Optional; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment; import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -294,33 +294,147 @@ public void testUpdateSubscriptionMetadata() { .addTopic(zarTopicId, "zar", 3) .build(); - ConsumerGroup consumerGroup = createConsumerGroup("foo"); + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") + .setSubscribedTopicNames(Arrays.asList("foo")) + .build(); + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") + .setSubscribedTopicNames(Arrays.asList("bar")) + .build(); + ConsumerGroupMember member3 = new ConsumerGroupMember.Builder("member3") + .setSubscribedTopicNames(Arrays.asList("zar")) + .build(); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member1") - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .build()); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member2") - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .build()); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member3") - .setSubscribedTopicNames(Arrays.asList("bar", "zar")) - .build()); + ConsumerGroup consumerGroup = createConsumerGroup("group-foo"); - Map expectedSubscriptionMetadata = new HashMap<>(); - expectedSubscriptionMetadata.put("foo", new TopicMetadata(fooTopicId, "foo", 1)); - expectedSubscriptionMetadata.put("bar", new TopicMetadata(barTopicId, "bar", 2)); - expectedSubscriptionMetadata.put("zar", new TopicMetadata(zarTopicId, "zar", 3)); - assertEquals(expectedSubscriptionMetadata, consumerGroup.computeSubscriptionMetadata( - null, - Collections.emptyList(), - image - )); - - expectedSubscriptionMetadata.remove("zar"); - assertEquals(expectedSubscriptionMetadata, consumerGroup.computeSubscriptionMetadata( - "member3", - Collections.emptyList(), - image - )); + // It should be empty by default. + assertEquals( + Collections.emptyMap(), + consumerGroup.computeSubscriptionMetadata( + null, + null, + image + ) + ); + + // Adding member1. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)) + ), + consumerGroup.computeSubscriptionMetadata( + null, + member1, + image + ) + ); + + // Updating the group with member1. + consumerGroup.updateMember(member1); + + // It should return foo now. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)) + ), + consumerGroup.computeSubscriptionMetadata( + null, + null, + image + ) + ); + + // Removing member1 results in an empty map. + assertEquals( + Collections.emptyMap(), + consumerGroup.computeSubscriptionMetadata( + member1, + null, + image + ) + ); + + // Adding member2 adds bar. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)), + mkEntry("bar", new TopicMetadata(barTopicId, "bar", 2)) + ), + consumerGroup.computeSubscriptionMetadata( + null, + member2, + image + ) + ); + + // Updating the group with member2. + consumerGroup.updateMember(member2); + + // It should return foo and bar. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)), + mkEntry("bar", new TopicMetadata(barTopicId, "bar", 2)) + ), + consumerGroup.computeSubscriptionMetadata( + null, + null, + image + ) + ); + + // Removing member2 results in returning foo. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)) + ), + consumerGroup.computeSubscriptionMetadata( + member2, + null, + image + ) + ); + + // Removing member1 results in returning bar. + assertEquals( + mkMap( + mkEntry("bar", new TopicMetadata(barTopicId, "bar", 2)) + ), + consumerGroup.computeSubscriptionMetadata( + member1, + null, + image + ) + ); + + // Adding member3 adds zar. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)), + mkEntry("bar", new TopicMetadata(barTopicId, "bar", 2)), + mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3)) + ), + consumerGroup.computeSubscriptionMetadata( + null, + member3, + image + ) + ); + + // Updating group with member3. + consumerGroup.updateMember(member3); + + // It should return foo, bar and zar. + assertEquals( + mkMap( + mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)), + mkEntry("bar", new TopicMetadata(barTopicId, "bar", 2)), + mkEntry("zar", new TopicMetadata(zarTopicId, "zar", 3)) + ), + consumerGroup.computeSubscriptionMetadata( + null, + member3, + image + ) + ); } } From 81544d6e7770cd7bd1c86f454817380a2bbbd9b4 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 22 May 2023 14:41:11 +0200 Subject: [PATCH 07/16] optimize preferredServerAssignor --- .../group/GroupMetadataManager.java | 32 +++--- .../group/consumer/ConsumerGroup.java | 99 ++++++++++++----- .../group/consumer/ConsumerGroupMember.java | 3 +- .../group/consumer/ConsumerGroupTest.java | 102 ++++++++++++++---- 4 files changed, 170 insertions(+), 66 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 0313657ac6e52..491163456080a 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 @@ -458,12 +458,13 @@ private Result consumerGroupHeartbeat( // Get or create the consumer group. boolean createIfNotExists = memberEpoch == 0; - ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, createIfNotExists); + final ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, createIfNotExists); throwIfConsumerGroupIsFull(group, memberId); - // Get or create the member. + // Get or create the member. Note that member is the persisted member anytime + // in this method. if (memberId.isEmpty()) memberId = Uuid.randomUuid().toString(); - ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, createIfNotExists); + final ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, createIfNotExists); throwIfMemberEpochIsInvalid(member, memberEpoch, ownedTopicPartitions); if (memberEpoch == 0) { @@ -512,8 +513,6 @@ private Result consumerGroupHeartbeat( log.info("[GroupId " + groupId + "] Bumped group epoch to " + groupEpoch + "."); } - - member = updatedMember; } // 2. Update the target assignment if the group epoch is larger than the target assignment epoch. The @@ -522,8 +521,8 @@ private Result consumerGroupHeartbeat( Assignment targetAssignment = group.targetAssignment(memberId); if (groupEpoch > targetAssignmentEpoch) { String preferredServerAssignor = group.preferredServerAssignor( - member.memberId(), - member.serverAssignorName() + member, + updatedMember ).orElse(defaultAssignor.name()); try { @@ -532,14 +531,14 @@ private Result consumerGroupHeartbeat( .withMembers(group.members()) .withSubscriptionMetadata(subscriptionMetadata) .withTargetAssignment(group.targetAssignments()) - .addOrUpdateMember(member.memberId(), member) + .addOrUpdateMember(memberId, updatedMember) .build(); log.info("[GroupId " + groupId + "] Computed a new target assignment for epoch " + groupEpoch + ": " + assignmentResult.targetAssignment() + "."); records.addAll(assignmentResult.records()); - targetAssignment = assignmentResult.targetAssignment().get(member.memberId()); + targetAssignment = assignmentResult.targetAssignment().get(memberId); targetAssignmentEpoch = groupEpoch; } catch (PartitionAssignorException ex) { String msg = "Failed to compute a new target assignment for epoch " + groupEpoch + ": " + ex + "."; @@ -551,8 +550,9 @@ private Result consumerGroupHeartbeat( // 3. Reconcile the member's assignment with the target assignment. This is only required if // the member is not stable or if a new target assignment has been installed. boolean assignmentUpdated = false; - if (member.state() != ConsumerGroupMember.MemberState.STABLE || member.nextMemberEpoch() != targetAssignmentEpoch) { - updatedMember = new CurrentAssignmentBuilder(member) + if (updatedMember.state() != ConsumerGroupMember.MemberState.STABLE || updatedMember.nextMemberEpoch() != targetAssignmentEpoch) { + ConsumerGroupMember prevMember = updatedMember; + updatedMember = new CurrentAssignmentBuilder(updatedMember) .withTargetAssignment(targetAssignmentEpoch, targetAssignment) .withCurrentPartitionEpoch(group::currentPartitionEpoch) .withOwnedTopicPartitions(ownedTopicPartitions) @@ -560,7 +560,7 @@ private Result consumerGroupHeartbeat( // Checking the reference is enough here because a new instance // is created only when the state has changed. - if (updatedMember != member) { + if (updatedMember != prevMember) { assignmentUpdated = true; records.add(newCurrentAssignmentRecord(groupId, updatedMember)); @@ -568,8 +568,6 @@ private Result consumerGroupHeartbeat( member.currentAssignmentSummary() + " to " + updatedMember.currentAssignmentSummary() + "."); // TODO(dajac) Starts or restarts the timer for the revocation timeout. - - member = updatedMember; } } @@ -577,8 +575,8 @@ private Result consumerGroupHeartbeat( // Prepare the response. ConsumerGroupHeartbeatResponseData response = new ConsumerGroupHeartbeatResponseData() - .setMemberId(member.memberId()) - .setMemberEpoch(member.memberEpoch()) + .setMemberId(updatedMember.memberId()) + .setMemberEpoch(updatedMember.memberEpoch()) .setHeartbeatIntervalMs(consumerGroupHeartbeatIntervalMs); // The assignment is only provided in the following cases: @@ -586,7 +584,7 @@ private Result consumerGroupHeartbeat( // 2. The member just joined or rejoined to group. This is signaled with epoch equals to zero; // 3. The member's assignment has been updated. if (ownedTopicPartitions != null || memberEpoch == 0 || assignmentUpdated) { - response.setAssignment(createResponseAssignment(member)); + response.setAssignment(createResponseAssignment(updatedMember)); } return new Result<>(records, response); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index 1585f4d57e25d..0b70aa9bbfb67 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -85,6 +85,11 @@ public String toString() { */ private final TimelineHashMap members; + /** + * The number of members per server assignor name. + */ + private final TimelineHashMap serverAssignors; + /** * The number of subscribers per topic. */ @@ -123,6 +128,7 @@ public ConsumerGroup( this.state = new TimelineObject<>(snapshotRegistry, ConsumerGroupState.EMPTY); this.groupEpoch = new TimelineInteger(snapshotRegistry); this.members = new TimelineHashMap<>(snapshotRegistry, 0); + this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0); this.subscribedTopicNames = new TimelineHashMap<>(snapshotRegistry, 0); this.subscribedTopicMetadata = new TimelineHashMap<>(snapshotRegistry, 0); this.assignmentEpoch = new TimelineInteger(snapshotRegistry); @@ -244,6 +250,7 @@ public void updateMember(ConsumerGroupMember newMember) { } ConsumerGroupMember oldMember = members.put(newMember.memberId(), newMember); maybeUpdateSubscribedTopicNames(oldMember, newMember); + maybeUpdateServerAssignors(oldMember, newMember); maybeUpdatePartitionEpoch(oldMember, newMember); maybeUpdateGroupState(); } @@ -350,26 +357,18 @@ public int currentPartitionEpoch( * Compute the preferred (server side) assignor for the group while * using the provided assignor for the member. * - * @param updatedMemberId The member id. - * @param serverAssignorNameOpt The assignor name. + * @param oldMember The old member. + * @param newMember The new member. * * @return An Optional containing the preferred assignor. */ public Optional preferredServerAssignor( - String updatedMemberId, - Optional serverAssignorNameOpt + ConsumerGroupMember oldMember, + ConsumerGroupMember newMember ) { - Map counts = new HashMap<>(); - - serverAssignorNameOpt.ifPresent(serverAssignorName -> - counts.put(serverAssignorName, 1) - ); - - members.forEach((memberId, member) -> { - if (!memberId.equals(updatedMemberId) && member.serverAssignorName().isPresent()) { - counts.compute(member.serverAssignorName().get(), (k, v) -> v == null ? 1 : v + 1); - } - }); + // Copy the current count and update it. + Map counts = new HashMap<>(this.serverAssignors); + maybeUpdateServerAssignors(counts, oldMember, newMember); return counts.entrySet().stream() .max(Map.Entry.comparingByValue()) @@ -457,6 +456,43 @@ private void maybeUpdateGroupState() { } } + /** + * Updates the server assignors count. + * + * @param oldMember The old member. + * @param newMember The new member. + */ + private void maybeUpdateServerAssignors( + ConsumerGroupMember oldMember, + ConsumerGroupMember newMember + ) { + maybeUpdateServerAssignors(serverAssignors, oldMember, newMember); + } + + /** + * Updates the server assignors count. + * + * @param serverAssignorCount The count to update. + * @param oldMember The old member. + * @param newMember The new member. + */ + private static void maybeUpdateServerAssignors( + Map serverAssignorCount, + ConsumerGroupMember oldMember, + ConsumerGroupMember newMember + ) { + if (oldMember != null) { + oldMember.serverAssignorName().ifPresent(name -> + serverAssignorCount.compute(name, ConsumerGroup::decValue) + ); + } + if (newMember != null) { + newMember.serverAssignorName().ifPresent(name -> + serverAssignorCount.compute(name, ConsumerGroup::incValue) + ); + } + } + /** * Updates the subscribed topic names count. * @@ -483,20 +519,15 @@ private static void maybeUpdateSubscribedTopicNames( ConsumerGroupMember newMember ) { if (oldMember != null) { - oldMember.subscribedTopicNames().forEach(topicName -> { - subscribedTopicCount.compute(topicName, (__, value) -> { - if (value == null) return null; - return value == 1 ? null : value - 1; - }); - }); + oldMember.subscribedTopicNames().forEach(topicName -> + subscribedTopicCount.compute(topicName, ConsumerGroup::decValue) + ); } if (newMember != null) { - newMember.subscribedTopicNames().forEach(topicName -> { - subscribedTopicCount.compute(topicName, (__, value) -> { - return value == null ? 1 : value + 1; - }); - }); + newMember.subscribedTopicNames().forEach(topicName -> + subscribedTopicCount.compute(topicName, ConsumerGroup::incValue) + ); } } @@ -585,4 +616,20 @@ private void addPartitionEpochs( }); }); } + + /** + * Decrements value by 1; returns null when reaching zero. This helper is + * meant to be used with Map#compute. + */ + private static Integer decValue(String key, Integer value) { + if (value == null) return null; + return value == 1 ? null : value - 1; + } + + /** + * Increments value by 1; This helper is meant to be used with Map#compute. + */ + private static Integer incValue(String key, Integer value) { + return value == null ? 1 : value + 1; + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java index c40bb7c937c5f..958f1ae7cdf44 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java @@ -535,8 +535,7 @@ public Map> partitionsPendingAssignment() { * @return A string representation of the current assignment state. */ public String currentAssignmentSummary() { - return "CurrentAssignment(" + - ", memberEpoch=" + memberEpoch + + return "CurrentAssignment(memberEpoch=" + memberEpoch + ", previousMemberEpoch=" + previousMemberEpoch + ", nextMemberEpoch=" + nextMemberEpoch + ", state=" + state + diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index 22e37e102b735..83cb97d346444 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -246,39 +246,99 @@ public void testGroupState() { public void testPreferredServerAssignor() { ConsumerGroup consumerGroup = createConsumerGroup("foo"); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member1") + ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") .setServerAssignorName("range") - .build()); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member2") + .build(); + ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") .setServerAssignorName("range") - .build()); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member3") + .build(); + ConsumerGroupMember member3 = new ConsumerGroupMember.Builder("member3") .setServerAssignorName("uniform") - .build()); + .build(); + + assertEquals( + Optional.empty(), + consumerGroup.preferredServerAssignor(null, null) + ); + + assertEquals( + Optional.of("range"), + consumerGroup.preferredServerAssignor(null, member1) + ); + + consumerGroup.updateMember(member1); + + assertEquals( + Optional.of("range"), + consumerGroup.preferredServerAssignor(null, null) + ); + + assertEquals( + Optional.empty(), + consumerGroup.preferredServerAssignor(member1, null) + ); + + assertEquals( + Optional.of("range"), + consumerGroup.preferredServerAssignor(null, member2) + ); + + consumerGroup.updateMember(member2); - assertEquals(Optional.of("range"), consumerGroup.preferredServerAssignor( - null, - Optional.empty()) + assertEquals( + Optional.of("range"), + consumerGroup.preferredServerAssignor(null, null) ); - assertEquals(Optional.of("uniform"), consumerGroup.preferredServerAssignor( - "member2", - Optional.of("uniform")) + consumerGroup.updateMember(member3); + + assertEquals( + Optional.of("range"), + consumerGroup.preferredServerAssignor(null, null) ); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member1") + // Members without assignors + ConsumerGroupMember updatedMember1 = new ConsumerGroupMember.Builder("member1") .setServerAssignorName(null) - .build()); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member2") + .build(); + ConsumerGroupMember updatedMember2 = new ConsumerGroupMember.Builder("member2") .setServerAssignorName(null) - .build()); - consumerGroup.updateMember(new ConsumerGroupMember.Builder("member3") + .build(); + ConsumerGroupMember updatedMember3 = new ConsumerGroupMember.Builder("member3") .setServerAssignorName(null) - .build()); + .build(); + + + Optional assignor = consumerGroup.preferredServerAssignor(member1, updatedMember1); + assertTrue(assignor.equals(Optional.of("range")) || assignor.equals(Optional.of("uniform"))); + + consumerGroup.updateMember(updatedMember1); + + assignor = consumerGroup.preferredServerAssignor(member1, updatedMember1); + assertTrue(assignor.equals(Optional.of("range")) || assignor.equals(Optional.of("uniform"))); - assertEquals(Optional.empty(), consumerGroup.preferredServerAssignor( - null, - Optional.empty()) + assertEquals( + Optional.of("uniform"), + consumerGroup.preferredServerAssignor(member2, updatedMember2) + ); + + consumerGroup.updateMember(updatedMember2); + + assertEquals( + Optional.of("uniform"), + consumerGroup.preferredServerAssignor(null, null) + ); + + assertEquals( + Optional.empty(), + consumerGroup.preferredServerAssignor(member3, updatedMember3) + ); + + consumerGroup.updateMember(updatedMember3); + + assertEquals( + Optional.empty(), + consumerGroup.preferredServerAssignor(null, null) ); } From eb4263c7d3e4206ecc4df9427f90d74ac71035ba Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 22 May 2023 14:47:03 +0200 Subject: [PATCH 08/16] fix --- .../coordinator/group/GroupMetadataManager.java | 13 ++++++------- .../coordinator/group/GroupMetadataManagerTest.java | 3 +-- 2 files changed, 7 insertions(+), 9 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 491163456080a..08c0960005e02 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 @@ -371,17 +371,16 @@ private void throwIfMemberEpochIsInvalid( List ownedTopicPartitions ) { if (receivedMemberEpoch > member.memberEpoch()) { - // The member has likely got a bump from another coordinator and this coordinator - // is stale. Return NOT_COORDINATOR to force the member to refresh its coordinator. - throw new NotCoordinatorException("The consumer group member has a larger member " - + "epoch (" + receivedMemberEpoch + ") than the one known by this group coordinator (" - + member.memberEpoch() + ")."); + throw new FencedMemberEpochException("The consumer 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() || !isSubset(ownedTopicPartitions, member.assignedPartitions())) { - throw new FencedMemberEpochException("The consumer group member has an old member " - + "epoch. The member must abandon all its partitions and rejoin."); + throw new FencedMemberEpochException("The consumer 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."); } } } 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 b4c40a9f3b06e..42663a3780d50 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 @@ -21,7 +21,6 @@ import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.apache.kafka.common.errors.GroupMaxSizeReachedException; import org.apache.kafka.common.errors.InvalidRequestException; -import org.apache.kafka.common.errors.NotCoordinatorException; import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnsupportedAssignorException; @@ -641,7 +640,7 @@ public void testConsumerGroupMemberEpochValidation() { context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, member)); // Member epoch is greater than the expected epoch. - assertThrows(NotCoordinatorException.class, () -> + assertThrows(FencedMemberEpochException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) From 35bd0579a8d75f06627dee0703acbb46be8bd182 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Mon, 22 May 2023 15:28:16 +0200 Subject: [PATCH 09/16] fix imports --- .../kafka/coordinator/group/GroupMetadataManagerTest.java | 4 ++-- .../kafka/coordinator/group/consumer/ConsumerGroupTest.java | 4 ++-- 2 files changed, 4 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 42663a3780d50..dd7f50d3b7f01 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 @@ -75,8 +75,8 @@ import java.util.Objects; import java.util.Set; -import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment; -import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.junit.jupiter.api.AssertionFailureBuilder.assertionFailure; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index 83cb97d346444..17b38a31cd2ef 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -30,8 +30,8 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; -import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkAssignment; -import static org.apache.kafka.coordinator.group.consumer.AssignmentTestUtil.mkTopicAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment; +import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; From a57c32cc5112d028216de57654c80c4e61a05e30 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 24 May 2023 09:37:47 +0200 Subject: [PATCH 10/16] fix request validation --- .../group/GroupMetadataManager.java | 58 ++++++++++++------- .../group/GroupMetadataManagerTest.java | 8 +-- 2 files changed, 42 insertions(+), 24 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 08c0960005e02..22e470133de31 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 @@ -249,6 +249,38 @@ private void removeGroup( groups.remove(groupId); } + /** + * Throws an InvalidRequestException if the value is non-null and empty. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + private void throwIfEmptyString( + String value, + String error + ) throws InvalidRequestException { + if (value != null && value.isEmpty()) { + throw new InvalidRequestException(error); + } + } + + /** + * Throws an InvalidRequestException if the value is non-null. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + private void throwIfNotNull( + Object value, + String error + ) throws InvalidRequestException { + if (value != null) { + throw new InvalidRequestException(error); + } + } + /** * Validates the request. * @@ -260,20 +292,14 @@ private void removeGroup( private void throwIfConsumerGroupHeartbeatRequestIsInvalid( ConsumerGroupHeartbeatRequestData request ) throws InvalidRequestException, UnsupportedAssignorException { - if (request.groupId().isEmpty()) { - throw new InvalidRequestException("GroupId 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."); + throwIfNotNull(request.subscribedTopicRegex(), "SubscribedTopicRegex is not supported yet."); + throwIfNotNull(request.clientAssignors(), "Client side assignors are not supported yet."); if (request.memberEpoch() > 0 || request.memberEpoch() == -1) { - if (request.memberId().isEmpty()) { - throw new InvalidRequestException("MemberId can't be empty."); - } - if (request.instanceId() != null) { - throw new InvalidRequestException("InstanceId should only be provided in first request."); - } - if (request.rackId() != null) { - throw new InvalidRequestException("RackId should only be provided in first request."); - } + throwIfEmptyString(request.memberId(), "MemberId can't be empty."); } else if (request.memberEpoch() == 0) { if (request.rebalanceTimeoutMs() == -1) { throw new InvalidRequestException("RebalanceTimeoutMs must be provided in first request."); @@ -293,14 +319,6 @@ private void throwIfConsumerGroupHeartbeatRequestIsInvalid( + " is not supported. Supported assignors: " + String.join(", ", assignors.keySet()) + "."); } - - if (request.subscribedTopicRegex() != null) { - throw new InvalidRequestException("SubscribedTopicRegex is not supported yet."); - } - - if (request.clientAssignors() != null) { - throw new InvalidRequestException("Client side assignors are not supported yet."); - } } /** 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 dd7f50d3b7f01..93b2d3e028cd6 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 @@ -485,16 +485,16 @@ public void testConsumerHeartbeatRequestValidation() { .setGroupId("foo") .setMemberId(Uuid.randomUuid().toString()) .setMemberEpoch(1) - .setInstanceId("instance-id"))); - assertEquals("InstanceId should only be provided in first request.", ex.getMessage()); + .setInstanceId(""))); + assertEquals("InstanceId can't be empty.", ex.getMessage()); ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") .setMemberId(Uuid.randomUuid().toString()) .setMemberEpoch(1) - .setRackId("rack-id"))); - assertEquals("RackId should only be provided in first request.", ex.getMessage()); + .setRackId(""))); + assertEquals("RackId can't be empty.", ex.getMessage()); ex = assertThrows(UnsupportedAssignorException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() From f5261cc6f43a5d35067bae3333b7556ec9c7e2dd Mon Sep 17 00:00:00 2001 From: David Jacot Date: Wed, 24 May 2023 09:49:31 +0200 Subject: [PATCH 11/16] address minor comments --- .../group/GroupMetadataManager.java | 5 ++-- .../group/consumer/ConsumerGroup.java | 24 +++++++++---------- .../group/GroupMetadataManagerTest.java | 2 +- 3 files changed, 14 insertions(+), 17 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 22e470133de31..9d9afadc5024f 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 @@ -478,14 +478,13 @@ private Result consumerGroupHeartbeat( final ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, createIfNotExists); throwIfConsumerGroupIsFull(group, memberId); - // Get or create the member. Note that member is the persisted member anytime - // in this method. + // Get or create the member. if (memberId.isEmpty()) memberId = Uuid.randomUuid().toString(); final ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, createIfNotExists); throwIfMemberEpochIsInvalid(member, memberEpoch, ownedTopicPartitions); if (memberEpoch == 0) { - log.info("[GroupId " + groupId + "] Member " + memberId + " re-joins the consumer group."); + log.info("[GroupId " + groupId + "] Member " + memberId + " joins the consumer group."); } // 1. Create or update the member. If the member is new or has changed, a ConsumerGroupMemberMetadataValue diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index 0b70aa9bbfb67..e8a353dc24c1f 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -355,7 +355,9 @@ public int currentPartitionEpoch( /** * Compute the preferred (server side) assignor for the group while - * using the provided assignor for the member. + * taking into account the updated member. The computation relies + * on {{@link ConsumerGroup#serverAssignors}} persisted structure + * but it does not update it. * * @param oldMember The old member. * @param newMember The new member. @@ -419,18 +421,14 @@ public Map computeSubscriptionMetadata( // Create the topic metadata for each subscribed topic. Map newSubscriptionMetadata = new HashMap<>(subscribedTopicNames.size()); subscribedTopicNames.forEach((topicName, count) -> { - newSubscriptionMetadata.computeIfAbsent(topicName, __ -> { - TopicImage topicImage = topicsImage.getTopic(topicName); - if (topicImage == null) { - return null; - } else { - return new TopicMetadata( - topicImage.id(), - topicImage.name(), - topicImage.partitions().size() - ); - } - }); + TopicImage topicImage = topicsImage.getTopic(topicName); + if (topicImage != null) { + newSubscriptionMetadata.put(topicName, new TopicMetadata( + topicImage.id(), + topicImage.name(), + topicImage.partitions().size() + )); + } }); return Collections.unmodifiableMap(newSubscriptionMetadata); 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 93b2d3e028cd6..8be6ef1987f19 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 @@ -1073,7 +1073,7 @@ public void testLeavingMemberBumpsGroupEpoch() { .withAssignmentEpoch(10)) .build(); - // Member 3 leaves the consumer group. + // Member 2 leaves the consumer group. Result result = context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) From 67da94ef57c4d4ef4b58534d79821d7c7f78e819 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 25 May 2023 11:51:52 +0200 Subject: [PATCH 12/16] address minor comments --- .../group/GroupMetadataManager.java | 20 ++-- .../group/consumer/ConsumerGroup.java | 87 +++++++-------- .../group/GroupMetadataManagerTest.java | 74 ++++--------- .../group/consumer/ConsumerGroupTest.java | 104 +++++++++++++----- 4 files changed, 146 insertions(+), 139 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 9d9afadc5024f..0528e4816ae3f 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 @@ -536,7 +536,7 @@ private Result consumerGroupHeartbeat( int targetAssignmentEpoch = group.assignmentEpoch(); Assignment targetAssignment = group.targetAssignment(memberId); if (groupEpoch > targetAssignmentEpoch) { - String preferredServerAssignor = group.preferredServerAssignor( + String preferredServerAssignor = group.computePreferredServerAssignor( member, updatedMember ).orElse(defaultAssignor.name()); @@ -546,7 +546,7 @@ private Result consumerGroupHeartbeat( new TargetAssignmentBuilder(groupId, groupEpoch, assignors.get(preferredServerAssignor)) .withMembers(group.members()) .withSubscriptionMetadata(subscriptionMetadata) - .withTargetAssignment(group.targetAssignments()) + .withTargetAssignment(group.targetAssignment()) .addOrUpdateMember(memberId, updatedMember) .build(); @@ -597,7 +597,7 @@ private Result consumerGroupHeartbeat( // The assignment is only provided in the following cases: // 1. The member reported its owned partitions; - // 2. The member just joined or rejoined to group. This is signaled with epoch equals to zero; + // 2. The member just joined or rejoined to group (epoch equals to zero); // 3. The member's assignment has been updated. if (ownedTopicPartitions != null || memberEpoch == 0 || assignmentUpdated) { response.setAssignment(createResponseAssignment(updatedMember)); @@ -721,7 +721,7 @@ public void replay( throw new IllegalStateException("Received a tombstone record to delete member " + memberId + " but did not receive ConsumerGroupCurrentMemberAssignmentValue tombstone."); } - if (consumerGroup.targetAssignments().containsKey(memberId)) { + if (consumerGroup.targetAssignment().containsKey(memberId)) { throw new IllegalStateException("Received a tombstone record to delete member " + memberId + " but did not receive ConsumerGroupTargetAssignmentMetadataValue tombstone."); } @@ -752,9 +752,9 @@ public void replay( throw new IllegalStateException("Received a tombstone record to delete group " + groupId + " but the group still has " + consumerGroup.members().size() + " members."); } - if (!consumerGroup.targetAssignments().isEmpty()) { + if (!consumerGroup.targetAssignment().isEmpty()) { throw new IllegalStateException("Received a tombstone record to delete group " + groupId - + " but the target assignment still has " + consumerGroup.targetAssignments().size() + + " but the target assignment still has " + consumerGroup.targetAssignment().size() + " members."); } if (consumerGroup.assignmentEpoch() != -1) { @@ -830,13 +830,13 @@ public void replay( ConsumerGroup consumerGroup = getOrMaybeCreateConsumerGroup(groupId, false); if (value != null) { - consumerGroup.setAssignmentEpoch(value.assignmentEpoch()); + consumerGroup.setTargetAssignmentEpoch(value.assignmentEpoch()); } else { - if (!consumerGroup.targetAssignments().isEmpty()) { + if (!consumerGroup.targetAssignment().isEmpty()) { throw new IllegalStateException("Received a tombstone record to delete target assignment of " + groupId - + " but the assignment still has " + consumerGroup.targetAssignments().size() + " members."); + + " but the assignment still has " + consumerGroup.targetAssignment().size() + " members."); } - consumerGroup.setAssignmentEpoch(-1); + consumerGroup.setTargetAssignmentEpoch(-1); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index e8a353dc24c1f..59c9d666cddba 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -86,7 +86,7 @@ public String toString() { private final TimelineHashMap members; /** - * The number of members per server assignor name. + * The number of members supporting each server assignor name. */ private final TimelineHashMap serverAssignors; @@ -96,26 +96,26 @@ public String toString() { private final TimelineHashMap subscribedTopicNames; /** - * The metadata of the subscribed topics. + * The metadata associated with each subscribed topic name. */ private final TimelineHashMap subscribedTopicMetadata; /** - * The assignment epoch. An assignment epoch smaller than the group epoch means - * that a new assignment is required. The assignment epoch is updated when a new - * assignment is installed. + * The target assignment epoch. An assignment epoch smaller than the group epoch + * means that a new assignment is required. The assignment epoch is updated when + * a new assignment is installed. */ - private final TimelineInteger assignmentEpoch; + private final TimelineInteger targetAssignmentEpoch; /** - * The target assignment. + * The target assignment per member id. */ - private final TimelineHashMap assignments; + private final TimelineHashMap targetAssignment; /** * The current partition epoch maps each topic-partitions to their current epoch where * the epoch is the epoch of their owners. When a member revokes a partition, it removes - * itself from this map. When a member gets a partition, it adds itself to this map. + * its epochs from this map. When a member gets a partition, it adds its epochs to this map. */ private final TimelineHashMap> currentPartitionEpoch; @@ -131,14 +131,12 @@ public ConsumerGroup( this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0); this.subscribedTopicNames = new TimelineHashMap<>(snapshotRegistry, 0); this.subscribedTopicMetadata = new TimelineHashMap<>(snapshotRegistry, 0); - this.assignmentEpoch = new TimelineInteger(snapshotRegistry); - this.assignments = new TimelineHashMap<>(snapshotRegistry, 0); + this.targetAssignmentEpoch = new TimelineInteger(snapshotRegistry); + this.targetAssignment = new TimelineHashMap<>(snapshotRegistry, 0); this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 0); } /** - * The type of this group. - * * @return The group type (Consumer). */ @Override @@ -147,8 +145,6 @@ public GroupType type() { } /** - * The state of this group. - * * @return The current state as a String. */ @Override @@ -157,8 +153,6 @@ public String stateAsString() { } /** - * The group id. - * * @return The group id. */ @Override @@ -167,8 +161,6 @@ public String groupId() { } /** - * The state of this group. - * * @return The current state. */ public ConsumerGroupState state() { @@ -176,8 +168,6 @@ public ConsumerGroupState state() { } /** - * Returns the current group epoch. - * * @return The group epoch. */ public int groupEpoch() { @@ -195,21 +185,19 @@ public void setGroupEpoch(int groupEpoch) { } /** - * Returns the current assignment epoch. - * - * @return The current assignment epoch. + * @return The target assignment epoch. */ public int assignmentEpoch() { - return assignmentEpoch.get(); + return targetAssignmentEpoch.get(); } /** * Sets the assignment epoch. * - * @param assignmentEpoch The new assignment epoch. + * @param targetAssignmentEpoch The new assignment epoch. */ - public void setAssignmentEpoch(int assignmentEpoch) { - this.assignmentEpoch.set(assignmentEpoch); + public void setTargetAssignmentEpoch(int targetAssignmentEpoch) { + this.targetAssignmentEpoch.set(targetAssignmentEpoch); maybeUpdateGroupState(); } @@ -278,8 +266,6 @@ public boolean hasMember(String memberId) { } /** - * Returns the number of members in the group. - * * @return The number of members. */ public int numMembers() { @@ -287,22 +273,20 @@ public int numMembers() { } /** - * Returns the members keyed by their id. - * - * @return An immutable Map containing all the members. + * @return An immutable Map containing all the members keyed by their id. */ public Map members() { return Collections.unmodifiableMap(members); } /** - * Returns the current target assignment of the member. + * Returns the target assignment of the member. * * @return The ConsumerGroupMemberAssignment or an EMPTY one if it does not * exist. */ public Assignment targetAssignment(String memberId) { - return assignments.getOrDefault(memberId, Assignment.EMPTY); + return targetAssignment.getOrDefault(memberId, Assignment.EMPTY); } /** @@ -312,7 +296,7 @@ public Assignment targetAssignment(String memberId) { * @param newTargetAssignment The new target assignment. */ public void updateTargetAssignment(String memberId, Assignment newTargetAssignment) { - assignments.put(memberId, newTargetAssignment); + targetAssignment.put(memberId, newTargetAssignment); } /** @@ -321,16 +305,14 @@ public void updateTargetAssignment(String memberId, Assignment newTargetAssignme * @param memberId The member id. */ public void removeTargetAssignment(String memberId) { - assignments.remove(memberId); + targetAssignment.remove(memberId); } /** - * Returns the target assignments for the entire group keyed by member id. - * - * @return An immutable Map containing all the target assignments. + * @return An immutable Map containing all the target assignment keyed by member id. */ - public Map targetAssignments() { - return Collections.unmodifiableMap(assignments); + public Map targetAssignment() { + return Collections.unmodifiableMap(targetAssignment); } /** @@ -364,7 +346,7 @@ public int currentPartitionEpoch( * * @return An Optional containing the preferred assignor. */ - public Optional preferredServerAssignor( + public Optional computePreferredServerAssignor( ConsumerGroupMember oldMember, ConsumerGroupMember newMember ) { @@ -378,10 +360,17 @@ public Optional preferredServerAssignor( } /** - * Returns the subscription metadata for all the topics whose - * members are subscribed to. - * - * @return An immutable Map containing the subscription metadata. + * @return The preferred assignor for the group. + */ + public Optional preferredServerAssignor() { + return serverAssignors.entrySet().stream() + .max(Map.Entry.comparingByValue()) + .map(Map.Entry::getKey); + } + + /** + * @return An immutable Map containing the subscription metadata for all the topics whose + * members are subscribed to. */ public Map subscriptionMetadata() { return Collections.unmodifiableMap(subscribedTopicMetadata); @@ -440,11 +429,11 @@ public Map computeSubscriptionMetadata( private void maybeUpdateGroupState() { if (members.isEmpty()) { state.set(ConsumerGroupState.EMPTY); - } else if (groupEpoch.get() > assignmentEpoch.get()) { + } else if (groupEpoch.get() > targetAssignmentEpoch.get()) { state.set(ConsumerGroupState.ASSIGNING); } else { for (ConsumerGroupMember member : members.values()) { - if (member.nextMemberEpoch() != assignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) { + if (member.nextMemberEpoch() != targetAssignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) { state.set(ConsumerGroupState.RECONCILING); return; } 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 8be6ef1987f19..245e5e1418419 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 @@ -89,17 +89,12 @@ public class GroupMetadataManagerTest { static class MockPartitionAssignor implements PartitionAssignor { private final String name; - private AssignmentSpec lastSpecReceived = null; private GroupAssignment prepareGroupAssignment = null; MockPartitionAssignor(String name) { this.name = name; } - public AssignmentSpec lastSpecReceived() { - return lastSpecReceived; - } - public void prepareGroupAssignment(GroupAssignment prepareGroupAssignment) { this.prepareGroupAssignment = prepareGroupAssignment; } @@ -111,7 +106,6 @@ public String name() { @Override public GroupAssignment assign(AssignmentSpec assignmentSpec) throws PartitionAssignorException { - lastSpecReceived = assignmentSpec; return prepareGroupAssignment; } } @@ -161,11 +155,6 @@ public ConsumerGroupBuilder withAssignment(String memberId, Map build(TopicsImage topicsImage) { // Add subscription metadata. Map subscriptionMetadata = new HashMap<>(); members.forEach((memberId, member) -> { - member.subscribedTopicNames().forEach(topicName -> - subscriptionMetadata.computeIfAbsent(topicName, __ -> { - TopicImage topicImage = topicsImage.getTopic(topicName); - if (topicImage == null) { - return null; - } else { - return new TopicMetadata( - topicImage.id(), - topicImage.name(), - topicImage.partitions().size() - ); - } - }) - ); + member.subscribedTopicNames().forEach(topicName -> { + TopicImage topicImage = topicsImage.getTopic(topicName); + if (topicImage != null) { + subscriptionMetadata.put(topicName, new TopicMetadata( + topicImage.id(), + topicImage.name(), + topicImage.partitions().size() + )); + } + }); }); + if (!subscriptionMetadata.isEmpty()) { records.add(RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, subscriptionMetadata)); } @@ -223,23 +209,13 @@ public List build(TopicsImage topicsImage) { static class GroupMetadataManagerTestContext { static class Builder { - private LogContext logContext; - private SnapshotRegistry snapshotRegistry; + final private LogContext logContext = new LogContext(); + final private SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext); private TopicsImage topicsImage; private List assignors; private List consumerGroupBuilders = new ArrayList<>(); private int consumerGroupMaxSize = Integer.MAX_VALUE; - public Builder withLogContext(LogContext logContext) { - this.logContext = logContext; - return this; - } - - public Builder withSnapshotRegistry(SnapshotRegistry snapshotRegistry) { - this.snapshotRegistry = snapshotRegistry; - return this; - } - public Builder withTopicsImage(TopicsImage topicsImage) { this.topicsImage = topicsImage; return this; @@ -261,8 +237,6 @@ public Builder withConsumerGroupMaxSize(int consumerGroupMaxSize) { } public GroupMetadataManagerTestContext build() { - if (logContext == null) logContext = new LogContext(); - if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); if (topicsImage == null) topicsImage = TopicsImage.EMPTY; if (assignors == null) assignors = Collections.emptyList(); @@ -439,16 +413,19 @@ public void testConsumerHeartbeatRequestValidation() { .build(); Exception ex; + // GroupId must be present in all requests. ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData())); assertEquals("GroupId can't be empty.", ex.getMessage()); + // RebalanceTimeoutMs must be present in the first request (epoch == 0). ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") .setMemberEpoch(0))); assertEquals("RebalanceTimeoutMs must be provided in first request.", ex.getMessage()); + // TopicPartitions must be present and empty in the first request (epoch == 0). ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") @@ -456,6 +433,7 @@ public void testConsumerHeartbeatRequestValidation() { .setRebalanceTimeoutMs(5000))); assertEquals("TopicPartitions must be empty when (re-)joining.", ex.getMessage()); + // SubscribedTopicNames must be present and empty in the first request (epoch == 0). ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") @@ -464,22 +442,15 @@ public void testConsumerHeartbeatRequestValidation() { .setTopicPartitions(Collections.emptyList()))); assertEquals("SubscribedTopicNames must be set in first request.", ex.getMessage()); - ex = assertThrows(UnsupportedAssignorException.class, () -> context.consumerGroupHeartbeat( - new ConsumerGroupHeartbeatRequestData() - .setGroupId("foo") - .setMemberEpoch(0) - .setRebalanceTimeoutMs(5000) - .setTopicPartitions(Collections.emptyList()) - .setSubscribedTopicNames(Collections.singletonList("foo")) - .setServerAssignor("bar"))); - assertEquals("ServerAssignor bar is not supported. Supported assignors: range.", ex.getMessage()); - + // MemberId must be non-empty in all requests except for the first one where it + // could be empty (epoch != 0). ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") .setMemberEpoch(1))); assertEquals("MemberId can't be empty.", ex.getMessage()); + // InstanceId must be non-empty if provided in all requests. ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") @@ -488,6 +459,7 @@ public void testConsumerHeartbeatRequestValidation() { .setInstanceId(""))); assertEquals("InstanceId can't be empty.", ex.getMessage()); + // RackId must be non-empty if provided in all requests. ex = assertThrows(InvalidRequestException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") @@ -496,6 +468,7 @@ public void testConsumerHeartbeatRequestValidation() { .setRackId(""))); assertEquals("RackId can't be empty.", ex.getMessage()); + // ServerAssignor must exist if provided in all requests. ex = assertThrows(UnsupportedAssignorException.class, () -> context.consumerGroupHeartbeat( new ConsumerGroupHeartbeatRequestData() .setGroupId("foo") @@ -1450,7 +1423,8 @@ public void testReconciliationProcess() { assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, context.consumerGroupMemberState(groupId, memberId3)); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, context.consumerGroupState(groupId)); - // Member 3 heartbeats but without acknowledging the revocation yet. This is basically a no-op. + // Member 3 heartbeats. Member 2 has not acknowledged the revocation of its partition so + // member keeps its current assignment. result = context.consumerGroupHeartbeat(new ConsumerGroupHeartbeatRequestData() .setGroupId(groupId) .setMemberId(memberId3) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index 17b38a31cd2ef..c3ae42b633a22 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -121,7 +121,7 @@ public void testUpdatingMemberUpdatesPartitionEpoch() { assertEquals(-1, consumerGroup.currentPartitionEpoch(zarTopicId, 9)); member = new ConsumerGroupMember.Builder(member) - .setMemberEpoch(10) + .setMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(barTopicId, 1, 2, 3))) .setPartitionsPendingRevocation(mkAssignment( @@ -132,12 +132,12 @@ public void testUpdatingMemberUpdatesPartitionEpoch() { consumerGroup.updateMember(member); - assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 1)); - assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 2)); - assertEquals(10, consumerGroup.currentPartitionEpoch(barTopicId, 3)); - assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 4)); - assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 5)); - assertEquals(10, consumerGroup.currentPartitionEpoch(zarTopicId, 6)); + assertEquals(11, consumerGroup.currentPartitionEpoch(barTopicId, 1)); + assertEquals(11, consumerGroup.currentPartitionEpoch(barTopicId, 2)); + assertEquals(11, consumerGroup.currentPartitionEpoch(barTopicId, 3)); + assertEquals(11, consumerGroup.currentPartitionEpoch(zarTopicId, 4)); + assertEquals(11, consumerGroup.currentPartitionEpoch(zarTopicId, 5)); + assertEquals(11, consumerGroup.currentPartitionEpoch(zarTopicId, 6)); assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 7)); assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 8)); assertEquals(-1, consumerGroup.currentPartitionEpoch(fooTopicId, 9)); @@ -189,6 +189,7 @@ public void testDeletingMemberRemovesPartitionEpoch() { @Test public void testGroupState() { + Uuid fooTopicId = Uuid.randomUuid(); ConsumerGroup consumerGroup = createConsumerGroup("foo"); assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, consumerGroup.state()); @@ -201,6 +202,7 @@ public void testGroupState() { consumerGroup.updateMember(member1); consumerGroup.setGroupEpoch(1); + assertEquals(ConsumerGroupMember.MemberState.STABLE, member1.state()); assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state()); ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") @@ -212,9 +214,10 @@ public void testGroupState() { consumerGroup.updateMember(member2); consumerGroup.setGroupEpoch(2); + assertEquals(ConsumerGroupMember.MemberState.STABLE, member2.state()); assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state()); - consumerGroup.setAssignmentEpoch(2); + consumerGroup.setTargetAssignmentEpoch(2); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); @@ -223,17 +226,38 @@ public void testGroupState() { .setPreviousMemberEpoch(1) .setNextMemberEpoch(2) .build(); + consumerGroup.updateMember(member1); + assertEquals(ConsumerGroupMember.MemberState.STABLE, member1.state()); assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); + // Member 2 is not stable so the group stays in reconciling state. member2 = new ConsumerGroupMember.Builder(member2) .setMemberEpoch(2) .setPreviousMemberEpoch(1) .setNextMemberEpoch(2) + .setPartitionsPendingAssignment(mkAssignment( + mkTopicAssignment(fooTopicId, 0))) .build(); + consumerGroup.updateMember(member2); + assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member2.state()); + assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state()); + + member2 = new ConsumerGroupMember.Builder(member2) + .setMemberEpoch(2) + .setPreviousMemberEpoch(1) + .setNextMemberEpoch(2) + .setAssignedPartitions(mkAssignment( + mkTopicAssignment(fooTopicId, 0))) + .setPartitionsPendingAssignment(Collections.emptyMap()) + .build(); + + consumerGroup.updateMember(member2); + + assertEquals(ConsumerGroupMember.MemberState.STABLE, member2.state()); assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, consumerGroup.state()); consumerGroup.removeMember("member1"); @@ -256,45 +280,58 @@ public void testPreferredServerAssignor() { .setServerAssignorName("uniform") .build(); + // The group is empty so the preferred assignor should be empty. assertEquals( Optional.empty(), - consumerGroup.preferredServerAssignor(null, null) + consumerGroup.preferredServerAssignor() ); + // Member 1 has got an updated assignor but this is not reflected in the group yet so + // we pass the updated member. The assignor should be range. assertEquals( Optional.of("range"), - consumerGroup.preferredServerAssignor(null, member1) + consumerGroup.computePreferredServerAssignor(null, member1) ); + // Update the group with member 1. consumerGroup.updateMember(member1); + // Member 1 is in the group so the assignor should be range. assertEquals( Optional.of("range"), - consumerGroup.preferredServerAssignor(null, null) + consumerGroup.preferredServerAssignor() ); + // Member 1 has been removed but this is not reflected in the group yet so + // we pass the removed member. The assignor should be range. assertEquals( Optional.empty(), - consumerGroup.preferredServerAssignor(member1, null) + consumerGroup.computePreferredServerAssignor(member1, null) ); + // Member 2 has got an updated assignor but this is not reflected in the group yet so + // we pass the updated member. The assignor should be range. assertEquals( Optional.of("range"), - consumerGroup.preferredServerAssignor(null, member2) + consumerGroup.computePreferredServerAssignor(null, member2) ); + // Update the group with member 2. consumerGroup.updateMember(member2); + // Member 1 and 2 are in the group so the assignor should be range. assertEquals( Optional.of("range"), - consumerGroup.preferredServerAssignor(null, null) + consumerGroup.preferredServerAssignor() ); + // Update the group with member 3. consumerGroup.updateMember(member3); + // Member 1, 2 and 3 are in the group so the assignor should be range. assertEquals( Optional.of("range"), - consumerGroup.preferredServerAssignor(null, null) + consumerGroup.preferredServerAssignor() ); // Members without assignors @@ -308,37 +345,44 @@ public void testPreferredServerAssignor() { .setServerAssignorName(null) .build(); - - Optional assignor = consumerGroup.preferredServerAssignor(member1, updatedMember1); + // Member 1 has removed it assignor but this is not reflected in the group yet so + // we pass the updated member. The assignor should be range or uniform. + Optional assignor = consumerGroup.computePreferredServerAssignor(member1, updatedMember1); assertTrue(assignor.equals(Optional.of("range")) || assignor.equals(Optional.of("uniform"))); + // Update the group. consumerGroup.updateMember(updatedMember1); - assignor = consumerGroup.preferredServerAssignor(member1, updatedMember1); - assertTrue(assignor.equals(Optional.of("range")) || assignor.equals(Optional.of("uniform"))); - + // Member 2 has removed it assignor but this is not reflected in the group yet so + // we pass the updated member. The assignor should be range or uniform. assertEquals( Optional.of("uniform"), - consumerGroup.preferredServerAssignor(member2, updatedMember2) + consumerGroup.computePreferredServerAssignor(member2, updatedMember2) ); + // Update the group. consumerGroup.updateMember(updatedMember2); + // Only member 3 is left in the group so the assignor should be uniform. assertEquals( Optional.of("uniform"), - consumerGroup.preferredServerAssignor(null, null) + consumerGroup.preferredServerAssignor() ); + // Member 3 has removed it assignor but this is not reflected in the group yet so + // we pass the updated member. The assignor should be empty. assertEquals( Optional.empty(), - consumerGroup.preferredServerAssignor(member3, updatedMember3) + consumerGroup.computePreferredServerAssignor(member3, updatedMember3) ); + // Update the group. consumerGroup.updateMember(updatedMember3); + // The group is empty so the assignor should be empty as well. assertEquals( Optional.empty(), - consumerGroup.preferredServerAssignor(null, null) + consumerGroup.preferredServerAssignor() ); } @@ -376,7 +420,7 @@ public void testUpdateSubscriptionMetadata() { ) ); - // Adding member1. + // Compute while taking into account member 1. assertEquals( mkMap( mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)) @@ -403,7 +447,7 @@ public void testUpdateSubscriptionMetadata() { ) ); - // Removing member1 results in an empty map. + // Compute while taking into account removal of member 1. assertEquals( Collections.emptyMap(), consumerGroup.computeSubscriptionMetadata( @@ -413,7 +457,7 @@ public void testUpdateSubscriptionMetadata() { ) ); - // Adding member2 adds bar. + // Compute while taking into account member 2. assertEquals( mkMap( mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)), @@ -442,7 +486,7 @@ public void testUpdateSubscriptionMetadata() { ) ); - // Removing member2 results in returning foo. + // Compute while taking into account removal of member 2. assertEquals( mkMap( mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)) @@ -466,7 +510,7 @@ public void testUpdateSubscriptionMetadata() { ) ); - // Adding member3 adds zar. + // Compute while taking into account member 3. assertEquals( mkMap( mkEntry("foo", new TopicMetadata(fooTopicId, "foo", 1)), @@ -492,7 +536,7 @@ public void testUpdateSubscriptionMetadata() { ), consumerGroup.computeSubscriptionMetadata( null, - member3, + null, image ) ); From 09a7d28112201b48bb5a4074383ca9cd2a28ac76 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 25 May 2023 11:56:15 +0200 Subject: [PATCH 13/16] clean up --- .../group/GroupMetadataManagerTest.java | 47 ++++++++++--------- 1 file changed, 24 insertions(+), 23 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 245e5e1418419..628ef29759427 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 @@ -39,6 +39,7 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.coordinator.group.assignor.AssignmentSpec; import org.apache.kafka.coordinator.group.assignor.GroupAssignment; +import org.apache.kafka.coordinator.group.assignor.MemberAssignment; import org.apache.kafka.coordinator.group.assignor.PartitionAssignor; import org.apache.kafka.coordinator.group.assignor.PartitionAssignorException; import org.apache.kafka.coordinator.group.consumer.Assignment; @@ -679,7 +680,7 @@ public void testMemberJoinsEmptyConsumerGroup() { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - Collections.singletonMap(memberId, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + Collections.singletonMap(memberId, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), mkTopicAssignment(barTopicId, 0, 1, 2) ))) @@ -783,7 +784,7 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - Collections.singletonMap(memberId, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + Collections.singletonMap(memberId, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), mkTopicAssignment(barTopicId, 0, 1, 2) ))) @@ -903,17 +904,17 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { .build(); assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() { + new HashMap() { { - put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId1, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), mkTopicAssignment(barTopicId, 0) ))); - put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId2, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 2, 3), mkTopicAssignment(barTopicId, 1) ))); - put(memberId3, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId3, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5), mkTopicAssignment(barTopicId, 2) ))); @@ -1139,17 +1140,17 @@ public void testReconciliationProcess() { // Prepare new assignment for the group. assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() { + new HashMap() { { - put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId1, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), mkTopicAssignment(barTopicId, 0) ))); - put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId2, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 2, 3), mkTopicAssignment(barTopicId, 2) ))); - put(memberId3, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId3, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5), mkTopicAssignment(barTopicId, 1) ))); @@ -1503,12 +1504,12 @@ public void testReconciliationProcess() { .setHeartbeatIntervalMs(5000) .setAssignment(new ConsumerGroupHeartbeatResponseData.Assignment() .setAssignedTopicPartitions(Arrays.asList( - new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(barTopicId) - .setPartitions(Arrays.asList(1)), new ConsumerGroupHeartbeatResponseData.TopicPartitions() .setTopicId(fooTopicId) - .setPartitions(Arrays.asList(4, 5))))), + .setPartitions(Arrays.asList(4, 5)), + new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(barTopicId) + .setPartitions(Arrays.asList(1))))), result.response() ); @@ -1518,8 +1519,8 @@ public void testReconciliationProcess() { .setPreviousMemberEpoch(11) .setNextMemberEpoch(11) .setAssignedPartitions(mkAssignment( - mkTopicAssignment(barTopicId, 1), - mkTopicAssignment(fooTopicId, 4, 5))) + mkTopicAssignment(fooTopicId, 4, 5), + mkTopicAssignment(barTopicId, 1))) .build())), result.records() ); @@ -1568,12 +1569,12 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { // Prepare new assignment for the group. assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() { + new HashMap() { { - put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId1, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1) ))); - put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId2, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 2) ))); } @@ -1654,15 +1655,15 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { // Prepare new assignment for the group. assignor.prepareGroupAssignment(new GroupAssignment( - new HashMap() { + new HashMap() { { - put(memberId1, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId1, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0) ))); - put(memberId2, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId2, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 2) ))); - put(memberId3, new org.apache.kafka.coordinator.group.assignor.MemberAssignment(mkAssignment( + put(memberId3, new MemberAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1) ))); } From 61bc3db5a631d11f123042b65ab0bb4dcf73e7b2 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 25 May 2023 15:11:02 +0200 Subject: [PATCH 14/16] fix flaky tests --- .../group/GroupMetadataManagerTest.java | 115 +++++++++++++++--- 1 file changed, 98 insertions(+), 17 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 628ef29759427..c2e87e990a95c 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 @@ -64,6 +64,7 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; +import org.opentest4j.AssertionFailedError; import java.net.InetAddress; import java.util.ArrayList; @@ -745,7 +746,7 @@ public void testMemberJoinsEmptyConsumerGroup() { RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember) ); - assertEquals(expectedRecords, result.records()); + assertRecordsEquals(expectedRecords, result.records()); } @Test @@ -844,7 +845,7 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember) ); - assertEquals(expectedRecords, result.records()); + assertRecordsEquals(expectedRecords, result.records()); } @Test @@ -983,9 +984,9 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { RecordHelpers.newCurrentAssignmentRecord(groupId, expectedMember3) ); - assertEquals(expectedRecords.subList(0, 2), result.records().subList(0, 2)); + assertRecordsEquals(expectedRecords.subList(0, 2), result.records().subList(0, 2)); assertUnorderedListEquals(expectedRecords.subList(2, 5), result.records().subList(2, 5)); - assertEquals(expectedRecords.subList(5, 7), result.records().subList(5, 7)); + assertRecordsEquals(expectedRecords.subList(5, 7), result.records().subList(5, 7)); } @Test @@ -1078,7 +1079,7 @@ public void testLeavingMemberBumpsGroupEpoch() { RecordHelpers.newGroupEpochRecord(groupId, 11) ); - assertEquals(expectedRecords, result.records()); + assertRecordsEquals(expectedRecords, result.records()); } @Test @@ -1197,7 +1198,7 @@ public void testReconciliationProcess() { // We only check the last record as the subscription/target assignment updates are // already covered by other tests. - assertEquals( + assertRecordEquals( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(0) @@ -1237,7 +1238,7 @@ public void testReconciliationProcess() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) @@ -1280,7 +1281,7 @@ public void testReconciliationProcess() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) @@ -1351,7 +1352,7 @@ public void testReconciliationProcess() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(11) .setPreviousMemberEpoch(10) @@ -1408,7 +1409,7 @@ public void testReconciliationProcess() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(11) @@ -1476,7 +1477,7 @@ public void testReconciliationProcess() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(11) .setPreviousMemberEpoch(10) @@ -1513,7 +1514,7 @@ public void testReconciliationProcess() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(11) @@ -1606,7 +1607,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { result.response() ); - assertEquals( + assertRecordEquals( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(11) .setPreviousMemberEpoch(0) @@ -1638,7 +1639,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) @@ -1695,7 +1696,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { result.response() ); - assertEquals( + assertRecordEquals( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(12) .setPreviousMemberEpoch(0) @@ -1727,7 +1728,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) @@ -1761,7 +1762,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { result.response() ); - assertEquals(Collections.singletonList( + assertRecordsEquals(Collections.singletonList( RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(12) .setPreviousMemberEpoch(11) @@ -1989,4 +1990,84 @@ private Map> fromAssignment( }); return assigmentMap; } + + private void assertRecordsEquals( + List expectedRecords, + List actualRecords + ) { + try { + assertEquals(expectedRecords.size(), actualRecords.size()); + + for (int i = 0; i < expectedRecords.size(); i++) { + Record expectedRecord = expectedRecords.get(i); + Record actualRecord = actualRecords.get(i); + assertRecordEquals(expectedRecord, actualRecord); + } + } catch (AssertionFailedError e) { + assertionFailure() + .expected(expectedRecords) + .actual(actualRecords) + .buildAndThrow(); + } + } + + private void assertRecordEquals( + Record expected, + Record actual + ) { + try { + assertApiMessageAndVersionEquals(expected.key(), actual.key()); + assertApiMessageAndVersionEquals(expected.value(), actual.value()); + } catch (AssertionFailedError e) { + assertionFailure() + .expected(expected) + .actual(actual) + .buildAndThrow(); + } + } + + private void assertApiMessageAndVersionEquals( + ApiMessageAndVersion expected, + ApiMessageAndVersion actual + ) { + if (expected == actual) return; + + assertEquals(expected.version(), actual.version()); + + if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) { + // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not + // always guaranteed. Therefore, we need a special comparator. + ConsumerGroupCurrentMemberAssignmentValue expectedValue = + (ConsumerGroupCurrentMemberAssignmentValue) expected.message(); + ConsumerGroupCurrentMemberAssignmentValue actualValue = + (ConsumerGroupCurrentMemberAssignmentValue) actual.message(); + + assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch()); + assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch()); + assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch()); + assertEquals(expectedValue.error(), actualValue.error()); + assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion()); + assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes()); + + // We transform those to Maps before comparing them. + assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()), + fromTopicPartitions(actualValue.assignedPartitions())); + assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()), + fromTopicPartitions(actualValue.partitionsPendingRevocation())); + assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()), + fromTopicPartitions(actualValue.partitionsPendingAssignment())); + } else { + assertEquals(expected.message(), actual.message()); + } + } + + private Map> fromTopicPartitions( + List assignment + ) { + Map> assignmentMap = new HashMap<>(); + assignment.forEach(topicPartitions -> { + assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions())); + }); + return assignmentMap; + } } From 172515038e30f8fd82734e2ebaff0115d88e29dc Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 26 May 2023 09:02:44 +0200 Subject: [PATCH 15/16] rename nextMemberEpoch to targetMemberEpoch --- .../group/GroupMetadataManager.java | 4 +- .../coordinator/group/RecordHelpers.java | 2 +- .../group/consumer/ConsumerGroup.java | 2 +- .../group/consumer/ConsumerGroupMember.java | 32 +++++------ .../consumer/CurrentAssignmentBuilder.java | 10 ++-- .../group/GroupMetadataManagerTest.java | 56 +++++++++---------- .../coordinator/group/RecordHelpersTest.java | 2 +- .../consumer/ConsumerGroupMemberTest.java | 12 ++-- .../group/consumer/ConsumerGroupTest.java | 10 ++-- .../CurrentAssignmentBuilderTest.java | 44 +++++++-------- 10 files changed, 87 insertions(+), 87 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 0528e4816ae3f..e85c31bcf19aa 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 @@ -566,7 +566,7 @@ private Result consumerGroupHeartbeat( // 3. Reconcile the member's assignment with the target assignment. This is only required if // the member is not stable or if a new target assignment has been installed. boolean assignmentUpdated = false; - if (updatedMember.state() != ConsumerGroupMember.MemberState.STABLE || updatedMember.nextMemberEpoch() != targetAssignmentEpoch) { + if (updatedMember.state() != ConsumerGroupMember.MemberState.STABLE || updatedMember.targetMemberEpoch() != targetAssignmentEpoch) { ConsumerGroupMember prevMember = updatedMember; updatedMember = new CurrentAssignmentBuilder(updatedMember) .withTargetAssignment(targetAssignmentEpoch, targetAssignment) @@ -865,7 +865,7 @@ public void replay( ConsumerGroupMember newMember = new ConsumerGroupMember.Builder(oldMember) .setMemberEpoch(-1) .setPreviousMemberEpoch(-1) - .setNextMemberEpoch(-1) + .setTargetMemberEpoch(-1) .setAssignedPartitions(Collections.emptyMap()) .setPartitionsPendingRevocation(Collections.emptyMap()) .setPartitionsPendingAssignment(Collections.emptyMap()) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java index bf6cd62a90fbf..fc041e3351fef 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java @@ -330,7 +330,7 @@ public static Record newCurrentAssignmentRecord( new ConsumerGroupCurrentMemberAssignmentValue() .setMemberEpoch(member.memberEpoch()) .setPreviousMemberEpoch(member.previousMemberEpoch()) - .setTargetMemberEpoch(member.nextMemberEpoch()) + .setTargetMemberEpoch(member.targetMemberEpoch()) .setAssignedPartitions(toTopicPartitions(member.assignedPartitions())) .setPartitionsPendingRevocation(toTopicPartitions(member.partitionsPendingRevocation())) .setPartitionsPendingAssignment(toTopicPartitions(member.partitionsPendingAssignment())), diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java index 59c9d666cddba..9df00733b301c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroup.java @@ -433,7 +433,7 @@ private void maybeUpdateGroupState() { state.set(ConsumerGroupState.ASSIGNING); } else { for (ConsumerGroupMember member : members.values()) { - if (member.nextMemberEpoch() != targetAssignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) { + if (member.targetMemberEpoch() != targetAssignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) { state.set(ConsumerGroupState.RECONCILING); return; } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java index 958f1ae7cdf44..729edf8d06bf8 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMember.java @@ -48,7 +48,7 @@ public static class Builder { private final String memberId; private int memberEpoch = 0; private int previousMemberEpoch = -1; - private int nextMemberEpoch = 0; + private int targetMemberEpoch = 0; private String instanceId = null; private String rackId = null; private int rebalanceTimeoutMs = -1; @@ -72,7 +72,7 @@ public Builder(ConsumerGroupMember member) { this.memberId = member.memberId; this.memberEpoch = member.memberEpoch; this.previousMemberEpoch = member.previousMemberEpoch; - this.nextMemberEpoch = member.nextMemberEpoch; + this.targetMemberEpoch = member.targetMemberEpoch; this.instanceId = member.instanceId; this.rackId = member.rackId; this.rebalanceTimeoutMs = member.rebalanceTimeoutMs; @@ -97,8 +97,8 @@ public Builder setPreviousMemberEpoch(int previousMemberEpoch) { return this; } - public Builder setNextMemberEpoch(int nextMemberEpoch) { - this.nextMemberEpoch = nextMemberEpoch; + public Builder setTargetMemberEpoch(int targetMemberEpoch) { + this.targetMemberEpoch = targetMemberEpoch; return this; } @@ -217,7 +217,7 @@ public Builder updateWith(ConsumerGroupMemberMetadataValue record) { public Builder updateWith(ConsumerGroupCurrentMemberAssignmentValue record) { setMemberEpoch(record.memberEpoch()); setPreviousMemberEpoch(record.previousMemberEpoch()); - setNextMemberEpoch(record.targetMemberEpoch()); + setTargetMemberEpoch(record.targetMemberEpoch()); setAssignedPartitions(assignmentFromTopicPartitions(record.assignedPartitions())); setPartitionsPendingRevocation(assignmentFromTopicPartitions(record.partitionsPendingRevocation())); setPartitionsPendingAssignment(assignmentFromTopicPartitions(record.partitionsPendingAssignment())); @@ -246,7 +246,7 @@ public ConsumerGroupMember build() { memberId, memberEpoch, previousMemberEpoch, - nextMemberEpoch, + targetMemberEpoch, instanceId, rackId, rebalanceTimeoutMs, @@ -305,7 +305,7 @@ public String toString() { * assignment epoch used to compute the current assigned, * revoking and assigning partitions. */ - private final int nextMemberEpoch; + private final int targetMemberEpoch; /** * The instance id provided by the member. @@ -378,7 +378,7 @@ private ConsumerGroupMember( String memberId, int memberEpoch, int previousMemberEpoch, - int nextMemberEpoch, + int targetMemberEpoch, String instanceId, String rackId, int rebalanceTimeoutMs, @@ -396,7 +396,7 @@ private ConsumerGroupMember( this.memberId = memberId; this.memberEpoch = memberEpoch; this.previousMemberEpoch = previousMemberEpoch; - this.nextMemberEpoch = nextMemberEpoch; + this.targetMemberEpoch = targetMemberEpoch; this.instanceId = instanceId; this.rackId = rackId; this.rebalanceTimeoutMs = rebalanceTimeoutMs; @@ -434,10 +434,10 @@ public int previousMemberEpoch() { } /** - * @return The next member epoch. + * @return The target member epoch. */ - public int nextMemberEpoch() { - return nextMemberEpoch; + public int targetMemberEpoch() { + return targetMemberEpoch; } /** @@ -537,7 +537,7 @@ public Map> partitionsPendingAssignment() { public String currentAssignmentSummary() { return "CurrentAssignment(memberEpoch=" + memberEpoch + ", previousMemberEpoch=" + previousMemberEpoch + - ", nextMemberEpoch=" + nextMemberEpoch + + ", targetMemberEpoch=" + targetMemberEpoch + ", state=" + state + ", assignedPartitions=" + assignedPartitions + ", partitionsPendingRevocation=" + partitionsPendingRevocation + @@ -552,7 +552,7 @@ public boolean equals(Object o) { ConsumerGroupMember that = (ConsumerGroupMember) o; return memberEpoch == that.memberEpoch && previousMemberEpoch == that.previousMemberEpoch - && nextMemberEpoch == that.nextMemberEpoch + && targetMemberEpoch == that.targetMemberEpoch && rebalanceTimeoutMs == that.rebalanceTimeoutMs && Objects.equals(memberId, that.memberId) && Objects.equals(instanceId, that.instanceId) @@ -573,7 +573,7 @@ public int hashCode() { int result = memberId != null ? memberId.hashCode() : 0; result = 31 * result + memberEpoch; result = 31 * result + previousMemberEpoch; - result = 31 * result + nextMemberEpoch; + result = 31 * result + targetMemberEpoch; result = 31 * result + Objects.hashCode(instanceId); result = 31 * result + Objects.hashCode(rackId); result = 31 * result + rebalanceTimeoutMs; @@ -595,7 +595,7 @@ public String toString() { "memberId='" + memberId + '\'' + ", memberEpoch=" + memberEpoch + ", previousMemberEpoch=" + previousMemberEpoch + - ", nextMemberEpoch=" + nextMemberEpoch + + ", targetMemberEpoch=" + targetMemberEpoch + ", instanceId='" + instanceId + '\'' + ", rackId='" + rackId + '\'' + ", rebalanceTimeoutMs=" + rebalanceTimeoutMs + diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java index 6a255ae8e53a9..fce5b8a85bd41 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilder.java @@ -172,7 +172,7 @@ public CurrentAssignmentBuilder withOwnedTopicPartitions( public ConsumerGroupMember build() { // A new target assignment has been installed, we need to restart // the reconciliation loop from the beginning. - if (targetAssignmentEpoch != member.nextMemberEpoch()) { + if (targetAssignmentEpoch != member.targetMemberEpoch()) { return transitionToNewTargetAssignmentState(); } @@ -258,7 +258,7 @@ private ConsumerGroupMember transitionToNewTargetAssignmentState() { .setAssignedPartitions(newAssignedPartitions) .setPartitionsPendingRevocation(newPartitionsPendingRevocation) .setPartitionsPendingAssignment(newPartitionsPendingAssignment) - .setNextMemberEpoch(targetAssignmentEpoch) + .setTargetMemberEpoch(targetAssignmentEpoch) .build(); } else { if (!newPartitionsPendingAssignment.isEmpty()) { @@ -277,7 +277,7 @@ private ConsumerGroupMember transitionToNewTargetAssignmentState() { .setPartitionsPendingAssignment(newPartitionsPendingAssignment) .setPreviousMemberEpoch(member.memberEpoch()) .setMemberEpoch(targetAssignmentEpoch) - .setNextMemberEpoch(targetAssignmentEpoch) + .setTargetMemberEpoch(targetAssignmentEpoch) .build(); } } @@ -311,7 +311,7 @@ private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() { .setPartitionsPendingAssignment(newPartitionsPendingAssignment) .setPreviousMemberEpoch(member.memberEpoch()) .setMemberEpoch(targetAssignmentEpoch) - .setNextMemberEpoch(targetAssignmentEpoch) + .setTargetMemberEpoch(targetAssignmentEpoch) .build(); } else { return member; @@ -340,7 +340,7 @@ private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() { .setPartitionsPendingAssignment(newPartitionsPendingAssignment) .setPreviousMemberEpoch(member.memberEpoch()) .setMemberEpoch(targetAssignmentEpoch) - .setNextMemberEpoch(targetAssignmentEpoch) + .setTargetMemberEpoch(targetAssignmentEpoch) .build(); } else { return member; 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 c2e87e990a95c..3546ff3158229 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 @@ -593,7 +593,7 @@ public void testConsumerGroupMemberEpochValidation() { ConsumerGroupMember member = new ConsumerGroupMember.Builder(memberId) .setMemberEpoch(100) .setPreviousMemberEpoch(99) - .setNextMemberEpoch(100) + .setTargetMemberEpoch(100) .setRebalanceTimeoutMs(5000) .setClientId("client") .setClientHost("localhost/127.0.0.1") @@ -720,7 +720,7 @@ public void testMemberJoinsEmptyConsumerGroup() { ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setNextMemberEpoch(1) + .setTargetMemberEpoch(1) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -771,7 +771,7 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { .withMember(new ConsumerGroupMember.Builder(memberId) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo")) @@ -818,7 +818,7 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { ConsumerGroupMember expectedMember = new ConsumerGroupMember.Builder(memberId) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -872,7 +872,7 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { .withMember(new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -885,7 +885,7 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { .withMember(new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -954,7 +954,7 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() { ConsumerGroupMember expectedMember3 = new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(0) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1017,7 +1017,7 @@ public void testLeavingMemberBumpsGroupEpoch() { .withMember(new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setSubscribedTopicNames(Arrays.asList("foo", "bar")) @@ -1029,7 +1029,7 @@ public void testLeavingMemberBumpsGroupEpoch() { .withMember(new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") // Use zar only here to ensure that metadata needs to be recomputed. @@ -1107,7 +1107,7 @@ public void testReconciliationProcess() { .withMember(new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1120,7 +1120,7 @@ public void testReconciliationProcess() { .withMember(new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1202,7 +1202,7 @@ public void testReconciliationProcess() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(0) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setPartitionsPendingAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5), mkTopicAssignment(barTopicId, 1))) @@ -1242,7 +1242,7 @@ public void testReconciliationProcess() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), mkTopicAssignment(barTopicId, 0))) @@ -1285,7 +1285,7 @@ public void testReconciliationProcess() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 3), mkTopicAssignment(barTopicId, 2))) @@ -1356,7 +1356,7 @@ public void testReconciliationProcess() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), mkTopicAssignment(barTopicId, 0))) @@ -1413,7 +1413,7 @@ public void testReconciliationProcess() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(barTopicId, 1))) .setPartitionsPendingAssignment(mkAssignment( @@ -1481,7 +1481,7 @@ public void testReconciliationProcess() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 2, 3), mkTopicAssignment(barTopicId, 2))) @@ -1518,7 +1518,7 @@ public void testReconciliationProcess() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 4, 5), mkTopicAssignment(barTopicId, 1))) @@ -1552,7 +1552,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { .withMember(new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1611,7 +1611,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(11) .setPreviousMemberEpoch(0) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setPartitionsPendingAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 2))) .build()), @@ -1643,7 +1643,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0, 1))) .setPartitionsPendingRevocation(mkAssignment( @@ -1700,7 +1700,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId3) .setMemberEpoch(12) .setPreviousMemberEpoch(0) - .setNextMemberEpoch(12) + .setTargetMemberEpoch(12) .setPartitionsPendingAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 1))) .build()), @@ -1732,7 +1732,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(12) + .setTargetMemberEpoch(12) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0))) .setPartitionsPendingRevocation(mkAssignment( @@ -1766,7 +1766,7 @@ public void testReconciliationRestartsWhenNewTargetAssignmentIsInstalled() { RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(12) .setPreviousMemberEpoch(11) - .setNextMemberEpoch(12) + .setTargetMemberEpoch(12) .setPartitionsPendingAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 2))) .build())), @@ -1802,7 +1802,7 @@ public void testNewMemberIsRejectedWithMaximumMembersIsReached() { .withMember(new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1815,7 +1815,7 @@ public void testNewMemberIsRejectedWithMaximumMembersIsReached() { .withMember(new ConsumerGroupMember.Builder(memberId2) .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setClientId("client") .setClientHost("localhost/127.0.0.1") .setRebalanceTimeoutMs(5000) @@ -1877,7 +1877,7 @@ public void testConsumerGroupStates() { context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2))) .setPartitionsPendingAssignment(mkAssignment(mkTopicAssignment(fooTopicId, 3))) .build())); @@ -1887,7 +1887,7 @@ public void testConsumerGroupStates() { context.replay(RecordHelpers.newCurrentAssignmentRecord(groupId, new ConsumerGroupMember.Builder(memberId1) .setMemberEpoch(11) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment(mkTopicAssignment(fooTopicId, 1, 2, 3))) .build())); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java index 40b6ddaedcc8c..cfa2d600f7c33 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java @@ -378,7 +378,7 @@ public void testNewCurrentAssignmentRecord() { new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(22) .setPreviousMemberEpoch(21) - .setNextMemberEpoch(23) + .setTargetMemberEpoch(23) .setAssignedPartitions(assigned) .setPartitionsPendingRevocation(revoking) .setPartitionsPendingAssignment(assigning) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java index e98a895d2beb8..13ac57bb2b06b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupMemberTest.java @@ -42,7 +42,7 @@ public void testNewMember() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -71,7 +71,7 @@ public void testNewMember() { assertEquals("member-id", member.memberId()); assertEquals(10, member.memberEpoch()); assertEquals(9, member.previousMemberEpoch()); - assertEquals(11, member.nextMemberEpoch()); + assertEquals(11, member.targetMemberEpoch()); assertEquals("instance-id", member.instanceId()); assertEquals("rack-id", member.rackId()); assertEquals("client-id", member.clientId()); @@ -105,7 +105,7 @@ public void testEquals() { ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -134,7 +134,7 @@ public void testEquals() { ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -172,7 +172,7 @@ public void testUpdateMember() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id") .setMemberEpoch(10) .setPreviousMemberEpoch(9) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setInstanceId("instance-id") .setRackId("rack-id") .setRebalanceTimeoutMs(5000) @@ -299,7 +299,7 @@ public void testUpdateWithConsumerGroupCurrentMemberAssignmentValue() { assertEquals(10, member.memberEpoch()); assertEquals(9, member.previousMemberEpoch()); - assertEquals(11, member.nextMemberEpoch()); + assertEquals(11, member.targetMemberEpoch()); assertEquals(mkAssignment(mkTopicAssignment(topicId1, 0, 1, 2)), member.assignedPartitions()); assertEquals(mkAssignment(mkTopicAssignment(topicId2, 3, 4, 5)), member.partitionsPendingRevocation()); assertEquals(mkAssignment(mkTopicAssignment(topicId3, 6, 7, 8)), member.partitionsPendingAssignment()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java index c3ae42b633a22..2454188ed946a 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/ConsumerGroupTest.java @@ -196,7 +196,7 @@ public void testGroupState() { ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1") .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setNextMemberEpoch(1) + .setTargetMemberEpoch(1) .build(); consumerGroup.updateMember(member1); @@ -208,7 +208,7 @@ public void testGroupState() { ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2") .setMemberEpoch(1) .setPreviousMemberEpoch(0) - .setNextMemberEpoch(1) + .setTargetMemberEpoch(1) .build(); consumerGroup.updateMember(member2); @@ -224,7 +224,7 @@ public void testGroupState() { member1 = new ConsumerGroupMember.Builder(member1) .setMemberEpoch(2) .setPreviousMemberEpoch(1) - .setNextMemberEpoch(2) + .setTargetMemberEpoch(2) .build(); consumerGroup.updateMember(member1); @@ -236,7 +236,7 @@ public void testGroupState() { member2 = new ConsumerGroupMember.Builder(member2) .setMemberEpoch(2) .setPreviousMemberEpoch(1) - .setNextMemberEpoch(2) + .setTargetMemberEpoch(2) .setPartitionsPendingAssignment(mkAssignment( mkTopicAssignment(fooTopicId, 0))) .build(); @@ -249,7 +249,7 @@ public void testGroupState() { member2 = new ConsumerGroupMember.Builder(member2) .setMemberEpoch(2) .setPreviousMemberEpoch(1) - .setNextMemberEpoch(2) + .setTargetMemberEpoch(2) .setAssignedPartitions(mkAssignment( mkTopicAssignment(fooTopicId, 0))) .setPartitionsPendingAssignment(Collections.emptyMap()) diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java index bbe5cc5e0968e..037a6ccbcd463 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/consumer/CurrentAssignmentBuilderTest.java @@ -44,7 +44,7 @@ public void testTransitionFromNewTargetToRevoke() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6))) @@ -65,7 +65,7 @@ public void testTransitionFromNewTargetToRevoke() { assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(10, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6) @@ -88,7 +88,7 @@ public void testTransitionFromNewTargetToAssigning() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6))) @@ -109,7 +109,7 @@ public void testTransitionFromNewTargetToAssigning() { assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6) @@ -129,7 +129,7 @@ public void testTransitionFromNewTargetToStable() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(10) + .setTargetMemberEpoch(10) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6))) @@ -150,7 +150,7 @@ public void testTransitionFromNewTargetToStable() { assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6) @@ -179,7 +179,7 @@ public void testTransitionFromRevokeToRevoke( ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) @@ -207,7 +207,7 @@ public void testTransitionFromRevokeToRevoke( assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(10, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6) @@ -230,7 +230,7 @@ public void testTransitionFromRevokeToAssigning() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) @@ -260,7 +260,7 @@ public void testTransitionFromRevokeToAssigning() { assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6) @@ -280,7 +280,7 @@ public void testTransitionFromRevokeToStable() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) @@ -310,7 +310,7 @@ public void testTransitionFromRevokeToStable() { assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 3, 4, 5), mkTopicAssignment(topicId2, 6, 7, 8) @@ -327,7 +327,7 @@ public void testTransitionFromRevokeToStableWhenPartitionsPendingRevocationAreRe ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) @@ -358,7 +358,7 @@ public void testTransitionFromRevokeToStableWhenPartitionsPendingRevocationAreRe assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(12, updatedMember.memberEpoch()); - assertEquals(12, updatedMember.nextMemberEpoch()); + assertEquals(12, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), mkTopicAssignment(topicId2, 4, 5, 6) @@ -375,7 +375,7 @@ public void testTransitionFromAssigningToAssigning() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(11) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) @@ -404,7 +404,7 @@ public void testTransitionFromAssigningToAssigning() { assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 3, 4, 5), mkTopicAssignment(topicId2, 6) @@ -423,7 +423,7 @@ public void testTransitionFromAssigningToStable() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(11) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) @@ -447,7 +447,7 @@ public void testTransitionFromAssigningToStable() { assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 3, 4, 5), mkTopicAssignment(topicId2, 6, 7, 8) @@ -464,7 +464,7 @@ public void testTransitionFromStableToStable() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(11) .setPreviousMemberEpoch(11) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3, 4, 5), mkTopicAssignment(topicId2, 6, 7, 8))) @@ -485,7 +485,7 @@ public void testTransitionFromStableToStable() { assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state()); assertEquals(11, updatedMember.previousMemberEpoch()); assertEquals(11, updatedMember.memberEpoch()); - assertEquals(11, updatedMember.nextMemberEpoch()); + assertEquals(11, updatedMember.targetMemberEpoch()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 3, 4, 5), mkTopicAssignment(topicId2, 6, 7, 8) @@ -502,7 +502,7 @@ public void testNewTargetRestartReconciliation() { ConsumerGroupMember member = new ConsumerGroupMember.Builder("member") .setMemberEpoch(10) .setPreviousMemberEpoch(10) - .setNextMemberEpoch(11) + .setTargetMemberEpoch(11) .setAssignedPartitions(mkAssignment( mkTopicAssignment(topicId1, 3), mkTopicAssignment(topicId2, 6))) @@ -529,7 +529,7 @@ public void testNewTargetRestartReconciliation() { assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state()); assertEquals(10, updatedMember.previousMemberEpoch()); assertEquals(10, updatedMember.memberEpoch()); - assertEquals(12, updatedMember.nextMemberEpoch()); + assertEquals(12, updatedMember.targetMemberEpoch()); assertEquals(Collections.emptyMap(), updatedMember.assignedPartitions()); assertEquals(mkAssignment( mkTopicAssignment(topicId1, 1, 2, 3), From fb3b88ce5db1890d4886b9e715c9e677c4511767 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 26 May 2023 09:08:27 +0200 Subject: [PATCH 16/16] small fixes --- .../group/GroupMetadataManager.java | 4 ++-- .../group/GroupMetadataManagerTest.java | 18 +++++++++--------- 2 files changed, 11 insertions(+), 11 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 e85c31bcf19aa..c05f7e3d9a51a 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 @@ -351,7 +351,7 @@ private boolean isSubset( /** * Checks whether the consumer group can accept a new member or not based on the - * max group side defined. + * max group size defined. * * @param group The consumer group. * @param memberId The member id. @@ -532,7 +532,7 @@ private Result consumerGroupHeartbeat( } // 2. Update the target assignment if the group epoch is larger than the target assignment epoch. The - // delta between the current and the new target assignment is persisted to the partition. + // delta between the existing and the new target assignment is persisted to the partition. int targetAssignmentEpoch = group.assignmentEpoch(); Assignment targetAssignment = group.targetAssignment(memberId); if (groupEpoch > targetAssignmentEpoch) { 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 3546ff3158229..9725a61aa4e36 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 @@ -648,15 +648,15 @@ public void testConsumerGroupMemberEpochValidation() { // 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. Result result = context.consumerGroupHeartbeat( - new ConsumerGroupHeartbeatRequestData() - .setGroupId(groupId) - .setMemberId(memberId) - .setMemberEpoch(99) - .setRebalanceTimeoutMs(5000) - .setSubscribedTopicNames(Arrays.asList("foo", "bar")) - .setTopicPartitions(Collections.singletonList(new ConsumerGroupHeartbeatRequestData.TopicPartitions() - .setTopicId(fooTopicId) - .setPartitions(Arrays.asList(1, 2))))); + new ConsumerGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId) + .setMemberEpoch(99) + .setRebalanceTimeoutMs(5000) + .setSubscribedTopicNames(Arrays.asList("foo", "bar")) + .setTopicPartitions(Collections.singletonList(new ConsumerGroupHeartbeatRequestData.TopicPartitions() + .setTopicId(fooTopicId) + .setPartitions(Arrays.asList(1, 2))))); assertEquals(100, result.response().memberEpoch()); }