mirror of https://github.com/apache/kafka.git
KAFKA-16822: Abstract consumer group to share functionality with share group (KIP-932) (#16054)
Abstracted code for 2 classes `ConsumerGroup` and `ConsumerGroupMember` to `ModernGroup` and `ModernGroupMember` respectively. The new abstract classes are created to share common functionality with `ShareGroup` and `ShareGroupMember` which are being introduced with KIP-932. The patch is majorly code refactoring from existing classes to abstract classes. Also created a new package called `modern` where `MemberState` class is moved, in upcoming patches, I will move common classes for `Share` and `Consumer` Group in `modern` package itself. Reviewers: Lianet Magrans <lianetmr@gmail.com>, Andrew Schofield <aschofield@confluent.io>, David Jacot <djacot@confluent.io>
This commit is contained in:
parent
c2de7c5e54
commit
60114a46a7
|
@ -169,4 +169,20 @@ public interface Group {
|
||||||
* @return true if the state includes, false otherwise.
|
* @return true if the state includes, false otherwise.
|
||||||
*/
|
*/
|
||||||
boolean isInStates(Set<String> statesFilter, long committedOffset);
|
boolean isInStates(Set<String> statesFilter, long committedOffset);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if the member exists.
|
||||||
|
*
|
||||||
|
* @param memberId The member id.
|
||||||
|
*
|
||||||
|
* @return A boolean indicating whether the member exists or not.
|
||||||
|
*/
|
||||||
|
boolean hasMember(String memberId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns number of members in the group.
|
||||||
|
*
|
||||||
|
* @return The number of members.
|
||||||
|
*/
|
||||||
|
int numMembers();
|
||||||
}
|
}
|
||||||
|
|
|
@ -68,7 +68,6 @@ import org.apache.kafka.coordinator.group.consumer.Assignment;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.consumer.CurrentAssignmentBuilder;
|
import org.apache.kafka.coordinator.group.consumer.CurrentAssignmentBuilder;
|
||||||
import org.apache.kafka.coordinator.group.consumer.MemberState;
|
|
||||||
import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
|
import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
|
||||||
import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
|
import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
||||||
|
@ -86,6 +85,7 @@ import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmen
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
||||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorTimer;
|
import org.apache.kafka.coordinator.group.runtime.CoordinatorTimer;
|
||||||
import org.apache.kafka.image.MetadataDelta;
|
import org.apache.kafka.image.MetadataDelta;
|
||||||
|
@ -881,7 +881,7 @@ public class GroupMetadataManager {
|
||||||
log.info("Cannot upgrade classic group {} to consumer group because the group does not use the consumer embedded protocol.",
|
log.info("Cannot upgrade classic group {} to consumer group because the group does not use the consumer embedded protocol.",
|
||||||
classicGroup.groupId());
|
classicGroup.groupId());
|
||||||
return false;
|
return false;
|
||||||
} else if (classicGroup.size() > consumerGroupMaxSize) {
|
} else if (classicGroup.numMembers() > consumerGroupMaxSize) {
|
||||||
log.info("Cannot upgrade classic group {} to consumer group because the group size exceeds the consumer group maximum size.",
|
log.info("Cannot upgrade classic group {} to consumer group because the group size exceeds the consumer group maximum size.",
|
||||||
classicGroup.groupId());
|
classicGroup.groupId());
|
||||||
return false;
|
return false;
|
||||||
|
@ -1856,8 +1856,8 @@ public class GroupMetadataManager {
|
||||||
updatedMember
|
updatedMember
|
||||||
).orElse(defaultConsumerGroupAssignor.name());
|
).orElse(defaultConsumerGroupAssignor.name());
|
||||||
try {
|
try {
|
||||||
TargetAssignmentBuilder assignmentResultBuilder =
|
TargetAssignmentBuilder<ConsumerGroupMember> assignmentResultBuilder =
|
||||||
new TargetAssignmentBuilder(group.groupId(), groupEpoch, consumerGroupAssignors.get(preferredServerAssignor))
|
new TargetAssignmentBuilder<ConsumerGroupMember>(group.groupId(), groupEpoch, consumerGroupAssignors.get(preferredServerAssignor))
|
||||||
.withMembers(group.members())
|
.withMembers(group.members())
|
||||||
.withStaticMembers(group.staticMembers())
|
.withStaticMembers(group.staticMembers())
|
||||||
.withSubscriptionMetadata(subscriptionMetadata)
|
.withSubscriptionMetadata(subscriptionMetadata)
|
||||||
|
@ -2600,10 +2600,10 @@ public class GroupMetadataManager {
|
||||||
rescheduleClassicGroupMemberHeartbeat(classicGroup, member);
|
rescheduleClassicGroupMemberHeartbeat(classicGroup, member);
|
||||||
});
|
});
|
||||||
|
|
||||||
if (classicGroup.size() > classicGroupMaxSize) {
|
if (classicGroup.numMembers() > classicGroupMaxSize) {
|
||||||
// In case the max size config has changed.
|
// In case the max size config has changed.
|
||||||
prepareRebalance(classicGroup, "Freshly-loaded group " + groupId +
|
prepareRebalance(classicGroup, "Freshly-loaded group " + groupId +
|
||||||
" (size " + classicGroup.size() + ") is over capacity " + classicGroupMaxSize +
|
" (size " + classicGroup.numMembers() + ") is over capacity " + classicGroupMaxSize +
|
||||||
". Rebalancing in order to give a chance for consumers to commit offsets");
|
". Rebalancing in order to give a chance for consumers to commit offsets");
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
|
@ -3255,7 +3255,7 @@ public class GroupMetadataManager {
|
||||||
|
|
||||||
} else {
|
} else {
|
||||||
log.info("Stabilized group {} generation {} with {} members.",
|
log.info("Stabilized group {} generation {} with {} members.",
|
||||||
groupId, group.generationId(), group.size());
|
groupId, group.generationId(), group.numMembers());
|
||||||
|
|
||||||
// Complete the awaiting join group response future for all the members after rebalancing
|
// Complete the awaiting join group response future for all the members after rebalancing
|
||||||
group.allMembers().forEach(member -> {
|
group.allMembers().forEach(member -> {
|
||||||
|
@ -3333,7 +3333,7 @@ public class GroupMetadataManager {
|
||||||
memberId, group.groupId());
|
memberId, group.groupId());
|
||||||
|
|
||||||
return removePendingMemberAndUpdateClassicGroup(group, memberId);
|
return removePendingMemberAndUpdateClassicGroup(group, memberId);
|
||||||
} else if (!group.hasMemberId(memberId)) {
|
} else if (!group.hasMember(memberId)) {
|
||||||
log.debug("Member {} has already been removed from the group.", memberId);
|
log.debug("Member {} has already been removed from the group.", memberId);
|
||||||
} else {
|
} else {
|
||||||
ClassicGroupMember member = group.member(memberId);
|
ClassicGroupMember member = group.member(memberId);
|
||||||
|
@ -3801,14 +3801,14 @@ public class GroupMetadataManager {
|
||||||
// if the max group size was reduced.
|
// if the max group size was reduced.
|
||||||
// 2) using the number of awaiting members allows to kick out the last rejoining
|
// 2) using the number of awaiting members allows to kick out the last rejoining
|
||||||
// members of the group.
|
// members of the group.
|
||||||
return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
|
return (group.hasMember(memberId) && group.member(memberId).isAwaitingJoin()) ||
|
||||||
group.numAwaitingJoinResponse() < classicGroupMaxSize;
|
group.numAwaitingJoinResponse() < classicGroupMaxSize;
|
||||||
case COMPLETING_REBALANCE:
|
case COMPLETING_REBALANCE:
|
||||||
case STABLE:
|
case STABLE:
|
||||||
// An existing member is accepted. New members are accepted up to the max group size.
|
// An existing member is accepted. New members are accepted up to the max group size.
|
||||||
// Note that the group size is used here. When the group transitions to CompletingRebalance,
|
// Note that the group size is used here. When the group transitions to CompletingRebalance,
|
||||||
// members who haven't rejoined are removed.
|
// members who haven't rejoined are removed.
|
||||||
return group.hasMemberId(memberId) || group.size() < classicGroupMaxSize;
|
return group.hasMember(memberId) || group.numMembers() < classicGroupMaxSize;
|
||||||
default:
|
default:
|
||||||
throw new IllegalStateException("Unknown group state: " + group.stateAsString());
|
throw new IllegalStateException("Unknown group state: " + group.stateAsString());
|
||||||
}
|
}
|
||||||
|
@ -4011,7 +4011,7 @@ public class GroupMetadataManager {
|
||||||
if (group.isLeader(memberId)) {
|
if (group.isLeader(memberId)) {
|
||||||
log.info("Assignment received from leader {} for group {} for generation {}. " +
|
log.info("Assignment received from leader {} for group {} for generation {}. " +
|
||||||
"The group has {} members, {} of which are static.",
|
"The group has {} members, {} of which are static.",
|
||||||
memberId, groupId, group.generationId(), group.size(), group.allStaticMemberIds().size());
|
memberId, groupId, group.generationId(), group.numMembers(), group.allStaticMemberIds().size());
|
||||||
|
|
||||||
// Fill all members with corresponding member assignment. If the member assignment
|
// Fill all members with corresponding member assignment. If the member assignment
|
||||||
// does not exist, fill with an empty assignment.
|
// does not exist, fill with an empty assignment.
|
||||||
|
|
|
@ -20,10 +20,12 @@ import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||||
import org.apache.kafka.common.message.ConsumerProtocolAssignment;
|
import org.apache.kafka.common.message.ConsumerProtocolAssignment;
|
||||||
import org.apache.kafka.common.message.ConsumerProtocolSubscription;
|
import org.apache.kafka.common.message.ConsumerProtocolSubscription;
|
||||||
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
|
||||||
import org.apache.kafka.image.TopicImage;
|
import org.apache.kafka.image.TopicImage;
|
||||||
import org.apache.kafka.image.TopicsImage;
|
import org.apache.kafka.image.TopicsImage;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
@ -33,6 +35,7 @@ import java.util.Optional;
|
||||||
import java.util.OptionalInt;
|
import java.util.OptionalInt;
|
||||||
import java.util.OptionalLong;
|
import java.util.OptionalLong;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class Utils {
|
public class Utils {
|
||||||
private Utils() {}
|
private Utils() {}
|
||||||
|
@ -80,6 +83,22 @@ public class Utils {
|
||||||
return builder.toString();
|
return builder.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Decrements value by 1; returns null when reaching zero. This helper is
|
||||||
|
* meant to be used with Map#compute.
|
||||||
|
*/
|
||||||
|
public 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.
|
||||||
|
*/
|
||||||
|
public static Integer incValue(String key, Integer value) {
|
||||||
|
return value == null ? 1 : value + 1;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return An Optional containing the provided string if it is not null and not empty,
|
* @return An Optional containing the provided string if it is not null and not empty,
|
||||||
* otherwise an empty Optional.
|
* otherwise an empty Optional.
|
||||||
|
@ -158,4 +177,18 @@ public class Utils {
|
||||||
}
|
}
|
||||||
return res;
|
return res;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a map of topic id and partition set from a list of consumer group TopicPartitions.
|
||||||
|
*
|
||||||
|
* @param topicPartitionsList The list of TopicPartitions.
|
||||||
|
* @return a map of topic id and partition set.
|
||||||
|
*/
|
||||||
|
public static Map<Uuid, Set<Integer>> assignmentFromTopicPartitions(
|
||||||
|
List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> topicPartitionsList
|
||||||
|
) {
|
||||||
|
return topicPartitionsList.stream().collect(Collectors.toMap(
|
||||||
|
ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions::topicId,
|
||||||
|
topicPartitions -> Collections.unmodifiableSet(new HashSet<>(topicPartitions.partitions()))));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -321,12 +321,10 @@ public class ClassicGroup implements Group {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* To identify whether the given member id is part of this group.
|
|
||||||
*
|
|
||||||
* @param memberId the given member id.
|
|
||||||
* @return true if the member is part of this group, false otherwise.
|
* @return true if the member is part of this group, false otherwise.
|
||||||
*/
|
*/
|
||||||
public boolean hasMemberId(String memberId) {
|
@Override
|
||||||
|
public boolean hasMember(String memberId) {
|
||||||
return members.containsKey(memberId);
|
return members.containsKey(memberId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -343,7 +341,8 @@ public class ClassicGroup implements Group {
|
||||||
/**
|
/**
|
||||||
* @return the total number of members in this group.
|
* @return the total number of members in this group.
|
||||||
*/
|
*/
|
||||||
public int size() {
|
@Override
|
||||||
|
public int numMembers() {
|
||||||
return members.size();
|
return members.size();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -611,7 +610,7 @@ public class ClassicGroup implements Group {
|
||||||
* false otherwise.
|
* false otherwise.
|
||||||
*/
|
*/
|
||||||
public boolean addPendingMember(String memberId) {
|
public boolean addPendingMember(String memberId) {
|
||||||
if (hasMemberId(memberId)) {
|
if (hasMember(memberId)) {
|
||||||
throw new IllegalStateException("Attempt to add pending member " + memberId +
|
throw new IllegalStateException("Attempt to add pending member " + memberId +
|
||||||
" which is already a stable member of the group.");
|
" which is already a stable member of the group.");
|
||||||
}
|
}
|
||||||
|
@ -633,7 +632,7 @@ public class ClassicGroup implements Group {
|
||||||
* false otherwise.
|
* false otherwise.
|
||||||
*/
|
*/
|
||||||
public boolean addPendingSyncMember(String memberId) {
|
public boolean addPendingSyncMember(String memberId) {
|
||||||
if (!hasMemberId(memberId)) {
|
if (!hasMember(memberId)) {
|
||||||
throw new IllegalStateException("Attempt to add pending sync member " + memberId +
|
throw new IllegalStateException("Attempt to add pending sync member " + memberId +
|
||||||
" which is already a stable member of the group.");
|
" which is already a stable member of the group.");
|
||||||
}
|
}
|
||||||
|
@ -648,7 +647,7 @@ public class ClassicGroup implements Group {
|
||||||
* @return true if the group did store this member, false otherwise.
|
* @return true if the group did store this member, false otherwise.
|
||||||
*/
|
*/
|
||||||
public boolean removePendingSyncMember(String memberId) {
|
public boolean removePendingSyncMember(String memberId) {
|
||||||
if (!hasMemberId(memberId)) {
|
if (!hasMember(memberId)) {
|
||||||
throw new IllegalStateException("Attempt to add pending member " + memberId +
|
throw new IllegalStateException("Attempt to add pending member " + memberId +
|
||||||
" which is already a stable member of the group.");
|
" which is already a stable member of the group.");
|
||||||
}
|
}
|
||||||
|
@ -808,7 +807,7 @@ public class ClassicGroup implements Group {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!hasMemberId(memberId)) {
|
if (!hasMember(memberId)) {
|
||||||
throw Errors.UNKNOWN_MEMBER_ID.exception();
|
throw Errors.UNKNOWN_MEMBER_ID.exception();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,19 +25,18 @@ import org.apache.kafka.common.errors.UnknownMemberIdException;
|
||||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
|
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
|
||||||
import org.apache.kafka.common.message.ConsumerProtocolSubscription;
|
import org.apache.kafka.common.message.ConsumerProtocolSubscription;
|
||||||
import org.apache.kafka.common.message.ListGroupsResponseData;
|
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.coordinator.group.CoordinatorRecord;
|
import org.apache.kafka.coordinator.group.CoordinatorRecord;
|
||||||
import org.apache.kafka.coordinator.group.CoordinatorRecordHelpers;
|
import org.apache.kafka.coordinator.group.CoordinatorRecordHelpers;
|
||||||
import org.apache.kafka.coordinator.group.Group;
|
|
||||||
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
|
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
|
||||||
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
|
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
|
||||||
import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType;
|
import org.apache.kafka.coordinator.group.Utils;
|
||||||
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
import org.apache.kafka.image.ClusterImage;
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
import org.apache.kafka.image.TopicImage;
|
import org.apache.kafka.coordinator.group.modern.ModernGroup;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.ModernGroupMember;
|
||||||
import org.apache.kafka.image.TopicsImage;
|
import org.apache.kafka.image.TopicsImage;
|
||||||
import org.apache.kafka.timeline.SnapshotRegistry;
|
import org.apache.kafka.timeline.SnapshotRegistry;
|
||||||
import org.apache.kafka.timeline.TimelineHashMap;
|
import org.apache.kafka.timeline.TimelineHashMap;
|
||||||
|
@ -47,7 +46,6 @@ import org.apache.kafka.timeline.TimelineObject;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -57,8 +55,6 @@ import java.util.Set;
|
||||||
|
|
||||||
import static org.apache.kafka.coordinator.group.Utils.toOptional;
|
import static org.apache.kafka.coordinator.group.Utils.toOptional;
|
||||||
import static org.apache.kafka.coordinator.group.Utils.toTopicPartitionMap;
|
import static org.apache.kafka.coordinator.group.Utils.toTopicPartitionMap;
|
||||||
import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HETEROGENEOUS;
|
|
||||||
import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HOMOGENEOUS;
|
|
||||||
import static org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState.ASSIGNING;
|
import static org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState.ASSIGNING;
|
||||||
import static org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState.EMPTY;
|
import static org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState.EMPTY;
|
||||||
import static org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState.RECONCILING;
|
import static org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState.RECONCILING;
|
||||||
|
@ -68,7 +64,7 @@ import static org.apache.kafka.coordinator.group.consumer.ConsumerGroup.Consumer
|
||||||
* A Consumer Group. All the metadata in this class are backed by
|
* A Consumer Group. All the metadata in this class are backed by
|
||||||
* records in the __consumer_offsets partitions.
|
* records in the __consumer_offsets partitions.
|
||||||
*/
|
*/
|
||||||
public class ConsumerGroup implements Group {
|
public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
|
||||||
|
|
||||||
public enum ConsumerGroupState {
|
public enum ConsumerGroupState {
|
||||||
EMPTY("Empty"),
|
EMPTY("Empty"),
|
||||||
|
@ -96,45 +92,11 @@ public class ConsumerGroup implements Group {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class DeadlineAndEpoch {
|
|
||||||
static final DeadlineAndEpoch EMPTY = new DeadlineAndEpoch(0L, 0);
|
|
||||||
|
|
||||||
public final long deadlineMs;
|
|
||||||
public final int epoch;
|
|
||||||
|
|
||||||
DeadlineAndEpoch(long deadlineMs, int epoch) {
|
|
||||||
this.deadlineMs = deadlineMs;
|
|
||||||
this.epoch = epoch;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The snapshot registry.
|
|
||||||
*/
|
|
||||||
private final SnapshotRegistry snapshotRegistry;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The group id.
|
|
||||||
*/
|
|
||||||
private final String groupId;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The group state.
|
* The group state.
|
||||||
*/
|
*/
|
||||||
private final TimelineObject<ConsumerGroupState> state;
|
private final TimelineObject<ConsumerGroupState> 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<String, ConsumerGroupMember> members;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The static group members.
|
* The static group members.
|
||||||
*/
|
*/
|
||||||
|
@ -145,64 +107,11 @@ public class ConsumerGroup implements Group {
|
||||||
*/
|
*/
|
||||||
private final TimelineHashMap<String, Integer> serverAssignors;
|
private final TimelineHashMap<String, Integer> serverAssignors;
|
||||||
|
|
||||||
/**
|
|
||||||
* The number of subscribers per topic.
|
|
||||||
*/
|
|
||||||
private final TimelineHashMap<String, Integer> subscribedTopicNames;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The metadata associated with each subscribed topic name.
|
|
||||||
*/
|
|
||||||
private final TimelineHashMap<String, TopicMetadata> subscribedTopicMetadata;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The consumer group's subscription type.
|
|
||||||
* This value is set to Homogeneous by default.
|
|
||||||
*/
|
|
||||||
private final TimelineObject<SubscriptionType> subscriptionType;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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 targetAssignmentEpoch;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The target assignment per member id.
|
|
||||||
*/
|
|
||||||
private final TimelineHashMap<String, Assignment> targetAssignment;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Reverse lookup map representing topic partitions with
|
|
||||||
* their current member assignments.
|
|
||||||
*/
|
|
||||||
private final TimelineHashMap<Uuid, TimelineHashMap<Integer, String>> invertedTargetAssignment;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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
|
|
||||||
* its epochs from this map. When a member gets a partition, it adds its epochs to this map.
|
|
||||||
*/
|
|
||||||
private final TimelineHashMap<Uuid, TimelineHashMap<Integer, Integer>> currentPartitionEpoch;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The coordinator metrics.
|
* The coordinator metrics.
|
||||||
*/
|
*/
|
||||||
private final GroupCoordinatorMetricsShard metrics;
|
private final GroupCoordinatorMetricsShard metrics;
|
||||||
|
|
||||||
/**
|
|
||||||
* The metadata refresh deadline. It consists of a timestamp in milliseconds together with
|
|
||||||
* the group epoch at the time of setting it. The metadata refresh time is considered as a
|
|
||||||
* soft state (read that it is not stored in a timeline data structure). It is like this
|
|
||||||
* because it is not persisted to the log. The group epoch is here to ensure that the
|
|
||||||
* metadata refresh deadline is invalidated if the group epoch does not correspond to
|
|
||||||
* the current group epoch. This can happen if the metadata refresh deadline is updated
|
|
||||||
* after having refreshed the metadata but the write operation failed. In this case, the
|
|
||||||
* time is not automatically rolled back.
|
|
||||||
*/
|
|
||||||
private DeadlineAndEpoch metadataRefreshDeadline = DeadlineAndEpoch.EMPTY;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The number of members that use the classic protocol.
|
* The number of members that use the classic protocol.
|
||||||
*/
|
*/
|
||||||
|
@ -218,20 +127,10 @@ public class ConsumerGroup implements Group {
|
||||||
String groupId,
|
String groupId,
|
||||||
GroupCoordinatorMetricsShard metrics
|
GroupCoordinatorMetricsShard metrics
|
||||||
) {
|
) {
|
||||||
this.snapshotRegistry = Objects.requireNonNull(snapshotRegistry);
|
super(snapshotRegistry, groupId);
|
||||||
this.groupId = Objects.requireNonNull(groupId);
|
|
||||||
this.state = new TimelineObject<>(snapshotRegistry, EMPTY);
|
this.state = new TimelineObject<>(snapshotRegistry, EMPTY);
|
||||||
this.groupEpoch = new TimelineInteger(snapshotRegistry);
|
|
||||||
this.members = new TimelineHashMap<>(snapshotRegistry, 0);
|
|
||||||
this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0);
|
this.staticMembers = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0);
|
this.serverAssignors = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
this.subscribedTopicNames = new TimelineHashMap<>(snapshotRegistry, 0);
|
|
||||||
this.subscribedTopicMetadata = new TimelineHashMap<>(snapshotRegistry, 0);
|
|
||||||
this.subscriptionType = new TimelineObject<>(snapshotRegistry, HOMOGENEOUS);
|
|
||||||
this.targetAssignmentEpoch = new TimelineInteger(snapshotRegistry);
|
|
||||||
this.targetAssignment = new TimelineHashMap<>(snapshotRegistry, 0);
|
|
||||||
this.invertedTargetAssignment = new TimelineHashMap<>(snapshotRegistry, 0);
|
|
||||||
this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 0);
|
|
||||||
this.metrics = Objects.requireNonNull(metrics);
|
this.metrics = Objects.requireNonNull(metrics);
|
||||||
this.numClassicProtocolMembers = new TimelineInteger(snapshotRegistry);
|
this.numClassicProtocolMembers = new TimelineInteger(snapshotRegistry);
|
||||||
this.classicProtocolMembersSupportedProtocols = new TimelineHashMap<>(snapshotRegistry, 0);
|
this.classicProtocolMembersSupportedProtocols = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
|
@ -245,6 +144,14 @@ public class ConsumerGroup implements Group {
|
||||||
return GroupType.CONSUMER;
|
return GroupType.CONSUMER;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The group protocol type (consumer).
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public String protocolType() {
|
||||||
|
return ConsumerProtocol.PROTOCOL_TYPE;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The current state as a String.
|
* @return The current state as a String.
|
||||||
*/
|
*/
|
||||||
|
@ -260,25 +167,6 @@ public class ConsumerGroup implements Group {
|
||||||
return state.get(committedOffset).toString();
|
return state.get(committedOffset).toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return the group formatted as a list group response based on the committed offset.
|
|
||||||
*/
|
|
||||||
public ListGroupsResponseData.ListedGroup asListedGroup(long committedOffset) {
|
|
||||||
return new ListGroupsResponseData.ListedGroup()
|
|
||||||
.setGroupId(groupId)
|
|
||||||
.setProtocolType(ConsumerProtocol.PROTOCOL_TYPE)
|
|
||||||
.setGroupState(state.get(committedOffset).toString())
|
|
||||||
.setGroupType(type().toString());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The group id.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public String groupId() {
|
|
||||||
return groupId;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The current state.
|
* @return The current state.
|
||||||
*/
|
*/
|
||||||
|
@ -293,40 +181,6 @@ public class ConsumerGroup implements Group {
|
||||||
return state.get(committedOffset);
|
return state.get(committedOffset);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @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();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The target assignment epoch.
|
|
||||||
*/
|
|
||||||
public int assignmentEpoch() {
|
|
||||||
return targetAssignmentEpoch.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Sets the assignment epoch.
|
|
||||||
*
|
|
||||||
* @param targetAssignmentEpoch The new assignment epoch.
|
|
||||||
*/
|
|
||||||
public void setTargetAssignmentEpoch(int targetAssignmentEpoch) {
|
|
||||||
this.targetAssignmentEpoch.set(targetAssignmentEpoch);
|
|
||||||
maybeUpdateGroupState();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets the number of members using the classic protocol.
|
* Sets the number of members using the classic protocol.
|
||||||
*
|
*
|
||||||
|
@ -386,11 +240,7 @@ public class ConsumerGroup implements Group {
|
||||||
return existingMemberId == null ? null : getOrMaybeCreateMember(existingMemberId, false);
|
return existingMemberId == null ? null : getOrMaybeCreateMember(existingMemberId, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@Override
|
||||||
* Adds or updates the member.
|
|
||||||
*
|
|
||||||
* @param newMember The new member state.
|
|
||||||
*/
|
|
||||||
public void updateMember(ConsumerGroupMember newMember) {
|
public void updateMember(ConsumerGroupMember newMember) {
|
||||||
if (newMember == null) {
|
if (newMember == null) {
|
||||||
throw new IllegalArgumentException("newMember cannot be null.");
|
throw new IllegalArgumentException("newMember cannot be null.");
|
||||||
|
@ -416,11 +266,7 @@ public class ConsumerGroup implements Group {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@Override
|
||||||
* Remove the member from the group.
|
|
||||||
*
|
|
||||||
* @param memberId The member id to remove.
|
|
||||||
*/
|
|
||||||
public void removeMember(String memberId) {
|
public void removeMember(String memberId) {
|
||||||
ConsumerGroupMember oldMember = members.remove(memberId);
|
ConsumerGroupMember oldMember = members.remove(memberId);
|
||||||
maybeUpdateSubscribedTopicNamesAndGroupSubscriptionType(oldMember, null);
|
maybeUpdateSubscribedTopicNamesAndGroupSubscriptionType(oldMember, null);
|
||||||
|
@ -443,24 +289,6 @@ public class ConsumerGroup implements Group {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* 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);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The number of members.
|
|
||||||
*/
|
|
||||||
public int numMembers() {
|
|
||||||
return members.size();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The number of members that use the classic protocol.
|
* @return The number of members that use the classic protocol.
|
||||||
*/
|
*/
|
||||||
|
@ -475,13 +303,6 @@ public class ConsumerGroup implements Group {
|
||||||
return Collections.unmodifiableMap(classicProtocolMembersSupportedProtocols);
|
return Collections.unmodifiableMap(classicProtocolMembersSupportedProtocols);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return An immutable Map containing all the members keyed by their id.
|
|
||||||
*/
|
|
||||||
public Map<String, ConsumerGroupMember> members() {
|
|
||||||
return Collections.unmodifiableMap(members);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return An immutable Map containing all the static members keyed by instance id.
|
* @return An immutable Map containing all the static members keyed by instance id.
|
||||||
*/
|
*/
|
||||||
|
@ -489,157 +310,6 @@ public class ConsumerGroup implements Group {
|
||||||
return Collections.unmodifiableMap(staticMembers);
|
return Collections.unmodifiableMap(staticMembers);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return An immutable map containing all the subscribed topic names
|
|
||||||
* with the subscribers counts per topic.
|
|
||||||
*/
|
|
||||||
public Map<String, Integer> subscribedTopicNames() {
|
|
||||||
return Collections.unmodifiableMap(subscribedTopicNames);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns true if the consumer group is actively subscribed to the topic.
|
|
||||||
*
|
|
||||||
* @param topic The topic name.
|
|
||||||
*
|
|
||||||
* @return Whether the group is subscribed to the topic.
|
|
||||||
*/
|
|
||||||
@Override
|
|
||||||
public boolean isSubscribedToTopic(String topic) {
|
|
||||||
return subscribedTopicNames.containsKey(topic);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The group's subscription type.
|
|
||||||
*/
|
|
||||||
public SubscriptionType subscriptionType() {
|
|
||||||
return subscriptionType.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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 targetAssignment.getOrDefault(memberId, Assignment.EMPTY);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return An immutable map containing all the topic partitions
|
|
||||||
* with their current member assignments.
|
|
||||||
*/
|
|
||||||
public Map<Uuid, Map<Integer, String>> invertedTargetAssignment() {
|
|
||||||
return Collections.unmodifiableMap(invertedTargetAssignment);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Updates the target assignment of a member.
|
|
||||||
*
|
|
||||||
* @param memberId The member id.
|
|
||||||
* @param newTargetAssignment The new target assignment.
|
|
||||||
*/
|
|
||||||
public void updateTargetAssignment(String memberId, Assignment newTargetAssignment) {
|
|
||||||
updateInvertedTargetAssignment(
|
|
||||||
memberId,
|
|
||||||
targetAssignment.getOrDefault(memberId, new Assignment(Collections.emptyMap())),
|
|
||||||
newTargetAssignment
|
|
||||||
);
|
|
||||||
targetAssignment.put(memberId, newTargetAssignment);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Updates the reverse lookup map of the target assignment.
|
|
||||||
*
|
|
||||||
* @param memberId The member Id.
|
|
||||||
* @param oldTargetAssignment The old target assignment.
|
|
||||||
* @param newTargetAssignment The new target assignment.
|
|
||||||
*/
|
|
||||||
private void updateInvertedTargetAssignment(
|
|
||||||
String memberId,
|
|
||||||
Assignment oldTargetAssignment,
|
|
||||||
Assignment newTargetAssignment
|
|
||||||
) {
|
|
||||||
// Combine keys from both old and new assignments.
|
|
||||||
Set<Uuid> allTopicIds = new HashSet<>();
|
|
||||||
allTopicIds.addAll(oldTargetAssignment.partitions().keySet());
|
|
||||||
allTopicIds.addAll(newTargetAssignment.partitions().keySet());
|
|
||||||
|
|
||||||
for (Uuid topicId : allTopicIds) {
|
|
||||||
Set<Integer> oldPartitions = oldTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet());
|
|
||||||
Set<Integer> newPartitions = newTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet());
|
|
||||||
|
|
||||||
TimelineHashMap<Integer, String> topicPartitionAssignment = invertedTargetAssignment.computeIfAbsent(
|
|
||||||
topicId, k -> new TimelineHashMap<>(snapshotRegistry, Math.max(oldPartitions.size(), newPartitions.size()))
|
|
||||||
);
|
|
||||||
|
|
||||||
// Remove partitions that aren't present in the new assignment only if the partition is currently
|
|
||||||
// still assigned to the member in question.
|
|
||||||
// If p0 was moved from A to B, and the target assignment map was updated for B first, we don't want to
|
|
||||||
// remove the key p0 from the inverted map and undo the action when A eventually tries to update its assignment.
|
|
||||||
for (Integer partition : oldPartitions) {
|
|
||||||
if (!newPartitions.contains(partition) && memberId.equals(topicPartitionAssignment.get(partition))) {
|
|
||||||
topicPartitionAssignment.remove(partition);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Add partitions that are in the new assignment but not in the old assignment.
|
|
||||||
for (Integer partition : newPartitions) {
|
|
||||||
if (!oldPartitions.contains(partition)) {
|
|
||||||
topicPartitionAssignment.put(partition, memberId);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (topicPartitionAssignment.isEmpty()) {
|
|
||||||
invertedTargetAssignment.remove(topicId);
|
|
||||||
} else {
|
|
||||||
invertedTargetAssignment.put(topicId, topicPartitionAssignment);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Removes the target assignment of a member.
|
|
||||||
*
|
|
||||||
* @param memberId The member id.
|
|
||||||
*/
|
|
||||||
public void removeTargetAssignment(String memberId) {
|
|
||||||
updateInvertedTargetAssignment(
|
|
||||||
memberId,
|
|
||||||
targetAssignment.getOrDefault(memberId, Assignment.EMPTY),
|
|
||||||
Assignment.EMPTY
|
|
||||||
);
|
|
||||||
targetAssignment.remove(memberId);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return An immutable Map containing all the target assignment keyed by member id.
|
|
||||||
*/
|
|
||||||
public Map<String, Assignment> targetAssignment() {
|
|
||||||
return Collections.unmodifiableMap(targetAssignment);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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<Integer, Integer> 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
|
* Compute the preferred (server side) assignor for the group while
|
||||||
* taking into account the updated member. The computation relies
|
* taking into account the updated member. The computation relies
|
||||||
|
@ -680,113 +350,6 @@ public class ConsumerGroup implements Group {
|
||||||
.map(Map.Entry::getKey);
|
.map(Map.Entry::getKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return An immutable Map of subscription metadata for
|
|
||||||
* each topic that the consumer group is subscribed to.
|
|
||||||
*/
|
|
||||||
public Map<String, TopicMetadata> subscriptionMetadata() {
|
|
||||||
return Collections.unmodifiableMap(subscribedTopicMetadata);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Updates the subscription metadata. This replaces the previous one.
|
|
||||||
*
|
|
||||||
* @param subscriptionMetadata The new subscription metadata.
|
|
||||||
*/
|
|
||||||
public void setSubscriptionMetadata(
|
|
||||||
Map<String, TopicMetadata> subscriptionMetadata
|
|
||||||
) {
|
|
||||||
this.subscribedTopicMetadata.clear();
|
|
||||||
this.subscribedTopicMetadata.putAll(subscriptionMetadata);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Computes the subscription metadata based on the current subscription info.
|
|
||||||
*
|
|
||||||
* @param subscribedTopicNames Map of topic names to the number of subscribers.
|
|
||||||
* @param topicsImage The current metadata for all available topics.
|
|
||||||
* @param clusterImage The current metadata for the Kafka cluster.
|
|
||||||
*
|
|
||||||
* @return An immutable map of subscription metadata for each topic that the consumer group is subscribed to.
|
|
||||||
*/
|
|
||||||
public Map<String, TopicMetadata> computeSubscriptionMetadata(
|
|
||||||
Map<String, Integer> subscribedTopicNames,
|
|
||||||
TopicsImage topicsImage,
|
|
||||||
ClusterImage clusterImage
|
|
||||||
) {
|
|
||||||
// Create the topic metadata for each subscribed topic.
|
|
||||||
Map<String, TopicMetadata> newSubscriptionMetadata = new HashMap<>(subscribedTopicNames.size());
|
|
||||||
|
|
||||||
subscribedTopicNames.forEach((topicName, count) -> {
|
|
||||||
TopicImage topicImage = topicsImage.getTopic(topicName);
|
|
||||||
if (topicImage != null) {
|
|
||||||
Map<Integer, Set<String>> partitionRacks = new HashMap<>();
|
|
||||||
topicImage.partitions().forEach((partition, partitionRegistration) -> {
|
|
||||||
Set<String> racks = new HashSet<>();
|
|
||||||
for (int replica : partitionRegistration.replicas) {
|
|
||||||
Optional<String> rackOptional = clusterImage.broker(replica).rack();
|
|
||||||
// Only add the rack if it is available for the broker/replica.
|
|
||||||
rackOptional.ifPresent(racks::add);
|
|
||||||
}
|
|
||||||
// If rack information is unavailable for all replicas of this partition,
|
|
||||||
// no corresponding entry will be stored for it in the map.
|
|
||||||
if (!racks.isEmpty())
|
|
||||||
partitionRacks.put(partition, racks);
|
|
||||||
});
|
|
||||||
|
|
||||||
newSubscriptionMetadata.put(topicName, new TopicMetadata(
|
|
||||||
topicImage.id(),
|
|
||||||
topicImage.name(),
|
|
||||||
topicImage.partitions().size(),
|
|
||||||
partitionRacks)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
return Collections.unmodifiableMap(newSubscriptionMetadata);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Updates the metadata refresh deadline.
|
|
||||||
*
|
|
||||||
* @param deadlineMs The deadline in milliseconds.
|
|
||||||
* @param groupEpoch The associated group epoch.
|
|
||||||
*/
|
|
||||||
public void setMetadataRefreshDeadline(
|
|
||||||
long deadlineMs,
|
|
||||||
int groupEpoch
|
|
||||||
) {
|
|
||||||
this.metadataRefreshDeadline = new DeadlineAndEpoch(deadlineMs, groupEpoch);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Requests a metadata refresh.
|
|
||||||
*/
|
|
||||||
public void requestMetadataRefresh() {
|
|
||||||
this.metadataRefreshDeadline = DeadlineAndEpoch.EMPTY;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Checks if a metadata refresh is required. A refresh is required in two cases:
|
|
||||||
* 1) The deadline is smaller or equal to the current time;
|
|
||||||
* 2) The group epoch associated with the deadline is larger than
|
|
||||||
* the current group epoch. This means that the operations which updated
|
|
||||||
* the deadline failed.
|
|
||||||
*
|
|
||||||
* @param currentTimeMs The current time in milliseconds.
|
|
||||||
* @return A boolean indicating whether a refresh is required or not.
|
|
||||||
*/
|
|
||||||
public boolean hasMetadataExpired(long currentTimeMs) {
|
|
||||||
return currentTimeMs >= metadataRefreshDeadline.deadlineMs || groupEpoch() < metadataRefreshDeadline.epoch;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The metadata refresh deadline.
|
|
||||||
*/
|
|
||||||
public DeadlineAndEpoch metadataRefreshDeadline() {
|
|
||||||
return metadataRefreshDeadline;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Validates the OffsetCommit request.
|
* Validates the OffsetCommit request.
|
||||||
*
|
*
|
||||||
|
@ -821,7 +384,7 @@ public class ConsumerGroup implements Group {
|
||||||
// the member should be using the OffsetCommit API version >= 9.
|
// the member should be using the OffsetCommit API version >= 9.
|
||||||
if (!isTransactional && !member.useClassicProtocol() && apiVersion < 9) {
|
if (!isTransactional && !member.useClassicProtocol() && apiVersion < 9) {
|
||||||
throw new UnsupportedVersionException("OffsetCommit version 9 or above must be used " +
|
throw new UnsupportedVersionException("OffsetCommit version 9 or above must be used " +
|
||||||
"by members using the consumer group protocol");
|
"by members using the modern group protocol");
|
||||||
}
|
}
|
||||||
|
|
||||||
validateMemberEpoch(memberEpoch, member.memberEpoch(), member.useClassicProtocol());
|
validateMemberEpoch(memberEpoch, member.memberEpoch(), member.useClassicProtocol());
|
||||||
|
@ -862,7 +425,9 @@ public class ConsumerGroup implements Group {
|
||||||
* Validates the OffsetDelete request.
|
* Validates the OffsetDelete request.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public void validateOffsetDelete() {}
|
public void validateOffsetDelete() {
|
||||||
|
// Do nothing.
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Validates the DeleteGroups request.
|
* Validates the DeleteGroups request.
|
||||||
|
@ -943,10 +508,8 @@ public class ConsumerGroup implements Group {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
@Override
|
||||||
* Updates the current state of the group.
|
protected void maybeUpdateGroupState() {
|
||||||
*/
|
|
||||||
private void maybeUpdateGroupState() {
|
|
||||||
ConsumerGroupState previousState = state.get();
|
ConsumerGroupState previousState = state.get();
|
||||||
ConsumerGroupState newState = STABLE;
|
ConsumerGroupState newState = STABLE;
|
||||||
if (members.isEmpty()) {
|
if (members.isEmpty()) {
|
||||||
|
@ -954,7 +517,7 @@ public class ConsumerGroup implements Group {
|
||||||
} else if (groupEpoch.get() > targetAssignmentEpoch.get()) {
|
} else if (groupEpoch.get() > targetAssignmentEpoch.get()) {
|
||||||
newState = ASSIGNING;
|
newState = ASSIGNING;
|
||||||
} else {
|
} else {
|
||||||
for (ConsumerGroupMember member : members.values()) {
|
for (ModernGroupMember member : members.values()) {
|
||||||
if (!member.isReconciledTo(targetAssignmentEpoch.get())) {
|
if (!member.isReconciledTo(targetAssignmentEpoch.get())) {
|
||||||
newState = RECONCILING;
|
newState = RECONCILING;
|
||||||
break;
|
break;
|
||||||
|
@ -993,12 +556,12 @@ public class ConsumerGroup implements Group {
|
||||||
) {
|
) {
|
||||||
if (oldMember != null) {
|
if (oldMember != null) {
|
||||||
oldMember.serverAssignorName().ifPresent(name ->
|
oldMember.serverAssignorName().ifPresent(name ->
|
||||||
serverAssignorCount.compute(name, ConsumerGroup::decValue)
|
serverAssignorCount.compute(name, Utils::decValue)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
if (newMember != null) {
|
if (newMember != null) {
|
||||||
newMember.serverAssignorName().ifPresent(name ->
|
newMember.serverAssignorName().ifPresent(name ->
|
||||||
serverAssignorCount.compute(name, ConsumerGroup::incValue)
|
serverAssignorCount.compute(name, Utils::incValue)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -1036,127 +599,19 @@ public class ConsumerGroup implements Group {
|
||||||
if (oldMember != null) {
|
if (oldMember != null) {
|
||||||
oldMember.supportedClassicProtocols().ifPresent(protocols ->
|
oldMember.supportedClassicProtocols().ifPresent(protocols ->
|
||||||
protocols.forEach(protocol ->
|
protocols.forEach(protocol ->
|
||||||
classicProtocolMembersSupportedProtocols.compute(protocol.name(), ConsumerGroup::decValue)
|
classicProtocolMembersSupportedProtocols.compute(protocol.name(), Utils::decValue)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
if (newMember != null) {
|
if (newMember != null) {
|
||||||
newMember.supportedClassicProtocols().ifPresent(protocols ->
|
newMember.supportedClassicProtocols().ifPresent(protocols ->
|
||||||
protocols.forEach(protocol ->
|
protocols.forEach(protocol ->
|
||||||
classicProtocolMembersSupportedProtocols.compute(protocol.name(), ConsumerGroup::incValue)
|
classicProtocolMembersSupportedProtocols.compute(protocol.name(), Utils::incValue)
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Updates the subscribed topic names count.
|
|
||||||
* The subscription type is updated as a consequence.
|
|
||||||
*
|
|
||||||
* @param oldMember The old member.
|
|
||||||
* @param newMember The new member.
|
|
||||||
*/
|
|
||||||
private void maybeUpdateSubscribedTopicNamesAndGroupSubscriptionType(
|
|
||||||
ConsumerGroupMember oldMember,
|
|
||||||
ConsumerGroupMember newMember
|
|
||||||
) {
|
|
||||||
maybeUpdateSubscribedTopicNames(subscribedTopicNames, oldMember, newMember);
|
|
||||||
subscriptionType.set(subscriptionType(subscribedTopicNames, members.size()));
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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<String, Integer> subscribedTopicCount,
|
|
||||||
ConsumerGroupMember oldMember,
|
|
||||||
ConsumerGroupMember newMember
|
|
||||||
) {
|
|
||||||
if (oldMember != null) {
|
|
||||||
oldMember.subscribedTopicNames().forEach(topicName ->
|
|
||||||
subscribedTopicCount.compute(topicName, ConsumerGroup::decValue)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (newMember != null) {
|
|
||||||
newMember.subscribedTopicNames().forEach(topicName ->
|
|
||||||
subscribedTopicCount.compute(topicName, ConsumerGroup::incValue)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Updates the subscription count.
|
|
||||||
*
|
|
||||||
* @param oldMember The old member.
|
|
||||||
* @param newMember The new member.
|
|
||||||
*
|
|
||||||
* @return Copy of the map of topics to the count of number of subscribers.
|
|
||||||
*/
|
|
||||||
public Map<String, Integer> computeSubscribedTopicNames(
|
|
||||||
ConsumerGroupMember oldMember,
|
|
||||||
ConsumerGroupMember newMember
|
|
||||||
) {
|
|
||||||
Map<String, Integer> subscribedTopicNames = new HashMap<>(this.subscribedTopicNames);
|
|
||||||
maybeUpdateSubscribedTopicNames(
|
|
||||||
subscribedTopicNames,
|
|
||||||
oldMember,
|
|
||||||
newMember
|
|
||||||
);
|
|
||||||
return subscribedTopicNames;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Updates the subscription count with a set of members removed.
|
|
||||||
*
|
|
||||||
* @param removedMembers The set of removed members.
|
|
||||||
*
|
|
||||||
* @return Copy of the map of topics to the count of number of subscribers.
|
|
||||||
*/
|
|
||||||
public Map<String, Integer> computeSubscribedTopicNames(
|
|
||||||
Set<ConsumerGroupMember> removedMembers
|
|
||||||
) {
|
|
||||||
Map<String, Integer> subscribedTopicNames = new HashMap<>(this.subscribedTopicNames);
|
|
||||||
if (removedMembers != null) {
|
|
||||||
removedMembers.forEach(removedMember ->
|
|
||||||
maybeUpdateSubscribedTopicNames(
|
|
||||||
subscribedTopicNames,
|
|
||||||
removedMember,
|
|
||||||
null
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
|
||||||
return subscribedTopicNames;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compute the subscription type of the consumer group.
|
|
||||||
*
|
|
||||||
* @param subscribedTopicNames A map of topic names to the count of members subscribed to each topic.
|
|
||||||
*
|
|
||||||
* @return {@link SubscriptionType#HOMOGENEOUS} if all members are subscribed to exactly the same topics;
|
|
||||||
* otherwise, {@link SubscriptionType#HETEROGENEOUS}.
|
|
||||||
*/
|
|
||||||
public static SubscriptionType subscriptionType(
|
|
||||||
Map<String, Integer> subscribedTopicNames,
|
|
||||||
int numberOfMembers
|
|
||||||
) {
|
|
||||||
if (subscribedTopicNames.isEmpty()) {
|
|
||||||
return HOMOGENEOUS;
|
|
||||||
}
|
|
||||||
|
|
||||||
for (int subscriberCount : subscribedTopicNames.values()) {
|
|
||||||
if (subscriberCount != numberOfMembers) {
|
|
||||||
return HETEROGENEOUS;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return HOMOGENEOUS;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Updates the partition epochs based on the old and the new member.
|
* Updates the partition epochs based on the old and the new member.
|
||||||
*
|
*
|
||||||
|
@ -1253,22 +708,6 @@ public class ConsumerGroup implements Group {
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* 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;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ConsumerGroupDescribeResponseData.DescribedGroup asDescribedGroup(
|
public ConsumerGroupDescribeResponseData.DescribedGroup asDescribedGroup(
|
||||||
long committedOffset,
|
long committedOffset,
|
||||||
String defaultAssignor,
|
String defaultAssignor,
|
||||||
|
|
|
@ -19,8 +19,11 @@ package org.apache.kafka.coordinator.group.consumer;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
|
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
|
||||||
import org.apache.kafka.common.message.JoinGroupRequestData;
|
import org.apache.kafka.common.message.JoinGroupRequestData;
|
||||||
|
import org.apache.kafka.coordinator.group.Utils;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.ModernGroupMember;
|
||||||
import org.apache.kafka.image.TopicImage;
|
import org.apache.kafka.image.TopicImage;
|
||||||
import org.apache.kafka.image.TopicsImage;
|
import org.apache.kafka.image.TopicsImage;
|
||||||
|
|
||||||
|
@ -33,14 +36,13 @@ import java.util.Objects;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.OptionalInt;
|
import java.util.OptionalInt;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* ConsumerGroupMember contains all the information related to a member
|
* ConsumerGroupMember contains all the information related to a member
|
||||||
* within a consumer group. This class is immutable and is fully backed
|
* within a consumer group. This class is immutable and is fully backed
|
||||||
* by records stored in the __consumer_offsets topic.
|
* by records stored in the __consumer_offsets topic.
|
||||||
*/
|
*/
|
||||||
public class ConsumerGroupMember {
|
public class ConsumerGroupMember extends ModernGroupMember {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A builder that facilitates the creation of a new member or the update of
|
* A builder that facilitates the creation of a new member or the update of
|
||||||
|
@ -214,19 +216,11 @@ public class ConsumerGroupMember {
|
||||||
setMemberEpoch(record.memberEpoch());
|
setMemberEpoch(record.memberEpoch());
|
||||||
setPreviousMemberEpoch(record.previousMemberEpoch());
|
setPreviousMemberEpoch(record.previousMemberEpoch());
|
||||||
setState(MemberState.fromValue(record.state()));
|
setState(MemberState.fromValue(record.state()));
|
||||||
setAssignedPartitions(assignmentFromTopicPartitions(record.assignedPartitions()));
|
setAssignedPartitions(Utils.assignmentFromTopicPartitions(record.assignedPartitions()));
|
||||||
setPartitionsPendingRevocation(assignmentFromTopicPartitions(record.partitionsPendingRevocation()));
|
setPartitionsPendingRevocation(Utils.assignmentFromTopicPartitions(record.partitionsPendingRevocation()));
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
private Map<Uuid, Set<Integer>> assignmentFromTopicPartitions(
|
|
||||||
List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> topicPartitionsList
|
|
||||||
) {
|
|
||||||
return topicPartitionsList.stream().collect(Collectors.toMap(
|
|
||||||
ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions::topicId,
|
|
||||||
topicPartitions -> Collections.unmodifiableSet(new HashSet<>(topicPartitions.partitions()))));
|
|
||||||
}
|
|
||||||
|
|
||||||
public ConsumerGroupMember build() {
|
public ConsumerGroupMember build() {
|
||||||
return new ConsumerGroupMember(
|
return new ConsumerGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
|
@ -248,55 +242,6 @@ public class ConsumerGroupMember {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* The member id.
|
|
||||||
*/
|
|
||||||
private final String memberId;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The current member epoch.
|
|
||||||
*/
|
|
||||||
private final int memberEpoch;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The previous member epoch.
|
|
||||||
*/
|
|
||||||
private final int previousMemberEpoch;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The member state.
|
|
||||||
*/
|
|
||||||
private final MemberState state;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The instance id provided by the member.
|
|
||||||
*/
|
|
||||||
private final String instanceId;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The rack id provided by the member.
|
|
||||||
*/
|
|
||||||
private final String rackId;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The rebalance timeout provided by the member.
|
|
||||||
*/
|
|
||||||
private final int rebalanceTimeoutMs;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The client id reported by the member.
|
|
||||||
*/
|
|
||||||
private final String clientId;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The host reported by the member.
|
|
||||||
*/
|
|
||||||
private final String clientHost;
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The list of subscriptions (topic names) configured by the member.
|
|
||||||
*/
|
|
||||||
private final Set<String> subscribedTopicNames;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The subscription pattern configured by the member.
|
* The subscription pattern configured by the member.
|
||||||
|
@ -308,11 +253,6 @@ public class ConsumerGroupMember {
|
||||||
*/
|
*/
|
||||||
private final String serverAssignorName;
|
private final String serverAssignorName;
|
||||||
|
|
||||||
/**
|
|
||||||
* The partitions assigned to this member.
|
|
||||||
*/
|
|
||||||
private final Map<Uuid, Set<Integer>> assignedPartitions;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The partitions being revoked by this member.
|
* The partitions being revoked by this member.
|
||||||
*/
|
*/
|
||||||
|
@ -340,86 +280,25 @@ public class ConsumerGroupMember {
|
||||||
Map<Uuid, Set<Integer>> partitionsPendingRevocation,
|
Map<Uuid, Set<Integer>> partitionsPendingRevocation,
|
||||||
ConsumerGroupMemberMetadataValue.ClassicMemberMetadata classicMemberMetadata
|
ConsumerGroupMemberMetadataValue.ClassicMemberMetadata classicMemberMetadata
|
||||||
) {
|
) {
|
||||||
this.memberId = memberId;
|
super(
|
||||||
this.memberEpoch = memberEpoch;
|
memberId,
|
||||||
this.previousMemberEpoch = previousMemberEpoch;
|
memberEpoch,
|
||||||
this.state = state;
|
previousMemberEpoch,
|
||||||
this.instanceId = instanceId;
|
instanceId,
|
||||||
this.rackId = rackId;
|
rackId,
|
||||||
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
|
rebalanceTimeoutMs,
|
||||||
this.clientId = clientId;
|
clientId,
|
||||||
this.clientHost = clientHost;
|
clientHost,
|
||||||
this.subscribedTopicNames = subscribedTopicNames;
|
subscribedTopicNames,
|
||||||
|
state,
|
||||||
|
assignedPartitions
|
||||||
|
);
|
||||||
this.subscribedTopicRegex = subscribedTopicRegex;
|
this.subscribedTopicRegex = subscribedTopicRegex;
|
||||||
this.serverAssignorName = serverAssignorName;
|
this.serverAssignorName = serverAssignorName;
|
||||||
this.assignedPartitions = assignedPartitions;
|
|
||||||
this.partitionsPendingRevocation = partitionsPendingRevocation;
|
this.partitionsPendingRevocation = partitionsPendingRevocation;
|
||||||
this.classicMemberMetadata = classicMemberMetadata;
|
this.classicMemberMetadata = classicMemberMetadata;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The member id.
|
|
||||||
*/
|
|
||||||
public String memberId() {
|
|
||||||
return memberId;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The current member epoch.
|
|
||||||
*/
|
|
||||||
public int memberEpoch() {
|
|
||||||
return memberEpoch;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The previous member epoch.
|
|
||||||
*/
|
|
||||||
public int previousMemberEpoch() {
|
|
||||||
return previousMemberEpoch;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The instance id.
|
|
||||||
*/
|
|
||||||
public String instanceId() {
|
|
||||||
return instanceId;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The rack id.
|
|
||||||
*/
|
|
||||||
public String rackId() {
|
|
||||||
return rackId;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The rebalance timeout in millis.
|
|
||||||
*/
|
|
||||||
public int rebalanceTimeoutMs() {
|
|
||||||
return rebalanceTimeoutMs;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The client id.
|
|
||||||
*/
|
|
||||||
public String clientId() {
|
|
||||||
return clientId;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The client host.
|
|
||||||
*/
|
|
||||||
public String clientHost() {
|
|
||||||
return clientHost;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The list of subscribed topic names.
|
|
||||||
*/
|
|
||||||
public Set<String> subscribedTopicNames() {
|
|
||||||
return subscribedTopicNames;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The regular expression based subscription.
|
* @return The regular expression based subscription.
|
||||||
*/
|
*/
|
||||||
|
@ -434,27 +313,6 @@ public class ConsumerGroupMember {
|
||||||
return Optional.ofNullable(serverAssignorName);
|
return Optional.ofNullable(serverAssignorName);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The current state.
|
|
||||||
*/
|
|
||||||
public MemberState state() {
|
|
||||||
return state;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return True if the member is in the Stable state and at the desired epoch.
|
|
||||||
*/
|
|
||||||
public boolean isReconciledTo(int targetAssignmentEpoch) {
|
|
||||||
return state == MemberState.STABLE && memberEpoch == targetAssignmentEpoch;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The set of assigned partitions.
|
|
||||||
*/
|
|
||||||
public Map<Uuid, Set<Integer>> assignedPartitions() {
|
|
||||||
return assignedPartitions;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The set of partitions awaiting revocation from the member.
|
* @return The set of partitions awaiting revocation from the member.
|
||||||
*/
|
*/
|
||||||
|
@ -640,14 +498,4 @@ public class ConsumerGroupMember {
|
||||||
", classicMemberMetadata='" + classicMemberMetadata + '\'' +
|
", classicMemberMetadata='" + classicMemberMetadata + '\'' +
|
||||||
')';
|
')';
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return True of the two provided members have different assigned partitions.
|
|
||||||
*/
|
|
||||||
public static boolean hasAssignedPartitionsChanged(
|
|
||||||
ConsumerGroupMember member1,
|
|
||||||
ConsumerGroupMember member2
|
|
||||||
) {
|
|
||||||
return !member1.assignedPartitions().equals(member2.assignedPartitions());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.kafka.coordinator.group.consumer;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
|
||||||
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignor;
|
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignor;
|
||||||
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignorException;
|
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignorException;
|
||||||
import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType;
|
import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.ModernGroupMember;
|
||||||
import org.apache.kafka.image.TopicsImage;
|
import org.apache.kafka.image.TopicsImage;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -48,7 +49,8 @@ import static org.apache.kafka.coordinator.group.CoordinatorRecordHelpers.newTar
|
||||||
* is deleted as part of the member deletion process. In other words, this class
|
* is deleted as part of the member deletion process. In other words, this class
|
||||||
* does not yield a tombstone for removed members.
|
* does not yield a tombstone for removed members.
|
||||||
*/
|
*/
|
||||||
public class TargetAssignmentBuilder {
|
public class TargetAssignmentBuilder<T extends ModernGroupMember> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The assignment result returned by {{@link TargetAssignmentBuilder#build()}}.
|
* The assignment result returned by {{@link TargetAssignmentBuilder#build()}}.
|
||||||
*/
|
*/
|
||||||
|
@ -107,7 +109,7 @@ public class TargetAssignmentBuilder {
|
||||||
/**
|
/**
|
||||||
* The members in the group.
|
* The members in the group.
|
||||||
*/
|
*/
|
||||||
private Map<String, ConsumerGroupMember> members = Collections.emptyMap();
|
private Map<String, T> members = Collections.emptyMap();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The subscription metadata.
|
* The subscription metadata.
|
||||||
|
@ -139,7 +141,7 @@ public class TargetAssignmentBuilder {
|
||||||
* The members which have been updated or deleted. Deleted members
|
* The members which have been updated or deleted. Deleted members
|
||||||
* are signaled by a null value.
|
* are signaled by a null value.
|
||||||
*/
|
*/
|
||||||
private final Map<String, ConsumerGroupMember> updatedMembers = new HashMap<>();
|
private final Map<String, T> updatedMembers = new HashMap<>();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The static members in the group.
|
* The static members in the group.
|
||||||
|
@ -169,8 +171,8 @@ public class TargetAssignmentBuilder {
|
||||||
* @param members The existing members in the consumer group.
|
* @param members The existing members in the consumer group.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder withMembers(
|
public TargetAssignmentBuilder<T> withMembers(
|
||||||
Map<String, ConsumerGroupMember> members
|
Map<String, T> members
|
||||||
) {
|
) {
|
||||||
this.members = members;
|
this.members = members;
|
||||||
return this;
|
return this;
|
||||||
|
@ -182,7 +184,7 @@ public class TargetAssignmentBuilder {
|
||||||
* @param staticMembers The existing static members in the consumer group.
|
* @param staticMembers The existing static members in the consumer group.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder withStaticMembers(
|
public TargetAssignmentBuilder<T> withStaticMembers(
|
||||||
Map<String, String> staticMembers
|
Map<String, String> staticMembers
|
||||||
) {
|
) {
|
||||||
this.staticMembers = staticMembers;
|
this.staticMembers = staticMembers;
|
||||||
|
@ -195,7 +197,7 @@ public class TargetAssignmentBuilder {
|
||||||
* @param subscriptionMetadata The subscription metadata.
|
* @param subscriptionMetadata The subscription metadata.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder withSubscriptionMetadata(
|
public TargetAssignmentBuilder<T> withSubscriptionMetadata(
|
||||||
Map<String, TopicMetadata> subscriptionMetadata
|
Map<String, TopicMetadata> subscriptionMetadata
|
||||||
) {
|
) {
|
||||||
this.subscriptionMetadata = subscriptionMetadata;
|
this.subscriptionMetadata = subscriptionMetadata;
|
||||||
|
@ -208,7 +210,7 @@ public class TargetAssignmentBuilder {
|
||||||
* @param subscriptionType Subscription type of the group.
|
* @param subscriptionType Subscription type of the group.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder withSubscriptionType(
|
public TargetAssignmentBuilder<T> withSubscriptionType(
|
||||||
SubscriptionType subscriptionType
|
SubscriptionType subscriptionType
|
||||||
) {
|
) {
|
||||||
this.subscriptionType = subscriptionType;
|
this.subscriptionType = subscriptionType;
|
||||||
|
@ -221,7 +223,7 @@ public class TargetAssignmentBuilder {
|
||||||
* @param targetAssignment The existing target assignment.
|
* @param targetAssignment The existing target assignment.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder withTargetAssignment(
|
public TargetAssignmentBuilder<T> withTargetAssignment(
|
||||||
Map<String, Assignment> targetAssignment
|
Map<String, Assignment> targetAssignment
|
||||||
) {
|
) {
|
||||||
this.targetAssignment = targetAssignment;
|
this.targetAssignment = targetAssignment;
|
||||||
|
@ -234,7 +236,7 @@ public class TargetAssignmentBuilder {
|
||||||
* @param invertedTargetAssignment The reverse lookup map of the current target assignment.
|
* @param invertedTargetAssignment The reverse lookup map of the current target assignment.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder withInvertedTargetAssignment(
|
public TargetAssignmentBuilder<T> withInvertedTargetAssignment(
|
||||||
Map<Uuid, Map<Integer, String>> invertedTargetAssignment
|
Map<Uuid, Map<Integer, String>> invertedTargetAssignment
|
||||||
) {
|
) {
|
||||||
this.invertedTargetAssignment = invertedTargetAssignment;
|
this.invertedTargetAssignment = invertedTargetAssignment;
|
||||||
|
@ -247,7 +249,7 @@ public class TargetAssignmentBuilder {
|
||||||
* @param topicsImage The topics image.
|
* @param topicsImage The topics image.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder withTopicsImage(
|
public TargetAssignmentBuilder<T> withTopicsImage(
|
||||||
TopicsImage topicsImage
|
TopicsImage topicsImage
|
||||||
) {
|
) {
|
||||||
this.topicsImage = topicsImage;
|
this.topicsImage = topicsImage;
|
||||||
|
@ -262,9 +264,9 @@ public class TargetAssignmentBuilder {
|
||||||
* @param member The member to add or update.
|
* @param member The member to add or update.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder addOrUpdateMember(
|
public TargetAssignmentBuilder<T> addOrUpdateMember(
|
||||||
String memberId,
|
String memberId,
|
||||||
ConsumerGroupMember member
|
T member
|
||||||
) {
|
) {
|
||||||
this.updatedMembers.put(memberId, member);
|
this.updatedMembers.put(memberId, member);
|
||||||
return this;
|
return this;
|
||||||
|
@ -277,7 +279,7 @@ public class TargetAssignmentBuilder {
|
||||||
* @param memberId The member id.
|
* @param memberId The member id.
|
||||||
* @return This object.
|
* @return This object.
|
||||||
*/
|
*/
|
||||||
public TargetAssignmentBuilder removeMember(
|
public TargetAssignmentBuilder<T> removeMember(
|
||||||
String memberId
|
String memberId
|
||||||
) {
|
) {
|
||||||
return addOrUpdateMember(memberId, null);
|
return addOrUpdateMember(memberId, null);
|
||||||
|
@ -382,8 +384,8 @@ public class TargetAssignmentBuilder {
|
||||||
}
|
}
|
||||||
|
|
||||||
// private for testing
|
// private for testing
|
||||||
static MemberSubscriptionAndAssignmentImpl createMemberSubscriptionAndAssignment(
|
static <T extends ModernGroupMember> MemberSubscriptionAndAssignmentImpl createMemberSubscriptionAndAssignment(
|
||||||
ConsumerGroupMember member,
|
T member,
|
||||||
Assignment memberAssignment,
|
Assignment memberAssignment,
|
||||||
TopicsImage topicsImage
|
TopicsImage topicsImage
|
||||||
) {
|
) {
|
||||||
|
|
|
@ -14,7 +14,9 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.coordinator.group.consumer;
|
package org.apache.kafka.coordinator.group.modern;
|
||||||
|
|
||||||
|
import org.apache.kafka.coordinator.group.consumer.CurrentAssignmentBuilder;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
|
@ -0,0 +1,632 @@
|
||||||
|
/*
|
||||||
|
* 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.modern;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.Uuid;
|
||||||
|
import org.apache.kafka.common.message.ListGroupsResponseData;
|
||||||
|
import org.apache.kafka.coordinator.group.Group;
|
||||||
|
import org.apache.kafka.coordinator.group.Utils;
|
||||||
|
import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType;
|
||||||
|
import org.apache.kafka.coordinator.group.consumer.Assignment;
|
||||||
|
import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
|
||||||
|
import org.apache.kafka.image.ClusterImage;
|
||||||
|
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.HashSet;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HETEROGENEOUS;
|
||||||
|
import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HOMOGENEOUS;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The abstract group provides definitions for the consumer and share group.
|
||||||
|
*/
|
||||||
|
public abstract class ModernGroup<T extends ModernGroupMember> implements Group {
|
||||||
|
|
||||||
|
public static class DeadlineAndEpoch {
|
||||||
|
static final DeadlineAndEpoch EMPTY = new DeadlineAndEpoch(0L, 0);
|
||||||
|
|
||||||
|
public final long deadlineMs;
|
||||||
|
public final int epoch;
|
||||||
|
|
||||||
|
DeadlineAndEpoch(long deadlineMs, int epoch) {
|
||||||
|
this.deadlineMs = deadlineMs;
|
||||||
|
this.epoch = epoch;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The snapshot registry.
|
||||||
|
*/
|
||||||
|
protected final SnapshotRegistry snapshotRegistry;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The group id.
|
||||||
|
*/
|
||||||
|
protected final String groupId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
protected final TimelineInteger groupEpoch;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The group members.
|
||||||
|
*/
|
||||||
|
protected final TimelineHashMap<String, T> members;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The number of subscribers per topic.
|
||||||
|
*/
|
||||||
|
protected final TimelineHashMap<String, Integer> subscribedTopicNames;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The metadata associated with each subscribed topic name.
|
||||||
|
*/
|
||||||
|
protected final TimelineHashMap<String, TopicMetadata> subscribedTopicMetadata;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The group's subscription type.
|
||||||
|
* This value is set to Homogeneous by default.
|
||||||
|
*/
|
||||||
|
protected final TimelineObject<SubscriptionType> subscriptionType;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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.
|
||||||
|
*/
|
||||||
|
protected final TimelineInteger targetAssignmentEpoch;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The target assignment per member id.
|
||||||
|
*/
|
||||||
|
protected final TimelineHashMap<String, Assignment> targetAssignment;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reverse lookup map representing topic partitions with
|
||||||
|
* their current member assignments.
|
||||||
|
*/
|
||||||
|
private final TimelineHashMap<Uuid, TimelineHashMap<Integer, String>> invertedTargetAssignment;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* its epochs from this map. When a member gets a partition, it adds its epochs to this map.
|
||||||
|
*/
|
||||||
|
protected final TimelineHashMap<Uuid, TimelineHashMap<Integer, Integer>> currentPartitionEpoch;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The metadata refresh deadline. It consists of a timestamp in milliseconds together with
|
||||||
|
* the group epoch at the time of setting it. The metadata refresh time is considered as a
|
||||||
|
* soft state (read that it is not stored in a timeline data structure). It is like this
|
||||||
|
* because it is not persisted to the log. The group epoch is here to ensure that the
|
||||||
|
* metadata refresh deadline is invalidated if the group epoch does not correspond to
|
||||||
|
* the current group epoch. This can happen if the metadata refresh deadline is updated
|
||||||
|
* after having refreshed the metadata but the write operation failed. In this case, the
|
||||||
|
* time is not automatically rolled back.
|
||||||
|
*/
|
||||||
|
protected DeadlineAndEpoch metadataRefreshDeadline = DeadlineAndEpoch.EMPTY;
|
||||||
|
|
||||||
|
protected ModernGroup(
|
||||||
|
SnapshotRegistry snapshotRegistry,
|
||||||
|
String groupId
|
||||||
|
) {
|
||||||
|
this.snapshotRegistry = Objects.requireNonNull(snapshotRegistry);
|
||||||
|
this.groupId = Objects.requireNonNull(groupId);
|
||||||
|
this.groupEpoch = new TimelineInteger(snapshotRegistry);
|
||||||
|
this.members = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
|
this.subscribedTopicNames = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
|
this.subscribedTopicMetadata = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
|
this.subscriptionType = new TimelineObject<>(snapshotRegistry, HOMOGENEOUS);
|
||||||
|
this.targetAssignmentEpoch = new TimelineInteger(snapshotRegistry);
|
||||||
|
this.targetAssignment = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
|
this.invertedTargetAssignment = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
|
this.currentPartitionEpoch = new TimelineHashMap<>(snapshotRegistry, 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the group formatted as a list group response based on the committed offset.
|
||||||
|
*/
|
||||||
|
public ListGroupsResponseData.ListedGroup asListedGroup(long committedOffset) {
|
||||||
|
return new ListGroupsResponseData.ListedGroup()
|
||||||
|
.setGroupId(groupId)
|
||||||
|
.setProtocolType(protocolType())
|
||||||
|
.setGroupState(stateAsString(committedOffset))
|
||||||
|
.setGroupType(type().toString());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The group id.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public String groupId() {
|
||||||
|
return groupId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @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();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The target assignment epoch.
|
||||||
|
*/
|
||||||
|
public int assignmentEpoch() {
|
||||||
|
return targetAssignmentEpoch.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the assignment epoch.
|
||||||
|
*
|
||||||
|
* @param targetAssignmentEpoch The new assignment epoch.
|
||||||
|
*/
|
||||||
|
public void setTargetAssignmentEpoch(int targetAssignmentEpoch) {
|
||||||
|
this.targetAssignmentEpoch.set(targetAssignmentEpoch);
|
||||||
|
maybeUpdateGroupState();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return A boolean indicating whether the member exists or not.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public boolean hasMember(String memberId) {
|
||||||
|
return members.containsKey(memberId);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The number of members.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public int numMembers() {
|
||||||
|
return members.size();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return An immutable Map containing all the members keyed by their id.
|
||||||
|
*/
|
||||||
|
public Map<String, T> members() {
|
||||||
|
return Collections.unmodifiableMap(members);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return An immutable map containing all the subscribed topic names
|
||||||
|
* with the subscribers counts per topic.
|
||||||
|
*/
|
||||||
|
public Map<String, Integer> subscribedTopicNames() {
|
||||||
|
return Collections.unmodifiableMap(subscribedTopicNames);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns true if the group is actively subscribed to the topic.
|
||||||
|
*
|
||||||
|
* @param topic The topic name.
|
||||||
|
*
|
||||||
|
* @return Whether the group is subscribed to the topic.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public boolean isSubscribedToTopic(String topic) {
|
||||||
|
return subscribedTopicNames.containsKey(topic);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The group's subscription type.
|
||||||
|
*/
|
||||||
|
public SubscriptionType subscriptionType() {
|
||||||
|
return subscriptionType.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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 targetAssignment.getOrDefault(memberId, Assignment.EMPTY);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return An immutable map containing all the topic partitions
|
||||||
|
* with their current member assignments.
|
||||||
|
*/
|
||||||
|
public Map<Uuid, Map<Integer, String>> invertedTargetAssignment() {
|
||||||
|
return Collections.unmodifiableMap(invertedTargetAssignment);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates target assignment of a member.
|
||||||
|
*
|
||||||
|
* @param memberId The member id.
|
||||||
|
* @param newTargetAssignment The new target assignment.
|
||||||
|
*/
|
||||||
|
public void updateTargetAssignment(String memberId, Assignment newTargetAssignment) {
|
||||||
|
updateInvertedTargetAssignment(
|
||||||
|
memberId,
|
||||||
|
targetAssignment.getOrDefault(memberId, new Assignment(Collections.emptyMap())),
|
||||||
|
newTargetAssignment
|
||||||
|
);
|
||||||
|
targetAssignment.put(memberId, newTargetAssignment);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates the reverse lookup map of the target assignment.
|
||||||
|
*
|
||||||
|
* @param memberId The member Id.
|
||||||
|
* @param oldTargetAssignment The old target assignment.
|
||||||
|
* @param newTargetAssignment The new target assignment.
|
||||||
|
*/
|
||||||
|
private void updateInvertedTargetAssignment(
|
||||||
|
String memberId,
|
||||||
|
Assignment oldTargetAssignment,
|
||||||
|
Assignment newTargetAssignment
|
||||||
|
) {
|
||||||
|
// Combine keys from both old and new assignments.
|
||||||
|
Set<Uuid> allTopicIds = new HashSet<>();
|
||||||
|
allTopicIds.addAll(oldTargetAssignment.partitions().keySet());
|
||||||
|
allTopicIds.addAll(newTargetAssignment.partitions().keySet());
|
||||||
|
|
||||||
|
for (Uuid topicId : allTopicIds) {
|
||||||
|
Set<Integer> oldPartitions = oldTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet());
|
||||||
|
Set<Integer> newPartitions = newTargetAssignment.partitions().getOrDefault(topicId, Collections.emptySet());
|
||||||
|
|
||||||
|
TimelineHashMap<Integer, String> topicPartitionAssignment = invertedTargetAssignment.computeIfAbsent(
|
||||||
|
topicId, k -> new TimelineHashMap<>(snapshotRegistry, Math.max(oldPartitions.size(), newPartitions.size()))
|
||||||
|
);
|
||||||
|
|
||||||
|
// Remove partitions that aren't present in the new assignment only if the partition is currently
|
||||||
|
// still assigned to the member in question.
|
||||||
|
// If p0 was moved from A to B, and the target assignment map was updated for B first, we don't want to
|
||||||
|
// remove the key p0 from the inverted map and undo the action when A eventually tries to update its assignment.
|
||||||
|
for (Integer partition : oldPartitions) {
|
||||||
|
if (!newPartitions.contains(partition) && memberId.equals(topicPartitionAssignment.get(partition))) {
|
||||||
|
topicPartitionAssignment.remove(partition);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Add partitions that are in the new assignment but not in the old assignment.
|
||||||
|
for (Integer partition : newPartitions) {
|
||||||
|
if (!oldPartitions.contains(partition)) {
|
||||||
|
topicPartitionAssignment.put(partition, memberId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (topicPartitionAssignment.isEmpty()) {
|
||||||
|
invertedTargetAssignment.remove(topicId);
|
||||||
|
} else {
|
||||||
|
invertedTargetAssignment.put(topicId, topicPartitionAssignment);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Removes the target assignment of a member.
|
||||||
|
*
|
||||||
|
* @param memberId The member id.
|
||||||
|
*/
|
||||||
|
public void removeTargetAssignment(String memberId) {
|
||||||
|
updateInvertedTargetAssignment(
|
||||||
|
memberId,
|
||||||
|
targetAssignment.getOrDefault(memberId, Assignment.EMPTY),
|
||||||
|
Assignment.EMPTY
|
||||||
|
);
|
||||||
|
targetAssignment.remove(memberId);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return An immutable Map containing all the target assignment keyed by member id.
|
||||||
|
*/
|
||||||
|
public Map<String, Assignment> targetAssignment() {
|
||||||
|
return Collections.unmodifiableMap(targetAssignment);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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<Integer, Integer> partitions = currentPartitionEpoch.get(topicId);
|
||||||
|
if (partitions == null) {
|
||||||
|
return -1;
|
||||||
|
} else {
|
||||||
|
return partitions.getOrDefault(partitionId, -1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return An immutable Map of subscription metadata for
|
||||||
|
* each topic that the consumer group is subscribed to.
|
||||||
|
*/
|
||||||
|
public Map<String, TopicMetadata> subscriptionMetadata() {
|
||||||
|
return Collections.unmodifiableMap(subscribedTopicMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates the subscription metadata. This replaces the previous one.
|
||||||
|
*
|
||||||
|
* @param subscriptionMetadata The new subscription metadata.
|
||||||
|
*/
|
||||||
|
public void setSubscriptionMetadata(
|
||||||
|
Map<String, TopicMetadata> subscriptionMetadata
|
||||||
|
) {
|
||||||
|
this.subscribedTopicMetadata.clear();
|
||||||
|
this.subscribedTopicMetadata.putAll(subscriptionMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Computes the subscription metadata based on the current subscription info.
|
||||||
|
*
|
||||||
|
* @param subscribedTopicNames Map of topic names to the number of subscribers.
|
||||||
|
* @param topicsImage The current metadata for all available topics.
|
||||||
|
* @param clusterImage The current metadata for the Kafka cluster.
|
||||||
|
*
|
||||||
|
* @return An immutable map of subscription metadata for each topic that the consumer group is subscribed to.
|
||||||
|
*/
|
||||||
|
public Map<String, TopicMetadata> computeSubscriptionMetadata(
|
||||||
|
Map<String, Integer> subscribedTopicNames,
|
||||||
|
TopicsImage topicsImage,
|
||||||
|
ClusterImage clusterImage
|
||||||
|
) {
|
||||||
|
// Create the topic metadata for each subscribed topic.
|
||||||
|
Map<String, TopicMetadata> newSubscriptionMetadata = new HashMap<>(subscribedTopicNames.size());
|
||||||
|
|
||||||
|
subscribedTopicNames.forEach((topicName, count) -> {
|
||||||
|
TopicImage topicImage = topicsImage.getTopic(topicName);
|
||||||
|
if (topicImage != null) {
|
||||||
|
Map<Integer, Set<String>> partitionRacks = new HashMap<>();
|
||||||
|
topicImage.partitions().forEach((partition, partitionRegistration) -> {
|
||||||
|
Set<String> racks = new HashSet<>();
|
||||||
|
for (int replica : partitionRegistration.replicas) {
|
||||||
|
Optional<String> rackOptional = clusterImage.broker(replica).rack();
|
||||||
|
// Only add the rack if it is available for the broker/replica.
|
||||||
|
rackOptional.ifPresent(racks::add);
|
||||||
|
}
|
||||||
|
// If rack information is unavailable for all replicas of this partition,
|
||||||
|
// no corresponding entry will be stored for it in the map.
|
||||||
|
if (!racks.isEmpty())
|
||||||
|
partitionRacks.put(partition, racks);
|
||||||
|
});
|
||||||
|
|
||||||
|
newSubscriptionMetadata.put(topicName, new TopicMetadata(
|
||||||
|
topicImage.id(),
|
||||||
|
topicImage.name(),
|
||||||
|
topicImage.partitions().size(),
|
||||||
|
partitionRacks)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
return Collections.unmodifiableMap(newSubscriptionMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates the metadata refresh deadline.
|
||||||
|
*
|
||||||
|
* @param deadlineMs The deadline in milliseconds.
|
||||||
|
* @param groupEpoch The associated group epoch.
|
||||||
|
*/
|
||||||
|
public void setMetadataRefreshDeadline(
|
||||||
|
long deadlineMs,
|
||||||
|
int groupEpoch
|
||||||
|
) {
|
||||||
|
this.metadataRefreshDeadline = new DeadlineAndEpoch(deadlineMs, groupEpoch);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Requests a metadata refresh.
|
||||||
|
*/
|
||||||
|
public void requestMetadataRefresh() {
|
||||||
|
this.metadataRefreshDeadline = DeadlineAndEpoch.EMPTY;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Checks if a metadata refresh is required. A refresh is required in two cases:
|
||||||
|
* 1) The deadline is smaller or equal to the current time;
|
||||||
|
* 2) The group epoch associated with the deadline is larger than
|
||||||
|
* the current group epoch. This means that the operations which updated
|
||||||
|
* the deadline failed.
|
||||||
|
*
|
||||||
|
* @param currentTimeMs The current time in milliseconds.
|
||||||
|
* @return A boolean indicating whether a refresh is required or not.
|
||||||
|
*/
|
||||||
|
public boolean hasMetadataExpired(long currentTimeMs) {
|
||||||
|
return currentTimeMs >= metadataRefreshDeadline.deadlineMs || groupEpoch() < metadataRefreshDeadline.epoch;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The metadata refresh deadline.
|
||||||
|
*/
|
||||||
|
public DeadlineAndEpoch metadataRefreshDeadline() {
|
||||||
|
return metadataRefreshDeadline;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates the subscribed topic names count.
|
||||||
|
* The subscription type is updated as a consequence.
|
||||||
|
*
|
||||||
|
* @param oldMember The old member.
|
||||||
|
* @param newMember The new member.
|
||||||
|
*/
|
||||||
|
protected void maybeUpdateSubscribedTopicNamesAndGroupSubscriptionType(
|
||||||
|
ModernGroupMember oldMember,
|
||||||
|
ModernGroupMember newMember
|
||||||
|
) {
|
||||||
|
maybeUpdateSubscribedTopicNames(subscribedTopicNames, oldMember, newMember);
|
||||||
|
subscriptionType.set(subscriptionType(subscribedTopicNames, members.size()));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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<String, Integer> subscribedTopicCount,
|
||||||
|
ModernGroupMember oldMember,
|
||||||
|
ModernGroupMember newMember
|
||||||
|
) {
|
||||||
|
if (oldMember != null) {
|
||||||
|
oldMember.subscribedTopicNames().forEach(topicName ->
|
||||||
|
subscribedTopicCount.compute(topicName, Utils::decValue)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (newMember != null) {
|
||||||
|
newMember.subscribedTopicNames().forEach(topicName ->
|
||||||
|
subscribedTopicCount.compute(topicName, Utils::incValue)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates the subscription count.
|
||||||
|
*
|
||||||
|
* @param oldMember The old member.
|
||||||
|
* @param newMember The new member.
|
||||||
|
*
|
||||||
|
* @return Copy of the map of topics to the count of number of subscribers.
|
||||||
|
*/
|
||||||
|
public Map<String, Integer> computeSubscribedTopicNames(
|
||||||
|
ModernGroupMember oldMember,
|
||||||
|
ModernGroupMember newMember
|
||||||
|
) {
|
||||||
|
Map<String, Integer> subscribedTopicNames = new HashMap<>(this.subscribedTopicNames);
|
||||||
|
maybeUpdateSubscribedTopicNames(
|
||||||
|
subscribedTopicNames,
|
||||||
|
oldMember,
|
||||||
|
newMember
|
||||||
|
);
|
||||||
|
return subscribedTopicNames;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates the subscription count with a set of members removed.
|
||||||
|
*
|
||||||
|
* @param removedMembers The set of removed members.
|
||||||
|
*
|
||||||
|
* @return Copy of the map of topics to the count of number of subscribers.
|
||||||
|
*/
|
||||||
|
public Map<String, Integer> computeSubscribedTopicNames(
|
||||||
|
Set<? extends ModernGroupMember> removedMembers
|
||||||
|
) {
|
||||||
|
Map<String, Integer> subscribedTopicNames = new HashMap<>(this.subscribedTopicNames);
|
||||||
|
if (removedMembers != null) {
|
||||||
|
removedMembers.forEach(removedMember ->
|
||||||
|
maybeUpdateSubscribedTopicNames(
|
||||||
|
subscribedTopicNames,
|
||||||
|
removedMember,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
);
|
||||||
|
}
|
||||||
|
return subscribedTopicNames;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Compute the subscription type of the consumer group.
|
||||||
|
*
|
||||||
|
* @param subscribedTopicNames A map of topic names to the count of members subscribed to each topic.
|
||||||
|
*
|
||||||
|
* @return {@link SubscriptionType#HOMOGENEOUS} if all members are subscribed to exactly the same topics;
|
||||||
|
* otherwise, {@link SubscriptionType#HETEROGENEOUS}.
|
||||||
|
*/
|
||||||
|
public static SubscriptionType subscriptionType(
|
||||||
|
Map<String, Integer> subscribedTopicNames,
|
||||||
|
int numberOfMembers
|
||||||
|
) {
|
||||||
|
if (subscribedTopicNames.isEmpty()) {
|
||||||
|
return HOMOGENEOUS;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int subscriberCount : subscribedTopicNames.values()) {
|
||||||
|
if (subscriberCount != numberOfMembers) {
|
||||||
|
return HETEROGENEOUS;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return HOMOGENEOUS;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Gets the protocol type for the group.
|
||||||
|
*
|
||||||
|
* @return The group protocol type.
|
||||||
|
*/
|
||||||
|
public abstract String protocolType();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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 abstract T getOrMaybeCreateMember(String memberId, boolean createIfNotExists);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adds or updates the member.
|
||||||
|
*
|
||||||
|
* @param newMember The new member state.
|
||||||
|
*/
|
||||||
|
public abstract void updateMember(T newMember);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Remove the member from the group.
|
||||||
|
*
|
||||||
|
* @param memberId The member id to remove.
|
||||||
|
*/
|
||||||
|
public abstract void removeMember(String memberId);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Updates the current state of the group.
|
||||||
|
*/
|
||||||
|
protected abstract void maybeUpdateGroupState();
|
||||||
|
}
|
|
@ -0,0 +1,203 @@
|
||||||
|
/*
|
||||||
|
* 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.modern;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.Uuid;
|
||||||
|
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Abstract member common for group members.
|
||||||
|
*/
|
||||||
|
public abstract class ModernGroupMember {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The member id.
|
||||||
|
*/
|
||||||
|
protected String memberId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The current member epoch.
|
||||||
|
*/
|
||||||
|
protected int memberEpoch;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The previous member epoch.
|
||||||
|
*/
|
||||||
|
protected int previousMemberEpoch;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The member state.
|
||||||
|
*/
|
||||||
|
protected MemberState state;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The instance id provided by the member.
|
||||||
|
*/
|
||||||
|
protected String instanceId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The rack id provided by the member.
|
||||||
|
*/
|
||||||
|
protected String rackId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The rebalance timeout provided by the member.
|
||||||
|
*/
|
||||||
|
protected int rebalanceTimeoutMs;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The client id reported by the member.
|
||||||
|
*/
|
||||||
|
protected String clientId;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The host reported by the member.
|
||||||
|
*/
|
||||||
|
protected String clientHost;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The list of subscriptions (topic names) configured by the member.
|
||||||
|
*/
|
||||||
|
protected Set<String> subscribedTopicNames;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The partitions assigned to this member.
|
||||||
|
*/
|
||||||
|
protected Map<Uuid, Set<Integer>> assignedPartitions;
|
||||||
|
|
||||||
|
protected ModernGroupMember(
|
||||||
|
String memberId,
|
||||||
|
int memberEpoch,
|
||||||
|
int previousMemberEpoch,
|
||||||
|
String instanceId,
|
||||||
|
String rackId,
|
||||||
|
int rebalanceTimeoutMs,
|
||||||
|
String clientId,
|
||||||
|
String clientHost,
|
||||||
|
Set<String> subscribedTopicNames,
|
||||||
|
MemberState state,
|
||||||
|
Map<Uuid, Set<Integer>> assignedPartitions
|
||||||
|
) {
|
||||||
|
this.memberId = memberId;
|
||||||
|
this.memberEpoch = memberEpoch;
|
||||||
|
this.previousMemberEpoch = previousMemberEpoch;
|
||||||
|
this.state = state;
|
||||||
|
this.instanceId = instanceId;
|
||||||
|
this.rackId = rackId;
|
||||||
|
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
|
||||||
|
this.clientId = clientId;
|
||||||
|
this.clientHost = clientHost;
|
||||||
|
this.subscribedTopicNames = subscribedTopicNames;
|
||||||
|
this.assignedPartitions = assignedPartitions;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The member id.
|
||||||
|
*/
|
||||||
|
public String memberId() {
|
||||||
|
return memberId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The current member epoch.
|
||||||
|
*/
|
||||||
|
public int memberEpoch() {
|
||||||
|
return memberEpoch;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The previous member epoch.
|
||||||
|
*/
|
||||||
|
public int previousMemberEpoch() {
|
||||||
|
return previousMemberEpoch;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The instance id.
|
||||||
|
*/
|
||||||
|
public String instanceId() {
|
||||||
|
return instanceId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The rack id.
|
||||||
|
*/
|
||||||
|
public String rackId() {
|
||||||
|
return rackId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The rebalance timeout in millis.
|
||||||
|
*/
|
||||||
|
public int rebalanceTimeoutMs() {
|
||||||
|
return rebalanceTimeoutMs;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The client id.
|
||||||
|
*/
|
||||||
|
public String clientId() {
|
||||||
|
return clientId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The client host.
|
||||||
|
*/
|
||||||
|
public String clientHost() {
|
||||||
|
return clientHost;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The list of subscribed topic names.
|
||||||
|
*/
|
||||||
|
public Set<String> subscribedTopicNames() {
|
||||||
|
return subscribedTopicNames;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The current state.
|
||||||
|
*/
|
||||||
|
public MemberState state() {
|
||||||
|
return state;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return True if the member is in the Stable state and at the desired epoch.
|
||||||
|
*/
|
||||||
|
public boolean isReconciledTo(int targetAssignmentEpoch) {
|
||||||
|
return state == MemberState.STABLE && memberEpoch == targetAssignmentEpoch;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The set of assigned partitions.
|
||||||
|
*/
|
||||||
|
public Map<Uuid, Set<Integer>> assignedPartitions() {
|
||||||
|
return assignedPartitions;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return True of the two provided members have different assigned partitions.
|
||||||
|
*/
|
||||||
|
public static boolean hasAssignedPartitionsChanged(
|
||||||
|
ModernGroupMember member1,
|
||||||
|
ModernGroupMember member2
|
||||||
|
) {
|
||||||
|
return !member1.assignedPartitions().equals(member2.assignedPartitions());
|
||||||
|
}
|
||||||
|
}
|
|
@ -26,7 +26,6 @@ import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.consumer.MemberState;
|
|
||||||
import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
|
import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
|
||||||
|
@ -45,6 +44,7 @@ import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||||
import org.apache.kafka.server.common.MetadataVersion;
|
import org.apache.kafka.server.common.MetadataVersion;
|
||||||
|
|
||||||
|
|
|
@ -73,11 +73,11 @@ import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupBuilder;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupBuilder;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.consumer.MemberAssignmentImpl;
|
import org.apache.kafka.coordinator.group.consumer.MemberAssignmentImpl;
|
||||||
import org.apache.kafka.coordinator.group.consumer.MemberState;
|
|
||||||
import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
|
import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
||||||
import org.apache.kafka.image.MetadataDelta;
|
import org.apache.kafka.image.MetadataDelta;
|
||||||
import org.apache.kafka.image.MetadataImage;
|
import org.apache.kafka.image.MetadataImage;
|
||||||
|
@ -3522,7 +3522,7 @@ public class GroupMetadataManagerTest {
|
||||||
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
|
||||||
|
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -3621,7 +3621,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
List<String> memberIds = verifyClassicGroupJoinResponses(firstRoundJoinResults, 0, Errors.MEMBER_ID_REQUIRED);
|
List<String> memberIds = verifyClassicGroupJoinResponses(firstRoundJoinResults, 0, Errors.MEMBER_ID_REQUIRED);
|
||||||
assertEquals(groupMaxSize + 1, memberIds.size());
|
assertEquals(groupMaxSize + 1, memberIds.size());
|
||||||
assertEquals(0, group.size());
|
assertEquals(0, group.numMembers());
|
||||||
assertTrue(group.isInState(EMPTY));
|
assertTrue(group.isInState(EMPTY));
|
||||||
assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
|
assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
|
||||||
|
|
||||||
|
@ -3641,7 +3641,7 @@ public class GroupMetadataManagerTest {
|
||||||
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(50));
|
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(50));
|
||||||
|
|
||||||
verifyClassicGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
verifyClassicGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(0, group.numPendingJoinMembers());
|
assertEquals(0, group.numPendingJoinMembers());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -3677,7 +3677,7 @@ public class GroupMetadataManagerTest {
|
||||||
requiredKnownMemberId
|
requiredKnownMemberId
|
||||||
)).collect(Collectors.toList());
|
)).collect(Collectors.toList());
|
||||||
|
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
|
|
||||||
|
@ -3698,7 +3698,7 @@ public class GroupMetadataManagerTest {
|
||||||
)).collect(Collectors.toList());
|
)).collect(Collectors.toList());
|
||||||
|
|
||||||
verifyClassicGroupJoinResponses(secondRoundJoinResults, 10, Errors.GROUP_MAX_SIZE_REACHED);
|
verifyClassicGroupJoinResponses(secondRoundJoinResults, 10, Errors.GROUP_MAX_SIZE_REACHED);
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(0, group.numAwaitingJoinResponse());
|
assertEquals(0, group.numAwaitingJoinResponse());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
}
|
}
|
||||||
|
@ -3729,7 +3729,7 @@ public class GroupMetadataManagerTest {
|
||||||
.map(instanceId -> context.sendClassicGroupJoin(request.setGroupInstanceId(instanceId)))
|
.map(instanceId -> context.sendClassicGroupJoin(request.setGroupInstanceId(instanceId)))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
|
|
||||||
|
@ -3752,7 +3752,7 @@ public class GroupMetadataManagerTest {
|
||||||
)).collect(Collectors.toList());
|
)).collect(Collectors.toList());
|
||||||
|
|
||||||
verifyClassicGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
verifyClassicGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(0, group.numAwaitingJoinResponse());
|
assertEquals(0, group.numAwaitingJoinResponse());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
}
|
}
|
||||||
|
@ -3779,7 +3779,7 @@ public class GroupMetadataManagerTest {
|
||||||
.mapToObj(__ -> context.sendClassicGroupJoin(request, requiredKnownMemberId))
|
.mapToObj(__ -> context.sendClassicGroupJoin(request, requiredKnownMemberId))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
assertEquals(0, group.size());
|
assertEquals(0, group.numMembers());
|
||||||
assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
|
assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
|
||||||
assertTrue(group.isInState(EMPTY));
|
assertTrue(group.isInState(EMPTY));
|
||||||
|
|
||||||
|
@ -3796,7 +3796,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(joinResult.records.isEmpty());
|
assertTrue(joinResult.records.isEmpty());
|
||||||
});
|
});
|
||||||
|
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
|
|
||||||
|
@ -3815,7 +3815,7 @@ public class GroupMetadataManagerTest {
|
||||||
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(50));
|
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(50));
|
||||||
|
|
||||||
verifyClassicGroupJoinResponses(thirdRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
verifyClassicGroupJoinResponses(thirdRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(0, group.numAwaitingJoinResponse());
|
assertEquals(0, group.numAwaitingJoinResponse());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
}
|
}
|
||||||
|
@ -3860,7 +3860,7 @@ public class GroupMetadataManagerTest {
|
||||||
.build()
|
.build()
|
||||||
)).collect(Collectors.toList());
|
)).collect(Collectors.toList());
|
||||||
|
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
|
|
||||||
|
@ -3869,14 +3869,14 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
verifyClassicGroupJoinResponses(joinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
verifyClassicGroupJoinResponses(joinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
|
||||||
|
|
||||||
assertEquals(groupMaxSize, group.size());
|
assertEquals(groupMaxSize, group.numMembers());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
memberIds.subList(groupMaxSize, groupMaxSize + 2)
|
memberIds.subList(groupMaxSize, groupMaxSize + 2)
|
||||||
.forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
|
.forEach(memberId -> assertFalse(group.hasMember(memberId)));
|
||||||
|
|
||||||
memberIds.subList(0, groupMaxSize)
|
memberIds.subList(0, groupMaxSize)
|
||||||
.forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
|
.forEach(memberId -> assertTrue(group.hasMember(memberId)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -4378,7 +4378,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(oldMemberJoinResult.records.isEmpty());
|
assertTrue(oldMemberJoinResult.records.isEmpty());
|
||||||
assertFalse(oldMemberJoinResult.joinFuture.isDone());
|
assertFalse(oldMemberJoinResult.joinFuture.isDone());
|
||||||
assertEquals(1, group.numAwaitingJoinResponse());
|
assertEquals(1, group.numAwaitingJoinResponse());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
|
|
||||||
// Replace static member with new member id. Old member id should be fenced.
|
// Replace static member with new member id. Old member id should be fenced.
|
||||||
GroupMetadataManagerTestContext.JoinResult newMemberJoinResult = context.sendClassicGroupJoin(request);
|
GroupMetadataManagerTestContext.JoinResult newMemberJoinResult = context.sendClassicGroupJoin(request);
|
||||||
|
@ -4388,14 +4388,14 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(oldMemberJoinResult.joinFuture.isDone());
|
assertTrue(oldMemberJoinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(1, group.numAwaitingJoinResponse());
|
assertEquals(1, group.numAwaitingJoinResponse());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
|
|
||||||
// Complete join for new member.
|
// Complete join for new member.
|
||||||
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
|
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
|
||||||
assertTrue(newMemberJoinResult.joinFuture.isDone());
|
assertTrue(newMemberJoinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), newMemberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), newMemberJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(0, group.numAwaitingJoinResponse());
|
assertEquals(0, group.numAwaitingJoinResponse());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -4416,7 +4416,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(joinResult.records.isEmpty());
|
assertTrue(joinResult.records.isEmpty());
|
||||||
assertTrue(joinResult.joinFuture.isDone());
|
assertTrue(joinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.MEMBER_ID_REQUIRED.code(), joinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.MEMBER_ID_REQUIRED.code(), joinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(0, group.size());
|
assertEquals(0, group.numMembers());
|
||||||
assertEquals(1, group.numPendingJoinMembers());
|
assertEquals(1, group.numPendingJoinMembers());
|
||||||
|
|
||||||
// Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
|
// Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
|
||||||
|
@ -4442,7 +4442,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(joinResult.records.isEmpty());
|
assertTrue(joinResult.records.isEmpty());
|
||||||
assertFalse(joinResult.joinFuture.isDone());
|
assertFalse(joinResult.joinFuture.isDone());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
|
|
||||||
String memberId = group.leaderOrNull();
|
String memberId = group.leaderOrNull();
|
||||||
// Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
|
// Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
|
||||||
|
@ -4460,7 +4460,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(joinResult.joinFuture.isDone());
|
assertTrue(joinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), joinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), joinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(0, group.size());
|
assertEquals(0, group.numMembers());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -4479,7 +4479,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(Errors.NONE.code(), response.errorCode());
|
assertEquals(Errors.NONE.code(), response.errorCode());
|
||||||
String memberId = response.memberId();
|
String memberId = response.memberId();
|
||||||
|
|
||||||
assertTrue(group.hasMemberId(memberId));
|
assertTrue(group.hasMember(memberId));
|
||||||
|
|
||||||
group.transitionTo(DEAD);
|
group.transitionTo(DEAD);
|
||||||
|
|
||||||
|
@ -4630,7 +4630,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(Errors.NONE.code(), leaderJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), leaderJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(Errors.NONE.code(), memberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), memberJoinResult.joinFuture.get().errorCode());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
|
|
||||||
group.transitionTo(STABLE);
|
group.transitionTo(STABLE);
|
||||||
|
@ -4658,7 +4658,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(Errors.NONE.code(), memberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), memberJoinResult.joinFuture.get().errorCode());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
assertEquals(3, group.generationId());
|
assertEquals(3, group.generationId());
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -4691,7 +4691,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(leaderJoinResult.joinFuture.isDone());
|
assertTrue(leaderJoinResult.joinFuture.isDone());
|
||||||
assertTrue(memberJoinResult.joinFuture.isDone());
|
assertTrue(memberJoinResult.joinFuture.isDone());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
|
|
||||||
String memberId = memberJoinResult.joinFuture.get().memberId();
|
String memberId = memberJoinResult.joinFuture.get().memberId();
|
||||||
|
@ -4762,7 +4762,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(memberJoinResult.records.isEmpty());
|
assertTrue(memberJoinResult.records.isEmpty());
|
||||||
assertFalse(memberJoinResult.joinFuture.isDone());
|
assertFalse(memberJoinResult.joinFuture.isDone());
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
|
|
||||||
// Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
|
// Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
|
||||||
|
@ -4770,8 +4770,8 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(memberJoinResult.joinFuture.isDone());
|
assertTrue(memberJoinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), memberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), memberJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertTrue(group.hasMemberId(memberJoinResult.joinFuture.get().memberId()));
|
assertTrue(group.hasMember(memberJoinResult.joinFuture.get().memberId()));
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4840,7 +4840,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(secondMemberJoinResult.joinFuture.isDone());
|
assertTrue(secondMemberJoinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), firstMemberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), firstMemberJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(Errors.NONE.code(), secondMemberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), secondMemberJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
|
|
||||||
String secondMemberId = secondMemberJoinResult.joinFuture.get().memberId();
|
String secondMemberId = secondMemberJoinResult.joinFuture.get().memberId();
|
||||||
|
@ -4848,7 +4848,7 @@ public class GroupMetadataManagerTest {
|
||||||
// Trigger a rebalance. No members rejoined.
|
// Trigger a rebalance. No members rejoined.
|
||||||
context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
|
context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
|
||||||
|
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
assertEquals(0, group.numAwaitingJoinResponse());
|
assertEquals(0, group.numAwaitingJoinResponse());
|
||||||
|
|
||||||
|
@ -4860,7 +4860,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(10000, context.timer.timeout("join-group-id").deadlineMs - context.time.milliseconds());
|
assertEquals(10000, context.timer.timeout("join-group-id").deadlineMs - context.time.milliseconds());
|
||||||
|
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
|
|
||||||
// Let first and second member rejoin. This should complete the join phase.
|
// Let first and second member rejoin. This should complete the join phase.
|
||||||
|
@ -4873,7 +4873,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(firstMemberJoinResult.records.isEmpty());
|
assertTrue(firstMemberJoinResult.records.isEmpty());
|
||||||
assertFalse(firstMemberJoinResult.joinFuture.isDone());
|
assertFalse(firstMemberJoinResult.joinFuture.isDone());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
|
|
||||||
secondMemberJoinResult = context.sendClassicGroupJoin(
|
secondMemberJoinResult = context.sendClassicGroupJoin(
|
||||||
|
@ -4888,7 +4888,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(Errors.NONE.code(), firstMemberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), firstMemberJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(Errors.NONE.code(), secondMemberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), secondMemberJoinResult.joinFuture.get().errorCode());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(3, group.generationId());
|
assertEquals(3, group.generationId());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -4913,7 +4913,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(Errors.NONE.code(), response.errorCode());
|
assertEquals(Errors.NONE.code(), response.errorCode());
|
||||||
String oldMemberId = response.memberId();
|
String oldMemberId = response.memberId();
|
||||||
|
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -4974,7 +4974,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(4500, updatedMember.sessionTimeoutMs());
|
assertEquals(4500, updatedMember.sessionTimeoutMs());
|
||||||
assertEquals(protocols, updatedMember.supportedProtocols());
|
assertEquals(protocols, updatedMember.supportedProtocols());
|
||||||
|
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(STABLE));
|
assertTrue(group.isInState(STABLE));
|
||||||
}
|
}
|
||||||
|
@ -4995,7 +4995,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, true, true);
|
JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, true, true);
|
||||||
assertEquals(Errors.NONE.code(), response.errorCode());
|
assertEquals(Errors.NONE.code(), response.errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -5010,7 +5010,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(joinResult.records.isEmpty());
|
assertTrue(joinResult.records.isEmpty());
|
||||||
assertTrue(joinResult.joinFuture.isDone());
|
assertTrue(joinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
}
|
}
|
||||||
|
@ -5036,7 +5036,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, false, false);
|
JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, false, false);
|
||||||
assertEquals(Errors.NONE.code(), response.errorCode());
|
assertEquals(Errors.NONE.code(), response.errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -5088,7 +5088,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(4000, revertedMember.rebalanceTimeoutMs());
|
assertEquals(4000, revertedMember.rebalanceTimeoutMs());
|
||||||
assertEquals(3000, revertedMember.sessionTimeoutMs());
|
assertEquals(3000, revertedMember.sessionTimeoutMs());
|
||||||
assertEquals(protocols, revertedMember.supportedProtocols());
|
assertEquals(protocols, revertedMember.supportedProtocols());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(STABLE));
|
assertTrue(group.isInState(STABLE));
|
||||||
}
|
}
|
||||||
|
@ -5118,7 +5118,7 @@ public class GroupMetadataManagerTest {
|
||||||
);
|
);
|
||||||
|
|
||||||
assertEquals(Errors.NONE.code(), response.errorCode());
|
assertEquals(Errors.NONE.code(), response.errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -5164,7 +5164,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(7000, newMember.rebalanceTimeoutMs());
|
assertEquals(7000, newMember.rebalanceTimeoutMs());
|
||||||
assertEquals(6000, newMember.sessionTimeoutMs());
|
assertEquals(6000, newMember.sessionTimeoutMs());
|
||||||
assertEquals(GroupMetadataManagerTestContext.toProtocols("range", "roundrobin"), newMember.supportedProtocols());
|
assertEquals(GroupMetadataManagerTestContext.toProtocols("range", "roundrobin"), newMember.supportedProtocols());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(STABLE));
|
assertTrue(group.isInState(STABLE));
|
||||||
}
|
}
|
||||||
|
@ -5185,7 +5185,7 @@ public class GroupMetadataManagerTest {
|
||||||
JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, true, true);
|
JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, true, true);
|
||||||
assertEquals(Errors.NONE.code(), response.errorCode());
|
assertEquals(Errors.NONE.code(), response.errorCode());
|
||||||
|
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -5195,7 +5195,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(joinResult.records.isEmpty());
|
assertTrue(joinResult.records.isEmpty());
|
||||||
assertTrue(joinResult.joinFuture.isDone());
|
assertTrue(joinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
}
|
}
|
||||||
|
@ -5257,11 +5257,11 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(syncResult.syncFuture.isDone());
|
assertTrue(syncResult.syncFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
|
|
||||||
// Make sure the NewMemberTimeout is not still in effect, and the member is not kicked
|
// Make sure the NewMemberTimeout is not still in effect, and the member is not kicked
|
||||||
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(context.classicGroupNewMemberJoinTimeoutMs));
|
GroupMetadataManagerTestContext.assertNoOrEmptyResult(context.sleep(context.classicGroupNewMemberJoinTimeoutMs));
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
|
|
||||||
// Member should be removed as heartbeat expires. The group is now empty.
|
// Member should be removed as heartbeat expires. The group is now empty.
|
||||||
List<ExpiredTimeout<Void, CoordinatorRecord>> timeouts = context.sleep(5000);
|
List<ExpiredTimeout<Void, CoordinatorRecord>> timeouts = context.sleep(5000);
|
||||||
|
@ -5284,7 +5284,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertEquals(expectedRecords, timeout.result.records());
|
assertEquals(expectedRecords, timeout.result.records());
|
||||||
});
|
});
|
||||||
|
|
||||||
assertEquals(0, group.size());
|
assertEquals(0, group.numMembers());
|
||||||
assertTrue(group.isInState(EMPTY));
|
assertTrue(group.isInState(EMPTY));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -5521,8 +5521,8 @@ public class GroupMetadataManagerTest {
|
||||||
assertTrue(duplicateFollowerJoinResult.joinFuture.isDone());
|
assertTrue(duplicateFollowerJoinResult.joinFuture.isDone());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
assertEquals(3, group.generationId());
|
assertEquals(3, group.generationId());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertTrue(group.hasMemberId(duplicateFollowerJoinResult.joinFuture.get().memberId()));
|
assertTrue(group.hasMember(duplicateFollowerJoinResult.joinFuture.get().memberId()));
|
||||||
assertEquals(duplicateFollowerJoinResult.joinFuture.get().memberId(), duplicateFollowerJoinResult.joinFuture.get().leader());
|
assertEquals(duplicateFollowerJoinResult.joinFuture.get().memberId(), duplicateFollowerJoinResult.joinFuture.get().leader());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -5792,7 +5792,7 @@ public class GroupMetadataManagerTest {
|
||||||
JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(request, true, true, 10000);
|
JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(request, true, true, 10000);
|
||||||
|
|
||||||
// Follower's heartbeat expires as the leader rejoins.
|
// Follower's heartbeat expires as the leader rejoins.
|
||||||
assertFalse(group.hasMemberId(rebalanceResult.followerId));
|
assertFalse(group.hasMember(rebalanceResult.followerId));
|
||||||
|
|
||||||
JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
|
JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
|
||||||
.setErrorCode(Errors.NONE.code())
|
.setErrorCode(Errors.NONE.code())
|
||||||
|
@ -7152,8 +7152,8 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(followerSyncResult.syncFuture.isDone());
|
assertTrue(followerSyncResult.syncFuture.isDone());
|
||||||
assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), followerSyncResult.syncFuture.get().errorCode());
|
assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), followerSyncResult.syncFuture.get().errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertTrue(group.hasMemberId(followerId));
|
assertTrue(group.hasMember(followerId));
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -7709,7 +7709,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(otherJoinResult.joinFuture.isDone());
|
assertTrue(otherJoinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), otherJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), otherJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
}
|
}
|
||||||
|
@ -7785,7 +7785,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertTrue(secondMemberJoinResult.joinFuture.isDone());
|
assertTrue(secondMemberJoinResult.joinFuture.isDone());
|
||||||
assertEquals(Errors.NONE.code(), secondMemberJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), secondMemberJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(2, group.generationId());
|
assertEquals(2, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -7820,7 +7820,7 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
assertEquals(expectedError.code(), heartbeatResponse.errorCode());
|
assertEquals(expectedError.code(), heartbeatResponse.errorCode());
|
||||||
}
|
}
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
|
|
||||||
GroupMetadataManagerTestContext.JoinResult otherMemberRejoinResult = context.sendClassicGroupJoin(
|
GroupMetadataManagerTestContext.JoinResult otherMemberRejoinResult = context.sendClassicGroupJoin(
|
||||||
|
@ -7951,8 +7951,8 @@ public class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// At this point the second member should have been removed from pending list (session timeout),
|
// At this point the second member should have been removed from pending list (session timeout),
|
||||||
// and the group should be in Stable state with only the first member in it.
|
// and the group should be in Stable state with only the first member in it.
|
||||||
assertEquals(1, group.size());
|
assertEquals(1, group.numMembers());
|
||||||
assertTrue(group.hasMemberId(firstMemberId));
|
assertTrue(group.hasMember(firstMemberId));
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(STABLE));
|
assertTrue(group.isInState(STABLE));
|
||||||
}
|
}
|
||||||
|
@ -10173,7 +10173,7 @@ public class GroupMetadataManagerTest {
|
||||||
assertFalse(joinResult1.joinFuture.isDone());
|
assertFalse(joinResult1.joinFuture.isDone());
|
||||||
assertFalse(joinResult2.joinFuture.isDone());
|
assertFalse(joinResult2.joinFuture.isDone());
|
||||||
assertTrue(preparingGroup.isInState(PREPARING_REBALANCE));
|
assertTrue(preparingGroup.isInState(PREPARING_REBALANCE));
|
||||||
assertEquals(2, preparingGroup.size());
|
assertEquals(2, preparingGroup.numMembers());
|
||||||
|
|
||||||
context.onUnloaded();
|
context.onUnloaded();
|
||||||
|
|
||||||
|
|
|
@ -50,7 +50,6 @@ import org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAss
|
||||||
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupBuilder;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupBuilder;
|
||||||
import org.apache.kafka.coordinator.group.consumer.MemberState;
|
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataKey;
|
||||||
|
@ -66,6 +65,7 @@ import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmen
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
||||||
import org.apache.kafka.image.MetadataImage;
|
import org.apache.kafka.image.MetadataImage;
|
||||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||||
|
@ -929,7 +929,7 @@ public class GroupMetadataManagerTestContext {
|
||||||
assertEquals(Errors.NONE.code(), followerJoinResult.joinFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), followerJoinResult.joinFuture.get().errorCode());
|
||||||
assertEquals(1, leaderJoinResult.joinFuture.get().generationId());
|
assertEquals(1, leaderJoinResult.joinFuture.get().generationId());
|
||||||
assertEquals(1, followerJoinResult.joinFuture.get().generationId());
|
assertEquals(1, followerJoinResult.joinFuture.get().generationId());
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
|
@ -979,7 +979,7 @@ public class GroupMetadataManagerTestContext {
|
||||||
assertEquals(Errors.NONE.code(), followerSyncResult.syncFuture.get().errorCode());
|
assertEquals(Errors.NONE.code(), followerSyncResult.syncFuture.get().errorCode());
|
||||||
assertTrue(group.isInState(STABLE));
|
assertTrue(group.isInState(STABLE));
|
||||||
|
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
|
|
||||||
return new RebalanceResult(
|
return new RebalanceResult(
|
||||||
|
@ -1094,7 +1094,7 @@ public class GroupMetadataManagerTestContext {
|
||||||
assertTrue(followerJoinResult.records.isEmpty());
|
assertTrue(followerJoinResult.records.isEmpty());
|
||||||
assertFalse(followerJoinResult.joinFuture.isDone());
|
assertFalse(followerJoinResult.joinFuture.isDone());
|
||||||
assertTrue(group.isInState(PREPARING_REBALANCE));
|
assertTrue(group.isInState(PREPARING_REBALANCE));
|
||||||
assertEquals(2, group.size());
|
assertEquals(2, group.numMembers());
|
||||||
assertEquals(1, group.numPendingJoinMembers());
|
assertEquals(1, group.numPendingJoinMembers());
|
||||||
|
|
||||||
return new PendingMemberGroupResult(
|
return new PendingMemberGroupResult(
|
||||||
|
@ -1131,7 +1131,7 @@ public class GroupMetadataManagerTestContext {
|
||||||
assertEquals(expectedRecords, timeouts.get(timeoutsSize - 1).result.records());
|
assertEquals(expectedRecords, timeouts.get(timeoutsSize - 1).result.records());
|
||||||
assertNoOrEmptyResult(timeouts.subList(0, timeoutsSize - 1));
|
assertNoOrEmptyResult(timeouts.subList(0, timeoutsSize - 1));
|
||||||
assertTrue(group.isInState(EMPTY));
|
assertTrue(group.isInState(EMPTY));
|
||||||
assertEquals(0, group.size());
|
assertEquals(0, group.numMembers());
|
||||||
}
|
}
|
||||||
|
|
||||||
public CoordinatorResult<HeartbeatResponseData, CoordinatorRecord> sendClassicGroupHeartbeat(
|
public CoordinatorResult<HeartbeatResponseData, CoordinatorRecord> sendClassicGroupHeartbeat(
|
||||||
|
@ -1250,7 +1250,7 @@ public class GroupMetadataManagerTestContext {
|
||||||
return null;
|
return null;
|
||||||
}).collect(Collectors.toList());
|
}).collect(Collectors.toList());
|
||||||
|
|
||||||
assertEquals(numMembers, group.size());
|
assertEquals(numMembers, group.numMembers());
|
||||||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||||
|
|
||||||
return joinResponses;
|
return joinResponses;
|
||||||
|
|
|
@ -962,7 +962,7 @@ public class OffsetMetadataManagerTest {
|
||||||
List<MockCoordinatorTimer.ExpiredTimeout<Void, CoordinatorRecord>> timeouts =
|
List<MockCoordinatorTimer.ExpiredTimeout<Void, CoordinatorRecord>> timeouts =
|
||||||
context.sleep(5000 / 2);
|
context.sleep(5000 / 2);
|
||||||
assertEquals(1, timeouts.size());
|
assertEquals(1, timeouts.size());
|
||||||
assertFalse(group.hasMemberId(member.memberId()));
|
assertFalse(group.hasMember(member.memberId()));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
|
@ -759,7 +759,7 @@ public class ClassicGroupTest {
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
group.addPendingMember(memberId);
|
group.addPendingMember(memberId);
|
||||||
assertFalse(group.hasMemberId(memberId));
|
assertFalse(group.hasMember(memberId));
|
||||||
assertTrue(group.isPendingMember(memberId));
|
assertTrue(group.isPendingMember(memberId));
|
||||||
|
|
||||||
ClassicGroupMember member = new ClassicGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
|
@ -774,18 +774,18 @@ public class ClassicGroupTest {
|
||||||
);
|
);
|
||||||
|
|
||||||
group.add(member);
|
group.add(member);
|
||||||
assertTrue(group.hasMemberId(memberId));
|
assertTrue(group.hasMember(memberId));
|
||||||
assertFalse(group.isPendingMember(memberId));
|
assertFalse(group.isPendingMember(memberId));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testRemovalFromPendingWhenMemberIsRemoved() {
|
public void testRemovalFromPendingWhenMemberIsRemoved() {
|
||||||
group.addPendingMember(memberId);
|
group.addPendingMember(memberId);
|
||||||
assertFalse(group.hasMemberId(memberId));
|
assertFalse(group.hasMember(memberId));
|
||||||
assertTrue(group.isPendingMember(memberId));
|
assertTrue(group.isPendingMember(memberId));
|
||||||
|
|
||||||
group.remove(memberId);
|
group.remove(memberId);
|
||||||
assertFalse(group.hasMemberId(memberId));
|
assertFalse(group.hasMember(memberId));
|
||||||
assertFalse(group.isPendingMember(memberId));
|
assertFalse(group.isPendingMember(memberId));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -808,7 +808,7 @@ public class ClassicGroupTest {
|
||||||
);
|
);
|
||||||
|
|
||||||
group.add(member);
|
group.add(member);
|
||||||
assertTrue(group.hasMemberId(memberId));
|
assertTrue(group.hasMember(memberId));
|
||||||
assertTrue(group.hasStaticMember(groupInstanceId));
|
assertTrue(group.hasStaticMember(groupInstanceId));
|
||||||
|
|
||||||
// We are not permitted to add the member again if it is already present
|
// We are not permitted to add the member again if it is already present
|
||||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
|
||||||
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
|
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
import org.apache.kafka.image.MetadataImage;
|
import org.apache.kafka.image.MetadataImage;
|
||||||
import org.apache.kafka.timeline.SnapshotRegistry;
|
import org.apache.kafka.timeline.SnapshotRegistry;
|
||||||
|
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.kafka.coordinator.group.consumer;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
import org.apache.kafka.common.errors.FencedMemberEpochException;
|
||||||
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||||
|
import org.apache.kafka.coordinator.group.modern.MemberState;
|
||||||
|
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
|
|
@ -225,7 +225,8 @@ public class TargetAssignmentBuilderTest {
|
||||||
.thenReturn(new GroupAssignment(memberAssignments));
|
.thenReturn(new GroupAssignment(memberAssignments));
|
||||||
|
|
||||||
// Create and populate the assignment builder.
|
// Create and populate the assignment builder.
|
||||||
TargetAssignmentBuilder builder = new TargetAssignmentBuilder(groupId, groupEpoch, assignor)
|
TargetAssignmentBuilder<ConsumerGroupMember> builder =
|
||||||
|
new TargetAssignmentBuilder<ConsumerGroupMember>(groupId, groupEpoch, assignor)
|
||||||
.withMembers(members)
|
.withMembers(members)
|
||||||
.withStaticMembers(staticMembers)
|
.withStaticMembers(staticMembers)
|
||||||
.withSubscriptionMetadata(subscriptionMetadata)
|
.withSubscriptionMetadata(subscriptionMetadata)
|
||||||
|
|
|
@ -85,7 +85,7 @@ public class TargetAssignmentBuilderBenchmark {
|
||||||
|
|
||||||
private Map<String, TopicMetadata> subscriptionMetadata = Collections.emptyMap();
|
private Map<String, TopicMetadata> subscriptionMetadata = Collections.emptyMap();
|
||||||
|
|
||||||
private TargetAssignmentBuilder targetAssignmentBuilder;
|
private TargetAssignmentBuilder<ConsumerGroupMember> targetAssignmentBuilder;
|
||||||
|
|
||||||
private GroupSpecImpl groupSpec;
|
private GroupSpecImpl groupSpec;
|
||||||
|
|
||||||
|
@ -109,7 +109,7 @@ public class TargetAssignmentBuilderBenchmark {
|
||||||
.setSubscribedTopicNames(allTopicNames)
|
.setSubscribedTopicNames(allTopicNames)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
targetAssignmentBuilder = new TargetAssignmentBuilder(GROUP_ID, GROUP_EPOCH, partitionAssignor)
|
targetAssignmentBuilder = new TargetAssignmentBuilder<ConsumerGroupMember>(GROUP_ID, GROUP_EPOCH, partitionAssignor)
|
||||||
.withMembers(members)
|
.withMembers(members)
|
||||||
.withSubscriptionMetadata(subscriptionMetadata)
|
.withSubscriptionMetadata(subscriptionMetadata)
|
||||||
.withSubscriptionType(HOMOGENEOUS)
|
.withSubscriptionType(HOMOGENEOUS)
|
||||||
|
|
Loading…
Reference in New Issue