mirror of https://github.com/apache/kafka.git
KAFKA-16249; Improve reconciliation state machine (#15364)
This patch re-work the reconciliation state machine on the server side with the goal to fix a few issues that we have recently discovered. * When a member acknowledges the revocation of partitions (by not reporting them in the heartbeat), the current implementation may miss it. The issue is that the current implementation re-compute the assignment of a member whenever there is a new target assignment installed. When it happens, it does not consider the reported owned partitions at all. As the member is supposed to only report its own partitions when they change, the member is stuck. * Similarly, as the current assignment is re-computed whenever there is a new target assignment, the rebalance timeout, as it is currently implemented, becomes useless. The issue is that the rebalance timeout is reset whenever the member enters the revocation state. In other words, in the current implementation, the timer is reset when there are no target available even if the previous revocation is not completed yet. The patch fixes these two issues by not automatically recomputing the assignment of a member when a new target assignment is available. When the member must revoke partitions, the coordinator waits. Otherwise, it recomputes the next assignment. In other words, revoking is really blocking now. The patch also proposes to include an explicit state in the record. It makes the implementation cleaner and it also makes it more extensible in the future. The patch also changes the record format. This is a non-backward compatible change. I think that we should do this change to cleanup the record. As KIP-848 is only in early access in 3.7 and that we clearly state that we don't plane to support upgrade from it, this is acceptable in my opinion. Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
This commit is contained in:
parent
612a1fe1bb
commit
e164d4d426
|
@ -58,6 +58,7 @@ 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;
|
||||||
|
@ -98,6 +99,7 @@ import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.function.BiFunction;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.function.Predicate;
|
import java.util.function.Predicate;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -119,6 +121,7 @@ import static org.apache.kafka.coordinator.group.RecordHelpers.newGroupSubscript
|
||||||
import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionRecord;
|
import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionRecord;
|
||||||
import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionTombstoneRecord;
|
import static org.apache.kafka.coordinator.group.RecordHelpers.newMemberSubscriptionTombstoneRecord;
|
||||||
import static org.apache.kafka.coordinator.group.RecordHelpers.newTargetAssignmentTombstoneRecord;
|
import static org.apache.kafka.coordinator.group.RecordHelpers.newTargetAssignmentTombstoneRecord;
|
||||||
|
import static org.apache.kafka.coordinator.group.Utils.assignmentToString;
|
||||||
import static org.apache.kafka.coordinator.group.Utils.ofSentinel;
|
import static org.apache.kafka.coordinator.group.Utils.ofSentinel;
|
||||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupMember.EMPTY_ASSIGNMENT;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupMember.EMPTY_ASSIGNMENT;
|
||||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
||||||
|
@ -126,6 +129,7 @@ import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD;
|
||||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.EMPTY;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.EMPTY;
|
||||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPARING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPARING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.STABLE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.STABLE;
|
||||||
|
import static org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember.hasAssignedPartitionsChanged;
|
||||||
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME;
|
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME;
|
||||||
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CONSUMER_GROUP_REBALANCES_SENSOR_NAME;
|
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CONSUMER_GROUP_REBALANCES_SENSOR_NAME;
|
||||||
|
|
||||||
|
@ -1058,7 +1062,6 @@ public class GroupMetadataManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
int groupEpoch = group.groupEpoch();
|
int groupEpoch = group.groupEpoch();
|
||||||
Map<String, TopicMetadata> subscriptionMetadata = group.subscriptionMetadata();
|
Map<String, TopicMetadata> subscriptionMetadata = group.subscriptionMetadata();
|
||||||
|
|
||||||
|
@ -1166,38 +1169,17 @@ public class GroupMetadataManager {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// 3. Reconcile the member's assignment with the target assignment. This is only required if
|
// 3. Reconcile the member's assignment with the target assignment if the member is not
|
||||||
// the member is not stable or if a new target assignment has been installed.
|
// fully reconciled yet.
|
||||||
boolean assignmentUpdated = false;
|
updatedMember = maybeReconcile(
|
||||||
if (updatedMember.state() != ConsumerGroupMember.MemberState.STABLE || updatedMember.targetMemberEpoch() != targetAssignmentEpoch) {
|
groupId,
|
||||||
ConsumerGroupMember prevMember = updatedMember;
|
updatedMember,
|
||||||
updatedMember = new CurrentAssignmentBuilder(updatedMember)
|
group::currentPartitionEpoch,
|
||||||
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
targetAssignmentEpoch,
|
||||||
.withCurrentPartitionEpoch(group::currentPartitionEpoch)
|
targetAssignment,
|
||||||
.withOwnedTopicPartitions(ownedTopicPartitions)
|
ownedTopicPartitions,
|
||||||
.build();
|
records
|
||||||
|
);
|
||||||
// Checking the reference is enough here because a new instance
|
|
||||||
// is created only when the state has changed.
|
|
||||||
if (updatedMember != prevMember) {
|
|
||||||
assignmentUpdated = true;
|
|
||||||
records.add(newCurrentAssignmentRecord(groupId, updatedMember));
|
|
||||||
|
|
||||||
log.info("[GroupId {}] Member {} transitioned from {} to {}.",
|
|
||||||
groupId, memberId, member.currentAssignmentSummary(), updatedMember.currentAssignmentSummary());
|
|
||||||
|
|
||||||
if (updatedMember.state() == ConsumerGroupMember.MemberState.REVOKING) {
|
|
||||||
scheduleConsumerGroupRevocationTimeout(
|
|
||||||
groupId,
|
|
||||||
memberId,
|
|
||||||
updatedMember.rebalanceTimeoutMs(),
|
|
||||||
updatedMember.memberEpoch()
|
|
||||||
);
|
|
||||||
} else {
|
|
||||||
cancelConsumerGroupRevocationTimeout(groupId, memberId);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
scheduleConsumerGroupSessionTimeout(groupId, memberId);
|
scheduleConsumerGroupSessionTimeout(groupId, memberId);
|
||||||
|
|
||||||
|
@ -1211,13 +1193,71 @@ public class GroupMetadataManager {
|
||||||
// 1. The member reported its owned partitions;
|
// 1. The member reported its owned partitions;
|
||||||
// 2. The member just joined or rejoined to group (epoch equals to zero);
|
// 2. The member just joined or rejoined to group (epoch equals to zero);
|
||||||
// 3. The member's assignment has been updated.
|
// 3. The member's assignment has been updated.
|
||||||
if (ownedTopicPartitions != null || memberEpoch == 0 || assignmentUpdated) {
|
if (ownedTopicPartitions != null || memberEpoch == 0 || hasAssignedPartitionsChanged(member, updatedMember)) {
|
||||||
response.setAssignment(createResponseAssignment(updatedMember));
|
response.setAssignment(createResponseAssignment(updatedMember));
|
||||||
}
|
}
|
||||||
|
|
||||||
return new CoordinatorResult<>(records, response);
|
return new CoordinatorResult<>(records, response);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reconciles the current assignment of the member towards the target assignment if needed.
|
||||||
|
*
|
||||||
|
* @param groupId The group id.
|
||||||
|
* @param member The member to reconcile.
|
||||||
|
* @param currentPartitionEpoch The function returning the current epoch of
|
||||||
|
* a given partition.
|
||||||
|
* @param targetAssignmentEpoch The target assignment epoch.
|
||||||
|
* @param targetAssignment The target assignment.
|
||||||
|
* @param ownedTopicPartitions The list of partitions owned by the member. This
|
||||||
|
* is reported in the ConsumerGroupHeartbeat API and
|
||||||
|
* it could be null if not provided.
|
||||||
|
* @param records The list to accumulate any new records.
|
||||||
|
* @return The received member if no changes have been made; or a new
|
||||||
|
* member containing the new assignment.
|
||||||
|
*/
|
||||||
|
private ConsumerGroupMember maybeReconcile(
|
||||||
|
String groupId,
|
||||||
|
ConsumerGroupMember member,
|
||||||
|
BiFunction<Uuid, Integer, Integer> currentPartitionEpoch,
|
||||||
|
int targetAssignmentEpoch,
|
||||||
|
Assignment targetAssignment,
|
||||||
|
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions,
|
||||||
|
List<Record> records
|
||||||
|
) {
|
||||||
|
if (member.isReconciledTo(targetAssignmentEpoch)) {
|
||||||
|
return member;
|
||||||
|
}
|
||||||
|
|
||||||
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withTargetAssignment(targetAssignmentEpoch, targetAssignment)
|
||||||
|
.withCurrentPartitionEpoch(currentPartitionEpoch)
|
||||||
|
.withOwnedTopicPartitions(ownedTopicPartitions)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
if (!updatedMember.equals(member)) {
|
||||||
|
records.add(newCurrentAssignmentRecord(groupId, updatedMember));
|
||||||
|
|
||||||
|
log.info("[GroupId {}] Member {} new assignment state: epoch={}, previousEpoch={}, state={}, "
|
||||||
|
+ "assignedPartitions={} and revokedPartitions={}.",
|
||||||
|
groupId, updatedMember.memberId(), updatedMember.memberEpoch(), updatedMember.previousMemberEpoch(), updatedMember.state(),
|
||||||
|
assignmentToString(updatedMember.assignedPartitions()), assignmentToString(updatedMember.partitionsPendingRevocation()));
|
||||||
|
|
||||||
|
if (updatedMember.state() == MemberState.UNREVOKED_PARTITIONS) {
|
||||||
|
scheduleConsumerGroupRebalanceTimeout(
|
||||||
|
groupId,
|
||||||
|
updatedMember.memberId(),
|
||||||
|
updatedMember.memberEpoch(),
|
||||||
|
updatedMember.rebalanceTimeoutMs()
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
cancelConsumerGroupRebalanceTimeout(groupId, updatedMember.memberId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return updatedMember;
|
||||||
|
}
|
||||||
|
|
||||||
private void removeMemberAndCancelTimers(
|
private void removeMemberAndCancelTimers(
|
||||||
List<Record> records,
|
List<Record> records,
|
||||||
String groupId,
|
String groupId,
|
||||||
|
@ -1353,7 +1393,7 @@ public class GroupMetadataManager {
|
||||||
*/
|
*/
|
||||||
private void cancelTimers(String groupId, String memberId) {
|
private void cancelTimers(String groupId, String memberId) {
|
||||||
cancelConsumerGroupSessionTimeout(groupId, memberId);
|
cancelConsumerGroupSessionTimeout(groupId, memberId);
|
||||||
cancelConsumerGroupRevocationTimeout(groupId, memberId);
|
cancelConsumerGroupRebalanceTimeout(groupId, memberId);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1400,35 +1440,35 @@ public class GroupMetadataManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Schedules a revocation timeout for the member.
|
* Schedules a rebalance timeout for the member.
|
||||||
*
|
*
|
||||||
* @param groupId The group id.
|
* @param groupId The group id.
|
||||||
* @param memberId The member id.
|
* @param memberId The member id.
|
||||||
* @param revocationTimeoutMs The revocation timeout.
|
* @param memberEpoch The member epoch.
|
||||||
* @param expectedMemberEpoch The expected member epoch.
|
* @param rebalanceTimeoutMs The rebalance timeout.
|
||||||
*/
|
*/
|
||||||
private void scheduleConsumerGroupRevocationTimeout(
|
private void scheduleConsumerGroupRebalanceTimeout(
|
||||||
String groupId,
|
String groupId,
|
||||||
String memberId,
|
String memberId,
|
||||||
long revocationTimeoutMs,
|
int memberEpoch,
|
||||||
int expectedMemberEpoch
|
int rebalanceTimeoutMs
|
||||||
) {
|
) {
|
||||||
String key = consumerGroupRevocationTimeoutKey(groupId, memberId);
|
String key = consumerGroupRebalanceTimeoutKey(groupId, memberId);
|
||||||
timer.schedule(key, revocationTimeoutMs, TimeUnit.MILLISECONDS, true, () -> {
|
timer.schedule(key, rebalanceTimeoutMs, TimeUnit.MILLISECONDS, true, () -> {
|
||||||
try {
|
try {
|
||||||
ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false);
|
ConsumerGroup group = getOrMaybeCreateConsumerGroup(groupId, false);
|
||||||
ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false);
|
ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false);
|
||||||
|
|
||||||
if (member.state() != ConsumerGroupMember.MemberState.REVOKING ||
|
if (member.memberEpoch() == memberEpoch) {
|
||||||
member.memberEpoch() != expectedMemberEpoch) {
|
log.info("[GroupId {}] Member {} fenced from the group because " +
|
||||||
log.debug("[GroupId {}] Ignoring revocation timeout for {} because the member " +
|
"it failed to transition from epoch {} within {}ms.",
|
||||||
"state does not match the expected state.", groupId, memberId);
|
groupId, memberId, memberEpoch, rebalanceTimeoutMs);
|
||||||
|
return new CoordinatorResult<>(consumerGroupFenceMember(group, member));
|
||||||
|
} else {
|
||||||
|
log.debug("[GroupId {}] Ignoring rebalance timeout for {} because the member " +
|
||||||
|
"left the epoch {}.", groupId, memberId, memberEpoch);
|
||||||
return new CoordinatorResult<>(Collections.emptyList());
|
return new CoordinatorResult<>(Collections.emptyList());
|
||||||
}
|
}
|
||||||
|
|
||||||
log.info("[GroupId {}] Member {} fenced from the group because " +
|
|
||||||
"it failed to revoke partitions within {}ms.", groupId, memberId, revocationTimeoutMs);
|
|
||||||
return new CoordinatorResult<>(consumerGroupFenceMember(group, member));
|
|
||||||
} catch (GroupIdNotFoundException ex) {
|
} catch (GroupIdNotFoundException ex) {
|
||||||
log.debug("[GroupId {}] Could not fence {}} because the group does not exist.",
|
log.debug("[GroupId {}] Could not fence {}} because the group does not exist.",
|
||||||
groupId, memberId);
|
groupId, memberId);
|
||||||
|
@ -1442,16 +1482,16 @@ public class GroupMetadataManager {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Cancels the revocation timeout of the member.
|
* Cancels the rebalance timeout of the member.
|
||||||
*
|
*
|
||||||
* @param groupId The group id.
|
* @param groupId The group id.
|
||||||
* @param memberId The member id.
|
* @param memberId The member id.
|
||||||
*/
|
*/
|
||||||
private void cancelConsumerGroupRevocationTimeout(
|
private void cancelConsumerGroupRebalanceTimeout(
|
||||||
String groupId,
|
String groupId,
|
||||||
String memberId
|
String memberId
|
||||||
) {
|
) {
|
||||||
timer.cancel(consumerGroupRevocationTimeoutKey(groupId, memberId));
|
timer.cancel(consumerGroupRebalanceTimeoutKey(groupId, memberId));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1744,10 +1784,8 @@ public class GroupMetadataManager {
|
||||||
ConsumerGroupMember newMember = new ConsumerGroupMember.Builder(oldMember)
|
ConsumerGroupMember newMember = new ConsumerGroupMember.Builder(oldMember)
|
||||||
.setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH)
|
.setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH)
|
||||||
.setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH)
|
.setPreviousMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH)
|
||||||
.setTargetMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH)
|
|
||||||
.setAssignedPartitions(Collections.emptyMap())
|
.setAssignedPartitions(Collections.emptyMap())
|
||||||
.setPartitionsPendingRevocation(Collections.emptyMap())
|
.setPartitionsPendingRevocation(Collections.emptyMap())
|
||||||
.setPartitionsPendingAssignment(Collections.emptyMap())
|
|
||||||
.build();
|
.build();
|
||||||
consumerGroup.updateMember(newMember);
|
consumerGroup.updateMember(newMember);
|
||||||
}
|
}
|
||||||
|
@ -1796,12 +1834,12 @@ public class GroupMetadataManager {
|
||||||
consumerGroup.members().forEach((memberId, member) -> {
|
consumerGroup.members().forEach((memberId, member) -> {
|
||||||
log.debug("Loaded member {} in consumer group {}.", memberId, groupId);
|
log.debug("Loaded member {} in consumer group {}.", memberId, groupId);
|
||||||
scheduleConsumerGroupSessionTimeout(groupId, memberId);
|
scheduleConsumerGroupSessionTimeout(groupId, memberId);
|
||||||
if (member.state() == ConsumerGroupMember.MemberState.REVOKING) {
|
if (member.state() == MemberState.UNREVOKED_PARTITIONS) {
|
||||||
scheduleConsumerGroupRevocationTimeout(
|
scheduleConsumerGroupRebalanceTimeout(
|
||||||
groupId,
|
groupId,
|
||||||
memberId,
|
member.memberId(),
|
||||||
member.rebalanceTimeoutMs(),
|
member.memberEpoch(),
|
||||||
member.memberEpoch()
|
member.rebalanceTimeoutMs()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -1834,8 +1872,8 @@ public class GroupMetadataManager {
|
||||||
return "session-timeout-" + groupId + "-" + memberId;
|
return "session-timeout-" + groupId + "-" + memberId;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
|
public static String consumerGroupRebalanceTimeoutKey(String groupId, String memberId) {
|
||||||
return "revocation-timeout-" + groupId + "-" + memberId;
|
return "rebalance-timeout-" + groupId + "-" + memberId;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -346,10 +346,9 @@ public class RecordHelpers {
|
||||||
new ConsumerGroupCurrentMemberAssignmentValue()
|
new ConsumerGroupCurrentMemberAssignmentValue()
|
||||||
.setMemberEpoch(member.memberEpoch())
|
.setMemberEpoch(member.memberEpoch())
|
||||||
.setPreviousMemberEpoch(member.previousMemberEpoch())
|
.setPreviousMemberEpoch(member.previousMemberEpoch())
|
||||||
.setTargetMemberEpoch(member.targetMemberEpoch())
|
.setState(member.state().value())
|
||||||
.setAssignedPartitions(toTopicPartitions(member.assignedPartitions()))
|
.setAssignedPartitions(toTopicPartitions(member.assignedPartitions()))
|
||||||
.setPartitionsPendingRevocation(toTopicPartitions(member.partitionsPendingRevocation()))
|
.setPartitionsPendingRevocation(toTopicPartitions(member.partitionsPendingRevocation())),
|
||||||
.setPartitionsPendingAssignment(toTopicPartitions(member.partitionsPendingAssignment())),
|
|
||||||
(short) 0
|
(short) 0
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
|
@ -16,8 +16,13 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.coordinator.group;
|
package org.apache.kafka.coordinator.group;
|
||||||
|
|
||||||
|
import org.apache.kafka.common.Uuid;
|
||||||
|
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.OptionalInt;
|
import java.util.OptionalInt;
|
||||||
import java.util.OptionalLong;
|
import java.util.OptionalLong;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
public class Utils {
|
public class Utils {
|
||||||
private Utils() {}
|
private Utils() {}
|
||||||
|
@ -37,4 +42,31 @@ public class Utils {
|
||||||
public static OptionalLong ofSentinel(long value) {
|
public static OptionalLong ofSentinel(long value) {
|
||||||
return value != -1 ? OptionalLong.of(value) : OptionalLong.empty();
|
return value != -1 ? OptionalLong.of(value) : OptionalLong.empty();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return The provided assignment as a String.
|
||||||
|
*
|
||||||
|
* Example:
|
||||||
|
* [topicid1-0, topicid1-1, topicid2-0, topicid2-1]
|
||||||
|
*/
|
||||||
|
public static String assignmentToString(
|
||||||
|
Map<Uuid, Set<Integer>> assignment
|
||||||
|
) {
|
||||||
|
StringBuilder builder = new StringBuilder("[");
|
||||||
|
Iterator<Map.Entry<Uuid, Set<Integer>>> topicsIterator = assignment.entrySet().iterator();
|
||||||
|
while (topicsIterator.hasNext()) {
|
||||||
|
Map.Entry<Uuid, Set<Integer>> entry = topicsIterator.next();
|
||||||
|
Iterator<Integer> partitionsIterator = entry.getValue().iterator();
|
||||||
|
while (partitionsIterator.hasNext()) {
|
||||||
|
builder.append(entry.getKey());
|
||||||
|
builder.append("-");
|
||||||
|
builder.append(partitionsIterator.next());
|
||||||
|
if (partitionsIterator.hasNext() || topicsIterator.hasNext()) {
|
||||||
|
builder.append(", ");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
builder.append("]");
|
||||||
|
return builder.toString();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -779,7 +779,7 @@ public class ConsumerGroup implements Group {
|
||||||
newState = ASSIGNING;
|
newState = ASSIGNING;
|
||||||
} else {
|
} else {
|
||||||
for (ConsumerGroupMember member : members.values()) {
|
for (ConsumerGroupMember member : members.values()) {
|
||||||
if (member.targetMemberEpoch() != targetAssignmentEpoch.get() || member.state() != ConsumerGroupMember.MemberState.STABLE) {
|
if (!member.isReconciledTo(targetAssignmentEpoch.get())) {
|
||||||
newState = RECONCILING;
|
newState = RECONCILING;
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
|
|
|
@ -41,6 +41,7 @@ import java.util.stream.Collectors;
|
||||||
* by records stored in the __consumer_offsets topic.
|
* by records stored in the __consumer_offsets topic.
|
||||||
*/
|
*/
|
||||||
public class ConsumerGroupMember {
|
public class ConsumerGroupMember {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* 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
|
||||||
* an existing one.
|
* an existing one.
|
||||||
|
@ -52,7 +53,7 @@ public class ConsumerGroupMember {
|
||||||
private final String memberId;
|
private final String memberId;
|
||||||
private int memberEpoch = 0;
|
private int memberEpoch = 0;
|
||||||
private int previousMemberEpoch = -1;
|
private int previousMemberEpoch = -1;
|
||||||
private int targetMemberEpoch = 0;
|
private MemberState state = MemberState.STABLE;
|
||||||
private String instanceId = null;
|
private String instanceId = null;
|
||||||
private String rackId = null;
|
private String rackId = null;
|
||||||
private int rebalanceTimeoutMs = -1;
|
private int rebalanceTimeoutMs = -1;
|
||||||
|
@ -64,7 +65,6 @@ public class ConsumerGroupMember {
|
||||||
private List<ClientAssignor> clientAssignors = Collections.emptyList();
|
private List<ClientAssignor> clientAssignors = Collections.emptyList();
|
||||||
private Map<Uuid, Set<Integer>> assignedPartitions = Collections.emptyMap();
|
private Map<Uuid, Set<Integer>> assignedPartitions = Collections.emptyMap();
|
||||||
private Map<Uuid, Set<Integer>> partitionsPendingRevocation = Collections.emptyMap();
|
private Map<Uuid, Set<Integer>> partitionsPendingRevocation = Collections.emptyMap();
|
||||||
private Map<Uuid, Set<Integer>> partitionsPendingAssignment = Collections.emptyMap();
|
|
||||||
|
|
||||||
public Builder(String memberId) {
|
public Builder(String memberId) {
|
||||||
this.memberId = Objects.requireNonNull(memberId);
|
this.memberId = Objects.requireNonNull(memberId);
|
||||||
|
@ -76,7 +76,6 @@ public class ConsumerGroupMember {
|
||||||
this.memberId = member.memberId;
|
this.memberId = member.memberId;
|
||||||
this.memberEpoch = member.memberEpoch;
|
this.memberEpoch = member.memberEpoch;
|
||||||
this.previousMemberEpoch = member.previousMemberEpoch;
|
this.previousMemberEpoch = member.previousMemberEpoch;
|
||||||
this.targetMemberEpoch = member.targetMemberEpoch;
|
|
||||||
this.instanceId = member.instanceId;
|
this.instanceId = member.instanceId;
|
||||||
this.rackId = member.rackId;
|
this.rackId = member.rackId;
|
||||||
this.rebalanceTimeoutMs = member.rebalanceTimeoutMs;
|
this.rebalanceTimeoutMs = member.rebalanceTimeoutMs;
|
||||||
|
@ -86,9 +85,16 @@ public class ConsumerGroupMember {
|
||||||
this.subscribedTopicRegex = member.subscribedTopicRegex;
|
this.subscribedTopicRegex = member.subscribedTopicRegex;
|
||||||
this.serverAssignorName = member.serverAssignorName;
|
this.serverAssignorName = member.serverAssignorName;
|
||||||
this.clientAssignors = member.clientAssignors;
|
this.clientAssignors = member.clientAssignors;
|
||||||
|
this.state = member.state;
|
||||||
this.assignedPartitions = member.assignedPartitions;
|
this.assignedPartitions = member.assignedPartitions;
|
||||||
this.partitionsPendingRevocation = member.partitionsPendingRevocation;
|
this.partitionsPendingRevocation = member.partitionsPendingRevocation;
|
||||||
this.partitionsPendingAssignment = member.partitionsPendingAssignment;
|
}
|
||||||
|
|
||||||
|
public Builder updateMemberEpoch(int memberEpoch) {
|
||||||
|
int currentMemberEpoch = this.memberEpoch;
|
||||||
|
this.memberEpoch = memberEpoch;
|
||||||
|
this.previousMemberEpoch = currentMemberEpoch;
|
||||||
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setMemberEpoch(int memberEpoch) {
|
public Builder setMemberEpoch(int memberEpoch) {
|
||||||
|
@ -101,11 +107,6 @@ public class ConsumerGroupMember {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setTargetMemberEpoch(int targetMemberEpoch) {
|
|
||||||
this.targetMemberEpoch = targetMemberEpoch;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder setInstanceId(String instanceId) {
|
public Builder setInstanceId(String instanceId) {
|
||||||
this.instanceId = instanceId;
|
this.instanceId = instanceId;
|
||||||
return this;
|
return this;
|
||||||
|
@ -188,6 +189,11 @@ public class ConsumerGroupMember {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public Builder setState(MemberState state) {
|
||||||
|
this.state = state;
|
||||||
|
return this;
|
||||||
|
}
|
||||||
|
|
||||||
public Builder setAssignedPartitions(Map<Uuid, Set<Integer>> assignedPartitions) {
|
public Builder setAssignedPartitions(Map<Uuid, Set<Integer>> assignedPartitions) {
|
||||||
this.assignedPartitions = assignedPartitions;
|
this.assignedPartitions = assignedPartitions;
|
||||||
return this;
|
return this;
|
||||||
|
@ -198,11 +204,6 @@ public class ConsumerGroupMember {
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder setPartitionsPendingAssignment(Map<Uuid, Set<Integer>> partitionsPendingAssignment) {
|
|
||||||
this.partitionsPendingAssignment = partitionsPendingAssignment;
|
|
||||||
return this;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Builder updateWith(ConsumerGroupMemberMetadataValue record) {
|
public Builder updateWith(ConsumerGroupMemberMetadataValue record) {
|
||||||
setInstanceId(record.instanceId());
|
setInstanceId(record.instanceId());
|
||||||
setRackId(record.rackId());
|
setRackId(record.rackId());
|
||||||
|
@ -221,10 +222,9 @@ public class ConsumerGroupMember {
|
||||||
public Builder updateWith(ConsumerGroupCurrentMemberAssignmentValue record) {
|
public Builder updateWith(ConsumerGroupCurrentMemberAssignmentValue record) {
|
||||||
setMemberEpoch(record.memberEpoch());
|
setMemberEpoch(record.memberEpoch());
|
||||||
setPreviousMemberEpoch(record.previousMemberEpoch());
|
setPreviousMemberEpoch(record.previousMemberEpoch());
|
||||||
setTargetMemberEpoch(record.targetMemberEpoch());
|
setState(MemberState.fromValue(record.state()));
|
||||||
setAssignedPartitions(assignmentFromTopicPartitions(record.assignedPartitions()));
|
setAssignedPartitions(assignmentFromTopicPartitions(record.assignedPartitions()));
|
||||||
setPartitionsPendingRevocation(assignmentFromTopicPartitions(record.partitionsPendingRevocation()));
|
setPartitionsPendingRevocation(assignmentFromTopicPartitions(record.partitionsPendingRevocation()));
|
||||||
setPartitionsPendingAssignment(assignmentFromTopicPartitions(record.partitionsPendingAssignment()));
|
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -237,20 +237,10 @@ public class ConsumerGroupMember {
|
||||||
}
|
}
|
||||||
|
|
||||||
public ConsumerGroupMember build() {
|
public ConsumerGroupMember build() {
|
||||||
MemberState state;
|
|
||||||
if (!partitionsPendingRevocation.isEmpty()) {
|
|
||||||
state = MemberState.REVOKING;
|
|
||||||
} else if (!partitionsPendingAssignment.isEmpty()) {
|
|
||||||
state = MemberState.ASSIGNING;
|
|
||||||
} else {
|
|
||||||
state = MemberState.STABLE;
|
|
||||||
}
|
|
||||||
|
|
||||||
return new ConsumerGroupMember(
|
return new ConsumerGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
memberEpoch,
|
memberEpoch,
|
||||||
previousMemberEpoch,
|
previousMemberEpoch,
|
||||||
targetMemberEpoch,
|
|
||||||
instanceId,
|
instanceId,
|
||||||
rackId,
|
rackId,
|
||||||
rebalanceTimeoutMs,
|
rebalanceTimeoutMs,
|
||||||
|
@ -262,33 +252,11 @@ public class ConsumerGroupMember {
|
||||||
clientAssignors,
|
clientAssignors,
|
||||||
state,
|
state,
|
||||||
assignedPartitions,
|
assignedPartitions,
|
||||||
partitionsPendingRevocation,
|
partitionsPendingRevocation
|
||||||
partitionsPendingAssignment
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* The various states that a member can be in. For their definition,
|
|
||||||
* refer to the documentation of {{@link CurrentAssignmentBuilder}}.
|
|
||||||
*/
|
|
||||||
public enum MemberState {
|
|
||||||
REVOKING("revoking"),
|
|
||||||
ASSIGNING("assigning"),
|
|
||||||
STABLE("stable");
|
|
||||||
|
|
||||||
private final String name;
|
|
||||||
|
|
||||||
MemberState(String name) {
|
|
||||||
this.name = name;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString() {
|
|
||||||
return name;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The member id.
|
* The member id.
|
||||||
*/
|
*/
|
||||||
|
@ -305,11 +273,9 @@ public class ConsumerGroupMember {
|
||||||
private final int previousMemberEpoch;
|
private final int previousMemberEpoch;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The next member epoch. This corresponds to the target
|
* The member state.
|
||||||
* assignment epoch used to compute the current assigned,
|
|
||||||
* revoking and assigning partitions.
|
|
||||||
*/
|
*/
|
||||||
private final int targetMemberEpoch;
|
private final MemberState state;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The instance id provided by the member.
|
* The instance id provided by the member.
|
||||||
|
@ -356,11 +322,6 @@ public class ConsumerGroupMember {
|
||||||
*/
|
*/
|
||||||
private final List<ClientAssignor> clientAssignors;
|
private final List<ClientAssignor> clientAssignors;
|
||||||
|
|
||||||
/**
|
|
||||||
* The member state.
|
|
||||||
*/
|
|
||||||
private final MemberState state;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The partitions assigned to this member.
|
* The partitions assigned to this member.
|
||||||
*/
|
*/
|
||||||
|
@ -371,18 +332,10 @@ public class ConsumerGroupMember {
|
||||||
*/
|
*/
|
||||||
private final Map<Uuid, Set<Integer>> partitionsPendingRevocation;
|
private final Map<Uuid, Set<Integer>> partitionsPendingRevocation;
|
||||||
|
|
||||||
/**
|
|
||||||
* The partitions waiting to be assigned to this
|
|
||||||
* member. They will be assigned when they are
|
|
||||||
* released by their previous owners.
|
|
||||||
*/
|
|
||||||
private final Map<Uuid, Set<Integer>> partitionsPendingAssignment;
|
|
||||||
|
|
||||||
private ConsumerGroupMember(
|
private ConsumerGroupMember(
|
||||||
String memberId,
|
String memberId,
|
||||||
int memberEpoch,
|
int memberEpoch,
|
||||||
int previousMemberEpoch,
|
int previousMemberEpoch,
|
||||||
int targetMemberEpoch,
|
|
||||||
String instanceId,
|
String instanceId,
|
||||||
String rackId,
|
String rackId,
|
||||||
int rebalanceTimeoutMs,
|
int rebalanceTimeoutMs,
|
||||||
|
@ -394,13 +347,12 @@ public class ConsumerGroupMember {
|
||||||
List<ClientAssignor> clientAssignors,
|
List<ClientAssignor> clientAssignors,
|
||||||
MemberState state,
|
MemberState state,
|
||||||
Map<Uuid, Set<Integer>> assignedPartitions,
|
Map<Uuid, Set<Integer>> assignedPartitions,
|
||||||
Map<Uuid, Set<Integer>> partitionsPendingRevocation,
|
Map<Uuid, Set<Integer>> partitionsPendingRevocation
|
||||||
Map<Uuid, Set<Integer>> partitionsPendingAssignment
|
|
||||||
) {
|
) {
|
||||||
this.memberId = memberId;
|
this.memberId = memberId;
|
||||||
this.memberEpoch = memberEpoch;
|
this.memberEpoch = memberEpoch;
|
||||||
this.previousMemberEpoch = previousMemberEpoch;
|
this.previousMemberEpoch = previousMemberEpoch;
|
||||||
this.targetMemberEpoch = targetMemberEpoch;
|
this.state = state;
|
||||||
this.instanceId = instanceId;
|
this.instanceId = instanceId;
|
||||||
this.rackId = rackId;
|
this.rackId = rackId;
|
||||||
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
|
this.rebalanceTimeoutMs = rebalanceTimeoutMs;
|
||||||
|
@ -410,10 +362,8 @@ public class ConsumerGroupMember {
|
||||||
this.subscribedTopicRegex = subscribedTopicRegex;
|
this.subscribedTopicRegex = subscribedTopicRegex;
|
||||||
this.serverAssignorName = serverAssignorName;
|
this.serverAssignorName = serverAssignorName;
|
||||||
this.clientAssignors = clientAssignors;
|
this.clientAssignors = clientAssignors;
|
||||||
this.state = state;
|
|
||||||
this.assignedPartitions = assignedPartitions;
|
this.assignedPartitions = assignedPartitions;
|
||||||
this.partitionsPendingRevocation = partitionsPendingRevocation;
|
this.partitionsPendingRevocation = partitionsPendingRevocation;
|
||||||
this.partitionsPendingAssignment = partitionsPendingAssignment;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -437,13 +387,6 @@ public class ConsumerGroupMember {
|
||||||
return previousMemberEpoch;
|
return previousMemberEpoch;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The target member epoch.
|
|
||||||
*/
|
|
||||||
public int targetMemberEpoch() {
|
|
||||||
return targetMemberEpoch;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The instance id.
|
* @return The instance id.
|
||||||
*/
|
*/
|
||||||
|
@ -514,6 +457,13 @@ public class ConsumerGroupMember {
|
||||||
return 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.
|
* @return The set of assigned partitions.
|
||||||
*/
|
*/
|
||||||
|
@ -528,27 +478,6 @@ public class ConsumerGroupMember {
|
||||||
return partitionsPendingRevocation;
|
return partitionsPendingRevocation;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* @return The set of partitions awaiting assignment to the member.
|
|
||||||
*/
|
|
||||||
public Map<Uuid, Set<Integer>> partitionsPendingAssignment() {
|
|
||||||
return partitionsPendingAssignment;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return A string representation of the current assignment state.
|
|
||||||
*/
|
|
||||||
public String currentAssignmentSummary() {
|
|
||||||
return "CurrentAssignment(memberEpoch=" + memberEpoch +
|
|
||||||
", previousMemberEpoch=" + previousMemberEpoch +
|
|
||||||
", targetMemberEpoch=" + targetMemberEpoch +
|
|
||||||
", state=" + state +
|
|
||||||
", assignedPartitions=" + assignedPartitions +
|
|
||||||
", partitionsPendingRevocation=" + partitionsPendingRevocation +
|
|
||||||
", partitionsPendingAssignment=" + partitionsPendingAssignment +
|
|
||||||
')';
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @param targetAssignment The target assignment of this member in the corresponding group.
|
* @param targetAssignment The target assignment of this member in the corresponding group.
|
||||||
*
|
*
|
||||||
|
@ -612,7 +541,7 @@ public class ConsumerGroupMember {
|
||||||
ConsumerGroupMember that = (ConsumerGroupMember) o;
|
ConsumerGroupMember that = (ConsumerGroupMember) o;
|
||||||
return memberEpoch == that.memberEpoch
|
return memberEpoch == that.memberEpoch
|
||||||
&& previousMemberEpoch == that.previousMemberEpoch
|
&& previousMemberEpoch == that.previousMemberEpoch
|
||||||
&& targetMemberEpoch == that.targetMemberEpoch
|
&& state == that.state
|
||||||
&& rebalanceTimeoutMs == that.rebalanceTimeoutMs
|
&& rebalanceTimeoutMs == that.rebalanceTimeoutMs
|
||||||
&& Objects.equals(memberId, that.memberId)
|
&& Objects.equals(memberId, that.memberId)
|
||||||
&& Objects.equals(instanceId, that.instanceId)
|
&& Objects.equals(instanceId, that.instanceId)
|
||||||
|
@ -624,8 +553,7 @@ public class ConsumerGroupMember {
|
||||||
&& Objects.equals(serverAssignorName, that.serverAssignorName)
|
&& Objects.equals(serverAssignorName, that.serverAssignorName)
|
||||||
&& Objects.equals(clientAssignors, that.clientAssignors)
|
&& Objects.equals(clientAssignors, that.clientAssignors)
|
||||||
&& Objects.equals(assignedPartitions, that.assignedPartitions)
|
&& Objects.equals(assignedPartitions, that.assignedPartitions)
|
||||||
&& Objects.equals(partitionsPendingRevocation, that.partitionsPendingRevocation)
|
&& Objects.equals(partitionsPendingRevocation, that.partitionsPendingRevocation);
|
||||||
&& Objects.equals(partitionsPendingAssignment, that.partitionsPendingAssignment);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -633,7 +561,7 @@ public class ConsumerGroupMember {
|
||||||
int result = memberId != null ? memberId.hashCode() : 0;
|
int result = memberId != null ? memberId.hashCode() : 0;
|
||||||
result = 31 * result + memberEpoch;
|
result = 31 * result + memberEpoch;
|
||||||
result = 31 * result + previousMemberEpoch;
|
result = 31 * result + previousMemberEpoch;
|
||||||
result = 31 * result + targetMemberEpoch;
|
result = 31 * result + Objects.hashCode(state);
|
||||||
result = 31 * result + Objects.hashCode(instanceId);
|
result = 31 * result + Objects.hashCode(instanceId);
|
||||||
result = 31 * result + Objects.hashCode(rackId);
|
result = 31 * result + Objects.hashCode(rackId);
|
||||||
result = 31 * result + rebalanceTimeoutMs;
|
result = 31 * result + rebalanceTimeoutMs;
|
||||||
|
@ -645,7 +573,6 @@ public class ConsumerGroupMember {
|
||||||
result = 31 * result + Objects.hashCode(clientAssignors);
|
result = 31 * result + Objects.hashCode(clientAssignors);
|
||||||
result = 31 * result + Objects.hashCode(assignedPartitions);
|
result = 31 * result + Objects.hashCode(assignedPartitions);
|
||||||
result = 31 * result + Objects.hashCode(partitionsPendingRevocation);
|
result = 31 * result + Objects.hashCode(partitionsPendingRevocation);
|
||||||
result = 31 * result + Objects.hashCode(partitionsPendingAssignment);
|
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -655,7 +582,7 @@ public class ConsumerGroupMember {
|
||||||
"memberId='" + memberId + '\'' +
|
"memberId='" + memberId + '\'' +
|
||||||
", memberEpoch=" + memberEpoch +
|
", memberEpoch=" + memberEpoch +
|
||||||
", previousMemberEpoch=" + previousMemberEpoch +
|
", previousMemberEpoch=" + previousMemberEpoch +
|
||||||
", targetMemberEpoch=" + targetMemberEpoch +
|
", state='" + state + '\'' +
|
||||||
", instanceId='" + instanceId + '\'' +
|
", instanceId='" + instanceId + '\'' +
|
||||||
", rackId='" + rackId + '\'' +
|
", rackId='" + rackId + '\'' +
|
||||||
", rebalanceTimeoutMs=" + rebalanceTimeoutMs +
|
", rebalanceTimeoutMs=" + rebalanceTimeoutMs +
|
||||||
|
@ -665,10 +592,18 @@ public class ConsumerGroupMember {
|
||||||
", subscribedTopicRegex='" + subscribedTopicRegex + '\'' +
|
", subscribedTopicRegex='" + subscribedTopicRegex + '\'' +
|
||||||
", serverAssignorName='" + serverAssignorName + '\'' +
|
", serverAssignorName='" + serverAssignorName + '\'' +
|
||||||
", clientAssignors=" + clientAssignors +
|
", clientAssignors=" + clientAssignors +
|
||||||
", state=" + state +
|
|
||||||
", assignedPartitions=" + assignedPartitions +
|
", assignedPartitions=" + assignedPartitions +
|
||||||
", partitionsPendingRevocation=" + partitionsPendingRevocation +
|
", partitionsPendingRevocation=" + partitionsPendingRevocation +
|
||||||
", partitionsPendingAssignment=" + partitionsPendingAssignment +
|
|
||||||
')';
|
')';
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @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());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,12 +17,12 @@
|
||||||
package org.apache.kafka.coordinator.group.consumer;
|
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.message.ConsumerGroupHeartbeatRequestData;
|
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||||
|
|
||||||
import java.util.Collections;
|
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.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
@ -33,49 +33,6 @@ import java.util.function.BiFunction;
|
||||||
* The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
|
* The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the
|
||||||
* consumer group protocol. Given the current state of a member and a desired or target
|
* consumer group protocol. Given the current state of a member and a desired or target
|
||||||
* assignment state, the state machine takes the necessary steps to converge them.
|
* assignment state, the state machine takes the necessary steps to converge them.
|
||||||
*
|
|
||||||
* The member state has the following properties:
|
|
||||||
* - Current Epoch:
|
|
||||||
* The current epoch of the member.
|
|
||||||
*
|
|
||||||
* - Next Epoch:
|
|
||||||
* The desired epoch of the member. It corresponds to the epoch of the target/desired assignment.
|
|
||||||
* The member transitions to this epoch when it has revoked the partitions that it does not own
|
|
||||||
* or if it does not have to revoke any.
|
|
||||||
*
|
|
||||||
* - Previous Epoch:
|
|
||||||
* The epoch of the member when the state was last updated.
|
|
||||||
*
|
|
||||||
* - Assigned Partitions:
|
|
||||||
* The set of partitions currently assigned to the member. This represents what the member should have.
|
|
||||||
*
|
|
||||||
* - Partitions Pending Revocation:
|
|
||||||
* The set of partitions that the member should revoke before it can transition to the next state.
|
|
||||||
*
|
|
||||||
* - Partitions Pending Assignment:
|
|
||||||
* The set of partitions that the member will eventually receive. The partitions in this set are
|
|
||||||
* still owned by other members in the group.
|
|
||||||
*
|
|
||||||
* The state machine has three states:
|
|
||||||
* - REVOKING:
|
|
||||||
* This state means that the member must revoke partitions before it can transition to the next epoch
|
|
||||||
* and thus start receiving new partitions. This is to guarantee that offsets of revoked partitions
|
|
||||||
* are committed with the current epoch. The member transitions to the next state only when it has
|
|
||||||
* acknowledged the revocation.
|
|
||||||
*
|
|
||||||
* - ASSIGNING:
|
|
||||||
* This state means that the member waits on partitions which are still owned by other members in the
|
|
||||||
* group. It remains in this state until they are all freed up.
|
|
||||||
*
|
|
||||||
* - STABLE:
|
|
||||||
* This state means that the member has received all its assigned partitions.
|
|
||||||
*
|
|
||||||
* The reconciliation process is started or re-started whenever a new target assignment is installed;
|
|
||||||
* the epoch of the new target assignment is different from the next epoch of the member. In this transient
|
|
||||||
* state, the assigned partitions, the partitions pending revocation and the partitions pending assignment
|
|
||||||
* are updated. If the partitions pending revocation is not empty, the state machine transitions to
|
|
||||||
* REVOKING; if partitions pending assignment is not empty, it transitions to ASSIGNING; otherwise it
|
|
||||||
* transitions to STABLE.
|
|
||||||
*/
|
*/
|
||||||
public class CurrentAssignmentBuilder {
|
public class CurrentAssignmentBuilder {
|
||||||
/**
|
/**
|
||||||
|
@ -170,72 +127,122 @@ public class CurrentAssignmentBuilder {
|
||||||
* @return A new ConsumerGroupMember or the current one.
|
* @return A new ConsumerGroupMember or the current one.
|
||||||
*/
|
*/
|
||||||
public ConsumerGroupMember build() {
|
public ConsumerGroupMember build() {
|
||||||
// A new target assignment has been installed, we need to restart
|
|
||||||
// the reconciliation loop from the beginning.
|
|
||||||
if (targetAssignmentEpoch != member.targetMemberEpoch()) {
|
|
||||||
return transitionToNewTargetAssignmentState();
|
|
||||||
}
|
|
||||||
|
|
||||||
switch (member.state()) {
|
switch (member.state()) {
|
||||||
// Check if the partitions have been revoked by the member.
|
|
||||||
case REVOKING:
|
|
||||||
return maybeTransitionFromRevokingToAssigningOrStable();
|
|
||||||
|
|
||||||
// Check if pending partitions have been freed up.
|
|
||||||
case ASSIGNING:
|
|
||||||
return maybeTransitionFromAssigningToAssigningOrStable();
|
|
||||||
|
|
||||||
// Nothing to do.
|
|
||||||
case STABLE:
|
case STABLE:
|
||||||
return member;
|
// When the member is in the STABLE state, we verify if a newer
|
||||||
|
// epoch (or target assignment) is available. If it is, we can
|
||||||
|
// reconcile the member towards it. Otherwise, we return.
|
||||||
|
if (member.memberEpoch() != targetAssignmentEpoch) {
|
||||||
|
return computeNextAssignment(
|
||||||
|
member.memberEpoch(),
|
||||||
|
member.assignedPartitions()
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
return member;
|
||||||
|
}
|
||||||
|
|
||||||
|
case UNREVOKED_PARTITIONS:
|
||||||
|
// When the member is in the UNREVOKED_PARTITIONS state, we wait
|
||||||
|
// until the member has revoked the necessary partitions. They are
|
||||||
|
// considered revoked when they are not anymore reported in the
|
||||||
|
// owned partitions set in the ConsumerGroupHeartbeat API.
|
||||||
|
|
||||||
|
// If the member does not provide its owned partitions. We cannot
|
||||||
|
// progress.
|
||||||
|
if (ownedTopicPartitions == null) {
|
||||||
|
return member;
|
||||||
|
}
|
||||||
|
|
||||||
|
// If the member provides its owned partitions. We verify if it still
|
||||||
|
// owns any of the revoked partitions. If it does, we cannot progress.
|
||||||
|
for (ConsumerGroupHeartbeatRequestData.TopicPartitions topicPartitions : ownedTopicPartitions) {
|
||||||
|
for (Integer partitionId : topicPartitions.partitions()) {
|
||||||
|
boolean stillHasRevokedPartition = member
|
||||||
|
.partitionsPendingRevocation()
|
||||||
|
.getOrDefault(topicPartitions.topicId(), Collections.emptySet())
|
||||||
|
.contains(partitionId);
|
||||||
|
if (stillHasRevokedPartition) {
|
||||||
|
return member;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// When the member has revoked all the pending partitions, it can
|
||||||
|
// transition to the next epoch (current + 1) and we can reconcile
|
||||||
|
// its state towards the latest target assignment.
|
||||||
|
return computeNextAssignment(
|
||||||
|
member.memberEpoch() + 1,
|
||||||
|
member.assignedPartitions()
|
||||||
|
);
|
||||||
|
|
||||||
|
case UNRELEASED_PARTITIONS:
|
||||||
|
// When the member is in the UNRELEASED_PARTITIONS, we reconcile the
|
||||||
|
// member towards the latest target assignment. This will assign any
|
||||||
|
// of the unreleased partitions when they become available.
|
||||||
|
return computeNextAssignment(
|
||||||
|
member.memberEpoch(),
|
||||||
|
member.assignedPartitions()
|
||||||
|
);
|
||||||
|
|
||||||
|
case UNKNOWN:
|
||||||
|
// We could only end up in this state if a new state is added in the
|
||||||
|
// future and the group coordinator is downgraded. In this case, the
|
||||||
|
// best option is to fence the member to force it to rejoin the group
|
||||||
|
// without any partitions and to reconcile it again from scratch.
|
||||||
|
if (ownedTopicPartitions == null || !ownedTopicPartitions.isEmpty()) {
|
||||||
|
throw new FencedMemberEpochException("The consumer group member is in a unknown state. "
|
||||||
|
+ "The member must abandon all its partitions and rejoin.");
|
||||||
|
}
|
||||||
|
|
||||||
|
return computeNextAssignment(
|
||||||
|
targetAssignmentEpoch,
|
||||||
|
member.assignedPartitions()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
return member;
|
return member;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Transitions to NewTargetAssignment state. This is a transient state where
|
* Computes the next assignment.
|
||||||
* we compute the assigned partitions, the partitions pending revocation,
|
|
||||||
* the partitions pending assignment, and transition to the next state.
|
|
||||||
*
|
*
|
||||||
|
* @param memberEpoch The epoch of the member to use. This may be different
|
||||||
|
* from the epoch in {@link CurrentAssignmentBuilder#member}.
|
||||||
|
* @param memberAssignedPartitions The assigned partitions of the member to use.
|
||||||
* @return A new ConsumerGroupMember.
|
* @return A new ConsumerGroupMember.
|
||||||
*/
|
*/
|
||||||
private ConsumerGroupMember transitionToNewTargetAssignmentState() {
|
private ConsumerGroupMember computeNextAssignment(
|
||||||
|
int memberEpoch,
|
||||||
|
Map<Uuid, Set<Integer>> memberAssignedPartitions
|
||||||
|
) {
|
||||||
|
boolean hasUnreleasedPartitions = false;
|
||||||
Map<Uuid, Set<Integer>> newAssignedPartitions = new HashMap<>();
|
Map<Uuid, Set<Integer>> newAssignedPartitions = new HashMap<>();
|
||||||
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation = new HashMap<>();
|
Map<Uuid, Set<Integer>> newPartitionsPendingRevocation = new HashMap<>();
|
||||||
Map<Uuid, Set<Integer>> newPartitionsPendingAssignment = new HashMap<>();
|
Map<Uuid, Set<Integer>> newPartitionsPendingAssignment = new HashMap<>();
|
||||||
|
|
||||||
// Compute the combined set of topics.
|
|
||||||
Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
|
Set<Uuid> allTopicIds = new HashSet<>(targetAssignment.partitions().keySet());
|
||||||
allTopicIds.addAll(member.assignedPartitions().keySet());
|
allTopicIds.addAll(memberAssignedPartitions.keySet());
|
||||||
allTopicIds.addAll(member.partitionsPendingRevocation().keySet());
|
|
||||||
allTopicIds.addAll(member.partitionsPendingAssignment().keySet());
|
|
||||||
|
|
||||||
for (Uuid topicId : allTopicIds) {
|
for (Uuid topicId : allTopicIds) {
|
||||||
Set<Integer> target = targetAssignment.partitions()
|
Set<Integer> target = targetAssignment.partitions()
|
||||||
.getOrDefault(topicId, Collections.emptySet());
|
.getOrDefault(topicId, Collections.emptySet());
|
||||||
Set<Integer> currentAssignedPartitions = member.assignedPartitions()
|
Set<Integer> currentAssignedPartitions = memberAssignedPartitions
|
||||||
.getOrDefault(topicId, Collections.emptySet());
|
|
||||||
Set<Integer> currentRevokingPartitions = member.partitionsPendingRevocation()
|
|
||||||
.getOrDefault(topicId, Collections.emptySet());
|
.getOrDefault(topicId, Collections.emptySet());
|
||||||
|
|
||||||
// Assigned_1 = (Assigned_0 + Pending_Revocation_0) ∩ Target
|
// New Assigned Partitions = Previous Assigned Partitions ∩ Target
|
||||||
// Assigned_0 + Pending_Revocation_0 is used here because the partitions
|
|
||||||
// being revoked are still owned until the revocation is acknowledged.
|
|
||||||
Set<Integer> assignedPartitions = new HashSet<>(currentAssignedPartitions);
|
Set<Integer> assignedPartitions = new HashSet<>(currentAssignedPartitions);
|
||||||
assignedPartitions.addAll(currentRevokingPartitions);
|
|
||||||
assignedPartitions.retainAll(target);
|
assignedPartitions.retainAll(target);
|
||||||
|
|
||||||
// Pending_Revocation_1 = (Assigned_0 + Pending_Revocation_0) - Assigned_1
|
// Partitions Pending Revocation = Previous Assigned Partitions - New Assigned Partitions
|
||||||
// Assigned_0 + Pending_Revocation_0 is used here because the partitions
|
|
||||||
// being revoked are still owned until the revocation is acknowledged.
|
|
||||||
Set<Integer> partitionsPendingRevocation = new HashSet<>(currentAssignedPartitions);
|
Set<Integer> partitionsPendingRevocation = new HashSet<>(currentAssignedPartitions);
|
||||||
partitionsPendingRevocation.addAll(currentRevokingPartitions);
|
|
||||||
partitionsPendingRevocation.removeAll(assignedPartitions);
|
partitionsPendingRevocation.removeAll(assignedPartitions);
|
||||||
|
|
||||||
// Pending_Assignment_1 = Target - Assigned_1
|
// Partitions Pending Assignment = Target - New Assigned Partitions - Unreleased Partitions
|
||||||
Set<Integer> partitionsPendingAssignment = new HashSet<>(target);
|
Set<Integer> partitionsPendingAssignment = new HashSet<>(target);
|
||||||
partitionsPendingAssignment.removeAll(assignedPartitions);
|
partitionsPendingAssignment.removeAll(assignedPartitions);
|
||||||
|
hasUnreleasedPartitions = partitionsPendingAssignment.removeIf(partitionId ->
|
||||||
|
currentPartitionEpoch.apply(topicId, partitionId) != -1
|
||||||
|
) || hasUnreleasedPartitions;
|
||||||
|
|
||||||
if (!assignedPartitions.isEmpty()) {
|
if (!assignedPartitions.isEmpty()) {
|
||||||
newAssignedPartitions.put(topicId, assignedPartitions);
|
newAssignedPartitions.put(topicId, assignedPartitions);
|
||||||
|
@ -251,195 +258,51 @@ public class CurrentAssignmentBuilder {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!newPartitionsPendingRevocation.isEmpty()) {
|
if (!newPartitionsPendingRevocation.isEmpty()) {
|
||||||
// If the partition pending revocation set is not empty, we transition the
|
// If there are partitions to be revoked, the member remains in its current
|
||||||
// member to revoking and keep the current epoch. The transition to the new
|
// epoch and requests the revocation of those partitions. It transitions to
|
||||||
// state is done when the member is updated.
|
// the UNREVOKED_PARTITIONS state to wait until the client acknowledges the
|
||||||
|
// revocation of the partitions.
|
||||||
return new ConsumerGroupMember.Builder(member)
|
return new ConsumerGroupMember.Builder(member)
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
|
.updateMemberEpoch(memberEpoch)
|
||||||
.setAssignedPartitions(newAssignedPartitions)
|
.setAssignedPartitions(newAssignedPartitions)
|
||||||
.setPartitionsPendingRevocation(newPartitionsPendingRevocation)
|
.setPartitionsPendingRevocation(newPartitionsPendingRevocation)
|
||||||
.setPartitionsPendingAssignment(newPartitionsPendingAssignment)
|
|
||||||
.setTargetMemberEpoch(targetAssignmentEpoch)
|
|
||||||
.build();
|
.build();
|
||||||
} else {
|
} else if (!newPartitionsPendingAssignment.isEmpty()) {
|
||||||
if (!newPartitionsPendingAssignment.isEmpty()) {
|
// If there are partitions to be assigned, the member transitions to the
|
||||||
// If the partitions pending assignment set is not empty, we check
|
// target epoch and requests the assignment of those partitions. Note that
|
||||||
// if some or all partitions are free to use. If they are, we move
|
// the partitions are directly added to the assigned partitions set. The
|
||||||
// them to the partitions assigned set.
|
// member transitions to the STABLE state or to the UNRELEASED_PARTITIONS
|
||||||
maybeAssignPendingPartitions(newAssignedPartitions, newPartitionsPendingAssignment);
|
// state depending on whether there are unreleased partitions or not.
|
||||||
}
|
newPartitionsPendingAssignment.forEach((topicId, partitions) -> newAssignedPartitions
|
||||||
|
.computeIfAbsent(topicId, __ -> new HashSet<>())
|
||||||
// We transition to the target epoch. If the partitions pending assignment
|
.addAll(partitions));
|
||||||
// set is empty, the member transition to stable, otherwise to assigning.
|
MemberState newState = hasUnreleasedPartitions ? MemberState.UNRELEASED_PARTITIONS : MemberState.STABLE;
|
||||||
// The transition to the new state is done when the member is updated.
|
|
||||||
return new ConsumerGroupMember.Builder(member)
|
return new ConsumerGroupMember.Builder(member)
|
||||||
|
.setState(newState)
|
||||||
|
.updateMemberEpoch(targetAssignmentEpoch)
|
||||||
.setAssignedPartitions(newAssignedPartitions)
|
.setAssignedPartitions(newAssignedPartitions)
|
||||||
.setPartitionsPendingRevocation(Collections.emptyMap())
|
.setPartitionsPendingRevocation(Collections.emptyMap())
|
||||||
.setPartitionsPendingAssignment(newPartitionsPendingAssignment)
|
|
||||||
.setPreviousMemberEpoch(member.memberEpoch())
|
|
||||||
.setMemberEpoch(targetAssignmentEpoch)
|
|
||||||
.setTargetMemberEpoch(targetAssignmentEpoch)
|
|
||||||
.build();
|
.build();
|
||||||
}
|
} else if (hasUnreleasedPartitions) {
|
||||||
}
|
// If there are no partitions to be revoked nor to be assigned but some
|
||||||
|
// partitions are not available yet, the member transitions to the target
|
||||||
/**
|
// epoch, to the UNRELEASED_PARTITIONS state and waits.
|
||||||
* Tries to transition from Revoke to Assigning or Stable. This is only
|
|
||||||
* possible when the member acknowledges that it only owns the partition
|
|
||||||
* in the assigned partitions.
|
|
||||||
*
|
|
||||||
* @return A new ConsumerGroupMember with the new state or the current one
|
|
||||||
* if the member stays in the current state.
|
|
||||||
*/
|
|
||||||
private ConsumerGroupMember maybeTransitionFromRevokingToAssigningOrStable() {
|
|
||||||
if (member.partitionsPendingRevocation().isEmpty() || matchesAssignedPartitions(ownedTopicPartitions)) {
|
|
||||||
Map<Uuid, Set<Integer>> newAssignedPartitions = deepCopy(member.assignedPartitions());
|
|
||||||
Map<Uuid, Set<Integer>> newPartitionsPendingAssignment = deepCopy(member.partitionsPendingAssignment());
|
|
||||||
|
|
||||||
if (!newPartitionsPendingAssignment.isEmpty()) {
|
|
||||||
// If the partitions pending assignment set is not empty, we check
|
|
||||||
// if some or all partitions are free to use. If they are, we move
|
|
||||||
// them to the assigned set.
|
|
||||||
maybeAssignPendingPartitions(newAssignedPartitions, newPartitionsPendingAssignment);
|
|
||||||
}
|
|
||||||
|
|
||||||
// We transition to the target epoch. If the partitions pending assignment
|
|
||||||
// set is empty, the member transition to stable, otherwise to assigning.
|
|
||||||
// The transition to the new state is done when the member is updated.
|
|
||||||
return new ConsumerGroupMember.Builder(member)
|
return new ConsumerGroupMember.Builder(member)
|
||||||
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
|
.updateMemberEpoch(targetAssignmentEpoch)
|
||||||
.setAssignedPartitions(newAssignedPartitions)
|
.setAssignedPartitions(newAssignedPartitions)
|
||||||
.setPartitionsPendingRevocation(Collections.emptyMap())
|
.setPartitionsPendingRevocation(Collections.emptyMap())
|
||||||
.setPartitionsPendingAssignment(newPartitionsPendingAssignment)
|
|
||||||
.setPreviousMemberEpoch(member.memberEpoch())
|
|
||||||
.setMemberEpoch(targetAssignmentEpoch)
|
|
||||||
.setTargetMemberEpoch(targetAssignmentEpoch)
|
|
||||||
.build();
|
.build();
|
||||||
} else {
|
} else {
|
||||||
return member;
|
// Otherwise, the member transitions to the target epoch and to the
|
||||||
}
|
// STABLE state.
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Tries to transition from Assigning to Assigning or Stable. This is only
|
|
||||||
* possible when one or more partitions in the partitions pending assignment
|
|
||||||
* set have been freed up by other members in the group.
|
|
||||||
*
|
|
||||||
* @return A new ConsumerGroupMember with the new state or the current one
|
|
||||||
* if the member stays in the current state.
|
|
||||||
*/
|
|
||||||
private ConsumerGroupMember maybeTransitionFromAssigningToAssigningOrStable() {
|
|
||||||
Map<Uuid, Set<Integer>> newAssignedPartitions = deepCopy(member.assignedPartitions());
|
|
||||||
Map<Uuid, Set<Integer>> newPartitionsPendingAssignment = deepCopy(member.partitionsPendingAssignment());
|
|
||||||
|
|
||||||
// If any partition can transition from assigning to assigned, we update
|
|
||||||
// the member. Otherwise, we return the current one. The transition to the
|
|
||||||
// new state is done when the member is updated.
|
|
||||||
if (maybeAssignPendingPartitions(newAssignedPartitions, newPartitionsPendingAssignment)) {
|
|
||||||
return new ConsumerGroupMember.Builder(member)
|
return new ConsumerGroupMember.Builder(member)
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
|
.updateMemberEpoch(targetAssignmentEpoch)
|
||||||
.setAssignedPartitions(newAssignedPartitions)
|
.setAssignedPartitions(newAssignedPartitions)
|
||||||
.setPartitionsPendingRevocation(Collections.emptyMap())
|
.setPartitionsPendingRevocation(Collections.emptyMap())
|
||||||
.setPartitionsPendingAssignment(newPartitionsPendingAssignment)
|
|
||||||
.setPreviousMemberEpoch(member.memberEpoch())
|
|
||||||
.setMemberEpoch(targetAssignmentEpoch)
|
|
||||||
.setTargetMemberEpoch(targetAssignmentEpoch)
|
|
||||||
.build();
|
.build();
|
||||||
} else {
|
|
||||||
return member;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Tries to move partitions from the partitions pending assignment set to
|
|
||||||
* the partitions assigned set if they are no longer owned.
|
|
||||||
*
|
|
||||||
* @param newAssignedPartitions The assigned partitions.
|
|
||||||
* @param newPartitionsPendingAssignment The partitions pending assignment.
|
|
||||||
* @return A boolean indicating if any partitions were moved.
|
|
||||||
*/
|
|
||||||
private boolean maybeAssignPendingPartitions(
|
|
||||||
Map<Uuid, Set<Integer>> newAssignedPartitions,
|
|
||||||
Map<Uuid, Set<Integer>> newPartitionsPendingAssignment
|
|
||||||
) {
|
|
||||||
boolean changed = false;
|
|
||||||
|
|
||||||
Iterator<Map.Entry<Uuid, Set<Integer>>> assigningSetIterator =
|
|
||||||
newPartitionsPendingAssignment.entrySet().iterator();
|
|
||||||
|
|
||||||
while (assigningSetIterator.hasNext()) {
|
|
||||||
Map.Entry<Uuid, Set<Integer>> pair = assigningSetIterator.next();
|
|
||||||
Uuid topicId = pair.getKey();
|
|
||||||
Set<Integer> assigning = pair.getValue();
|
|
||||||
|
|
||||||
Iterator<Integer> assigningIterator = assigning.iterator();
|
|
||||||
while (assigningIterator.hasNext()) {
|
|
||||||
Integer partitionId = assigningIterator.next();
|
|
||||||
|
|
||||||
// A partition can be assigned to this member iff it has been
|
|
||||||
// released by its previous owner. This is signaled by -1.
|
|
||||||
Integer partitionEpoch = currentPartitionEpoch.apply(topicId, partitionId);
|
|
||||||
if (partitionEpoch == -1) {
|
|
||||||
assigningIterator.remove();
|
|
||||||
put(newAssignedPartitions, topicId, partitionId);
|
|
||||||
changed = true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (assigning.isEmpty()) {
|
|
||||||
assigningSetIterator.remove();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return changed;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Checks whether the owned topic partitions passed by the member to the state
|
|
||||||
* machine via the ConsumerGroupHeartbeat request corresponds to the assigned
|
|
||||||
* partitions.
|
|
||||||
*
|
|
||||||
* @param ownedTopicPartitions The topic partitions owned by the remove client.
|
|
||||||
* @return A boolean indicating if the owned partitions matches the Assigned set.
|
|
||||||
*/
|
|
||||||
private boolean matchesAssignedPartitions(
|
|
||||||
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
|
|
||||||
) {
|
|
||||||
if (ownedTopicPartitions == null) return false;
|
|
||||||
if (ownedTopicPartitions.size() != member.assignedPartitions().size()) return false;
|
|
||||||
|
|
||||||
for (ConsumerGroupHeartbeatRequestData.TopicPartitions topicPartitions : ownedTopicPartitions) {
|
|
||||||
Set<Integer> partitions = member.assignedPartitions().get(topicPartitions.topicId());
|
|
||||||
if (partitions == null) return false;
|
|
||||||
for (Integer partitionId : topicPartitions.partitions()) {
|
|
||||||
if (!partitions.contains(partitionId)) return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Makes a deep copy of an assignment map.
|
|
||||||
*
|
|
||||||
* @param map The Map to copy.
|
|
||||||
* @return The copy.
|
|
||||||
*/
|
|
||||||
private Map<Uuid, Set<Integer>> deepCopy(Map<Uuid, Set<Integer>> map) {
|
|
||||||
Map<Uuid, Set<Integer>> copy = new HashMap<>();
|
|
||||||
map.forEach((topicId, partitions) -> copy.put(topicId, new HashSet<>(partitions)));
|
|
||||||
return copy;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Puts the given TopicId and Partitions to the given map.
|
|
||||||
*/
|
|
||||||
private void put(
|
|
||||||
Map<Uuid, Set<Integer>> map,
|
|
||||||
Uuid topicId,
|
|
||||||
Integer partitionId
|
|
||||||
) {
|
|
||||||
map.compute(topicId, (__, partitionsOrNull) -> {
|
|
||||||
if (partitionsOrNull == null) partitionsOrNull = new HashSet<>();
|
|
||||||
partitionsOrNull.add(partitionId);
|
|
||||||
return partitionsOrNull;
|
|
||||||
});
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,76 @@
|
||||||
|
/*
|
||||||
|
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
* contributor license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright ownership.
|
||||||
|
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
* (the "License"); you may not use this file except in compliance with
|
||||||
|
* the License. You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing, software
|
||||||
|
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
* See the License for the specific language governing permissions and
|
||||||
|
* limitations under the License.
|
||||||
|
*/
|
||||||
|
package org.apache.kafka.coordinator.group.consumer;
|
||||||
|
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.Map;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The various states that a member can be in. For their definition,
|
||||||
|
* refer to the documentation of {{@link CurrentAssignmentBuilder}}.
|
||||||
|
*/
|
||||||
|
public enum MemberState {
|
||||||
|
/**
|
||||||
|
* The member is fully reconciled with the desired target assignment.
|
||||||
|
*/
|
||||||
|
STABLE((byte) 0),
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The member must revoke some partitions in order to be able to
|
||||||
|
* transition to the next epoch.
|
||||||
|
*/
|
||||||
|
UNREVOKED_PARTITIONS((byte) 1),
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The member transitioned to the last epoch but waits on some
|
||||||
|
* partitions which have not been revoked by their previous
|
||||||
|
* owners yet.
|
||||||
|
*/
|
||||||
|
UNRELEASED_PARTITIONS((byte) 2),
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The member is in an unknown state. This can only happen if a future
|
||||||
|
* version of the software introduces a new state unknown by this version.
|
||||||
|
*/
|
||||||
|
UNKNOWN((byte) 127);
|
||||||
|
|
||||||
|
private final static Map<Byte, MemberState> VALUES_TO_ENUMS = new HashMap<>();
|
||||||
|
|
||||||
|
static {
|
||||||
|
for (MemberState state: MemberState.values()) {
|
||||||
|
VALUES_TO_ENUMS.put(state.value(), state);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private final byte value;
|
||||||
|
|
||||||
|
MemberState(byte value) {
|
||||||
|
this.value = value;
|
||||||
|
}
|
||||||
|
|
||||||
|
public byte value() {
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemberState fromValue(byte value) {
|
||||||
|
MemberState state = VALUES_TO_ENUMS.get(value);
|
||||||
|
if (state == null) {
|
||||||
|
return UNKNOWN;
|
||||||
|
}
|
||||||
|
return state;
|
||||||
|
}
|
||||||
|
}
|
|
@ -24,20 +24,12 @@
|
||||||
"about": "The current member epoch that is expected from the member in the heartbeat request." },
|
"about": "The current member epoch that is expected from the member in the heartbeat request." },
|
||||||
{ "name": "PreviousMemberEpoch", "versions": "0+", "type": "int32",
|
{ "name": "PreviousMemberEpoch", "versions": "0+", "type": "int32",
|
||||||
"about": "If the last epoch bump is lost before reaching the member, the member will retry with the previous epoch." },
|
"about": "If the last epoch bump is lost before reaching the member, the member will retry with the previous epoch." },
|
||||||
{ "name": "TargetMemberEpoch", "versions": "0+", "type": "int32",
|
{ "name": "State", "versions": "0+", "type": "int8",
|
||||||
"about": "The target epoch corresponding to the assignment used to compute the AssignedPartitions, the PartitionsPendingRevocation and the PartitionsPendingAssignment fields." },
|
"about": "The member state. See ConsumerGroupMember.MemberState for the possible values." },
|
||||||
{ "name": "AssignedPartitions", "versions": "0+", "type": "[]TopicPartitions",
|
{ "name": "AssignedPartitions", "versions": "0+", "type": "[]TopicPartitions",
|
||||||
"about": "The partitions assigned to (or owned by) this member." },
|
"about": "The partitions assigned to (or owned by) this member." },
|
||||||
{ "name": "PartitionsPendingRevocation", "versions": "0+", "type": "[]TopicPartitions",
|
{ "name": "PartitionsPendingRevocation", "versions": "0+", "type": "[]TopicPartitions",
|
||||||
"about": "The partitions that must be revoked by this member." },
|
"about": "The partitions that must be revoked by this member." }
|
||||||
{ "name": "PartitionsPendingAssignment", "versions": "0+", "type": "[]TopicPartitions",
|
|
||||||
"about": "The partitions that will be assigned to this member when they are freed up by their current owners." },
|
|
||||||
{ "name": "Error", "versions": "0+", "type": "int8",
|
|
||||||
"about": "The error reported by the assignor." },
|
|
||||||
{ "name": "MetadataVersion", "versions": "0+", "type": "int16",
|
|
||||||
"about": "The version of the metadata bytes." },
|
|
||||||
{ "name": "MetadataBytes", "versions": "0+", "type": "bytes",
|
|
||||||
"about": "The metadata bytes." }
|
|
||||||
],
|
],
|
||||||
"commonStructs": [
|
"commonStructs": [
|
||||||
{ "name": "TopicPartitions", "versions": "0+", "fields": [
|
{ "name": "TopicPartitions", "versions": "0+", "fields": [
|
||||||
|
|
|
@ -144,18 +144,12 @@ public class Assertions {
|
||||||
|
|
||||||
assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
|
assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
|
||||||
assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
|
assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
|
||||||
assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
|
|
||||||
assertEquals(expectedValue.error(), actualValue.error());
|
|
||||||
assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
|
|
||||||
assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
|
|
||||||
|
|
||||||
// We transform those to Maps before comparing them.
|
// We transform those to Maps before comparing them.
|
||||||
assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
|
assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
|
||||||
fromTopicPartitions(actualValue.assignedPartitions()));
|
fromTopicPartitions(actualValue.assignedPartitions()));
|
||||||
assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
|
assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
|
||||||
fromTopicPartitions(actualValue.partitionsPendingRevocation()));
|
fromTopicPartitions(actualValue.partitionsPendingRevocation()));
|
||||||
assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
|
|
||||||
fromTopicPartitions(actualValue.partitionsPendingAssignment()));
|
|
||||||
} else if (actual.message() instanceof ConsumerGroupPartitionMetadataValue) {
|
} else if (actual.message() instanceof ConsumerGroupPartitionMetadataValue) {
|
||||||
// The order of the racks stored in the PartitionMetadata of the ConsumerGroupPartitionMetadataValue
|
// The order of the racks stored in the PartitionMetadata of the ConsumerGroupPartitionMetadataValue
|
||||||
// is not always guaranteed. Therefore, we need a special comparator.
|
// is not always guaranteed. Therefore, we need a special comparator.
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -47,7 +47,7 @@ import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
|
||||||
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.ConsumerGroupMember;
|
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;
|
||||||
|
@ -85,7 +85,7 @@ import java.util.stream.IntStream;
|
||||||
import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID;
|
import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID;
|
||||||
import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT;
|
import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT;
|
||||||
import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupHeartbeatKey;
|
import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupHeartbeatKey;
|
||||||
import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRevocationTimeoutKey;
|
import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRebalanceTimeoutKey;
|
||||||
import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey;
|
import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey;
|
||||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD;
|
||||||
|
@ -487,7 +487,7 @@ public class GroupMetadataManagerTestContext {
|
||||||
.state();
|
.state();
|
||||||
}
|
}
|
||||||
|
|
||||||
public ConsumerGroupMember.MemberState consumerGroupMemberState(
|
public MemberState consumerGroupMemberState(
|
||||||
String groupId,
|
String groupId,
|
||||||
String memberId
|
String memberId
|
||||||
) {
|
) {
|
||||||
|
@ -556,24 +556,24 @@ public class GroupMetadataManagerTestContext {
|
||||||
assertNull(timeout);
|
assertNull(timeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
public MockCoordinatorTimer.ScheduledTimeout<Void, Record> assertRevocationTimeout(
|
public MockCoordinatorTimer.ScheduledTimeout<Void, Record> assertRebalanceTimeout(
|
||||||
String groupId,
|
String groupId,
|
||||||
String memberId,
|
String memberId,
|
||||||
long delayMs
|
long delayMs
|
||||||
) {
|
) {
|
||||||
MockCoordinatorTimer.ScheduledTimeout<Void, Record> timeout =
|
MockCoordinatorTimer.ScheduledTimeout<Void, Record> timeout =
|
||||||
timer.timeout(consumerGroupRevocationTimeoutKey(groupId, memberId));
|
timer.timeout(consumerGroupRebalanceTimeoutKey(groupId, memberId));
|
||||||
assertNotNull(timeout);
|
assertNotNull(timeout);
|
||||||
assertEquals(time.milliseconds() + delayMs, timeout.deadlineMs);
|
assertEquals(time.milliseconds() + delayMs, timeout.deadlineMs);
|
||||||
return timeout;
|
return timeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void assertNoRevocationTimeout(
|
public void assertNoRebalanceTimeout(
|
||||||
String groupId,
|
String groupId,
|
||||||
String memberId
|
String memberId
|
||||||
) {
|
) {
|
||||||
MockCoordinatorTimer.ScheduledTimeout<Void, Record> timeout =
|
MockCoordinatorTimer.ScheduledTimeout<Void, Record> timeout =
|
||||||
timer.timeout(consumerGroupRevocationTimeoutKey(groupId, memberId));
|
timer.timeout(consumerGroupRebalanceTimeoutKey(groupId, memberId));
|
||||||
assertNull(timeout);
|
assertNull(timeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -1116,7 +1116,6 @@ public class OffsetMetadataManagerTest {
|
||||||
// Add member.
|
// Add member.
|
||||||
group.updateMember(new ConsumerGroupMember.Builder("member")
|
group.updateMember(new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.build()
|
.build()
|
||||||
);
|
);
|
||||||
|
@ -1162,7 +1161,6 @@ public class OffsetMetadataManagerTest {
|
||||||
// Add member.
|
// Add member.
|
||||||
group.updateMember(new ConsumerGroupMember.Builder("member")
|
group.updateMember(new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.build()
|
.build()
|
||||||
);
|
);
|
||||||
|
@ -1256,7 +1254,6 @@ public class OffsetMetadataManagerTest {
|
||||||
// Add member.
|
// Add member.
|
||||||
group.updateMember(new ConsumerGroupMember.Builder("member")
|
group.updateMember(new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.build()
|
.build()
|
||||||
);
|
);
|
||||||
|
@ -1327,7 +1324,6 @@ public class OffsetMetadataManagerTest {
|
||||||
// Add member.
|
// Add member.
|
||||||
group.updateMember(new ConsumerGroupMember.Builder("member")
|
group.updateMember(new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.build()
|
.build()
|
||||||
);
|
);
|
||||||
|
@ -1405,7 +1401,6 @@ public class OffsetMetadataManagerTest {
|
||||||
// Add member.
|
// Add member.
|
||||||
group.updateMember(new ConsumerGroupMember.Builder("member")
|
group.updateMember(new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.build()
|
.build()
|
||||||
);
|
);
|
||||||
|
@ -1525,7 +1520,6 @@ public class OffsetMetadataManagerTest {
|
||||||
// Add member.
|
// Add member.
|
||||||
group.updateMember(new ConsumerGroupMember.Builder("member")
|
group.updateMember(new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.build()
|
.build()
|
||||||
);
|
);
|
||||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.kafka.common.utils.MockTime;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ClientAssignor;
|
import org.apache.kafka.coordinator.group.consumer.ClientAssignor;
|
||||||
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.consumer.VersionedMetadata;
|
import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
|
||||||
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentKey;
|
||||||
|
@ -397,11 +398,6 @@ public class RecordHelpersTest {
|
||||||
mkSortedTopicAssignment(topicId2, 24, 25, 26)
|
mkSortedTopicAssignment(topicId2, 24, 25, 26)
|
||||||
);
|
);
|
||||||
|
|
||||||
Map<Uuid, Set<Integer>> assigning = mkSortedAssignment(
|
|
||||||
mkSortedTopicAssignment(topicId1, 17, 18, 19),
|
|
||||||
mkSortedTopicAssignment(topicId2, 27, 28, 29)
|
|
||||||
);
|
|
||||||
|
|
||||||
Record expectedRecord = new Record(
|
Record expectedRecord = new Record(
|
||||||
new ApiMessageAndVersion(
|
new ApiMessageAndVersion(
|
||||||
new ConsumerGroupCurrentMemberAssignmentKey()
|
new ConsumerGroupCurrentMemberAssignmentKey()
|
||||||
|
@ -410,9 +406,9 @@ public class RecordHelpersTest {
|
||||||
(short) 8),
|
(short) 8),
|
||||||
new ApiMessageAndVersion(
|
new ApiMessageAndVersion(
|
||||||
new ConsumerGroupCurrentMemberAssignmentValue()
|
new ConsumerGroupCurrentMemberAssignmentValue()
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS.value())
|
||||||
.setMemberEpoch(22)
|
.setMemberEpoch(22)
|
||||||
.setPreviousMemberEpoch(21)
|
.setPreviousMemberEpoch(21)
|
||||||
.setTargetMemberEpoch(23)
|
|
||||||
.setAssignedPartitions(Arrays.asList(
|
.setAssignedPartitions(Arrays.asList(
|
||||||
new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
||||||
.setTopicId(topicId1)
|
.setTopicId(topicId1)
|
||||||
|
@ -426,25 +422,17 @@ public class RecordHelpersTest {
|
||||||
.setPartitions(Arrays.asList(14, 15, 16)),
|
.setPartitions(Arrays.asList(14, 15, 16)),
|
||||||
new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
||||||
.setTopicId(topicId2)
|
.setTopicId(topicId2)
|
||||||
.setPartitions(Arrays.asList(24, 25, 26))))
|
.setPartitions(Arrays.asList(24, 25, 26)))),
|
||||||
.setPartitionsPendingAssignment(Arrays.asList(
|
|
||||||
new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
|
||||||
.setTopicId(topicId1)
|
|
||||||
.setPartitions(Arrays.asList(17, 18, 19)),
|
|
||||||
new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
|
||||||
.setTopicId(topicId2)
|
|
||||||
.setPartitions(Arrays.asList(27, 28, 29)))),
|
|
||||||
(short) 0));
|
(short) 0));
|
||||||
|
|
||||||
assertEquals(expectedRecord, newCurrentAssignmentRecord(
|
assertEquals(expectedRecord, newCurrentAssignmentRecord(
|
||||||
"group-id",
|
"group-id",
|
||||||
new ConsumerGroupMember.Builder("member-id")
|
new ConsumerGroupMember.Builder("member-id")
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(22)
|
.setMemberEpoch(22)
|
||||||
.setPreviousMemberEpoch(21)
|
.setPreviousMemberEpoch(21)
|
||||||
.setTargetMemberEpoch(23)
|
|
||||||
.setAssignedPartitions(assigned)
|
.setAssignedPartitions(assigned)
|
||||||
.setPartitionsPendingRevocation(revoking)
|
.setPartitionsPendingRevocation(revoking)
|
||||||
.setPartitionsPendingAssignment(assigning)
|
|
||||||
.build()
|
.build()
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
|
|
@ -51,7 +51,6 @@ public class ConsumerGroupMemberTest {
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(9)
|
.setPreviousMemberEpoch(9)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setInstanceId("instance-id")
|
.setInstanceId("instance-id")
|
||||||
.setRackId("rack-id")
|
.setRackId("rack-id")
|
||||||
.setRebalanceTimeoutMs(5000)
|
.setRebalanceTimeoutMs(5000)
|
||||||
|
@ -73,14 +72,11 @@ public class ConsumerGroupMemberTest {
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3)))
|
mkTopicAssignment(topicId1, 1, 2, 3)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId3, 7, 8, 9)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals("member-id", member.memberId());
|
assertEquals("member-id", member.memberId());
|
||||||
assertEquals(10, member.memberEpoch());
|
assertEquals(10, member.memberEpoch());
|
||||||
assertEquals(9, member.previousMemberEpoch());
|
assertEquals(9, member.previousMemberEpoch());
|
||||||
assertEquals(11, member.targetMemberEpoch());
|
|
||||||
assertEquals("instance-id", member.instanceId());
|
assertEquals("instance-id", member.instanceId());
|
||||||
assertEquals("rack-id", member.rackId());
|
assertEquals("rack-id", member.rackId());
|
||||||
assertEquals("client-id", member.clientId());
|
assertEquals("client-id", member.clientId());
|
||||||
|
@ -102,7 +98,6 @@ public class ConsumerGroupMemberTest {
|
||||||
member.clientAssignors());
|
member.clientAssignors());
|
||||||
assertEquals(mkAssignment(mkTopicAssignment(topicId1, 1, 2, 3)), member.assignedPartitions());
|
assertEquals(mkAssignment(mkTopicAssignment(topicId1, 1, 2, 3)), member.assignedPartitions());
|
||||||
assertEquals(mkAssignment(mkTopicAssignment(topicId2, 4, 5, 6)), member.partitionsPendingRevocation());
|
assertEquals(mkAssignment(mkTopicAssignment(topicId2, 4, 5, 6)), member.partitionsPendingRevocation());
|
||||||
assertEquals(mkAssignment(mkTopicAssignment(topicId3, 7, 8, 9)), member.partitionsPendingAssignment());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -114,7 +109,6 @@ public class ConsumerGroupMemberTest {
|
||||||
ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member-id")
|
ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member-id")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(9)
|
.setPreviousMemberEpoch(9)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setInstanceId("instance-id")
|
.setInstanceId("instance-id")
|
||||||
.setRackId("rack-id")
|
.setRackId("rack-id")
|
||||||
.setRebalanceTimeoutMs(5000)
|
.setRebalanceTimeoutMs(5000)
|
||||||
|
@ -136,14 +130,11 @@ public class ConsumerGroupMemberTest {
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3)))
|
mkTopicAssignment(topicId1, 1, 2, 3)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId3, 7, 8, 9)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member-id")
|
ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member-id")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(9)
|
.setPreviousMemberEpoch(9)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setInstanceId("instance-id")
|
.setInstanceId("instance-id")
|
||||||
.setRackId("rack-id")
|
.setRackId("rack-id")
|
||||||
.setRebalanceTimeoutMs(5000)
|
.setRebalanceTimeoutMs(5000)
|
||||||
|
@ -165,8 +156,6 @@ public class ConsumerGroupMemberTest {
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3)))
|
mkTopicAssignment(topicId1, 1, 2, 3)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId3, 7, 8, 9)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(member1, member2);
|
assertEquals(member1, member2);
|
||||||
|
@ -181,7 +170,6 @@ public class ConsumerGroupMemberTest {
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id")
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(9)
|
.setPreviousMemberEpoch(9)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setInstanceId("instance-id")
|
.setInstanceId("instance-id")
|
||||||
.setRackId("rack-id")
|
.setRackId("rack-id")
|
||||||
.setRebalanceTimeoutMs(5000)
|
.setRebalanceTimeoutMs(5000)
|
||||||
|
@ -203,8 +191,6 @@ public class ConsumerGroupMemberTest {
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3)))
|
mkTopicAssignment(topicId1, 1, 2, 3)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId3, 7, 8, 9)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// This is a no-op.
|
// This is a no-op.
|
||||||
|
@ -291,16 +277,12 @@ public class ConsumerGroupMemberTest {
|
||||||
ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue()
|
ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue()
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(9)
|
.setPreviousMemberEpoch(9)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
.setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
||||||
.setTopicId(topicId1)
|
.setTopicId(topicId1)
|
||||||
.setPartitions(Arrays.asList(0, 1, 2))))
|
.setPartitions(Arrays.asList(0, 1, 2))))
|
||||||
.setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
.setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
||||||
.setTopicId(topicId2)
|
.setTopicId(topicId2)
|
||||||
.setPartitions(Arrays.asList(3, 4, 5))))
|
.setPartitions(Arrays.asList(3, 4, 5))));
|
||||||
.setPartitionsPendingAssignment(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
|
||||||
.setTopicId(topicId3)
|
|
||||||
.setPartitions(Arrays.asList(6, 7, 8))));
|
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member-id")
|
||||||
.updateWith(record)
|
.updateWith(record)
|
||||||
|
@ -308,10 +290,8 @@ public class ConsumerGroupMemberTest {
|
||||||
|
|
||||||
assertEquals(10, member.memberEpoch());
|
assertEquals(10, member.memberEpoch());
|
||||||
assertEquals(9, member.previousMemberEpoch());
|
assertEquals(9, member.previousMemberEpoch());
|
||||||
assertEquals(11, member.targetMemberEpoch());
|
|
||||||
assertEquals(mkAssignment(mkTopicAssignment(topicId1, 0, 1, 2)), member.assignedPartitions());
|
assertEquals(mkAssignment(mkTopicAssignment(topicId1, 0, 1, 2)), member.assignedPartitions());
|
||||||
assertEquals(mkAssignment(mkTopicAssignment(topicId2, 3, 4, 5)), member.partitionsPendingRevocation());
|
assertEquals(mkAssignment(mkTopicAssignment(topicId2, 3, 4, 5)), member.partitionsPendingRevocation());
|
||||||
assertEquals(mkAssignment(mkTopicAssignment(topicId3, 6, 7, 8)), member.partitionsPendingAssignment());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -331,16 +311,12 @@ public class ConsumerGroupMemberTest {
|
||||||
ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue()
|
ConsumerGroupCurrentMemberAssignmentValue record = new ConsumerGroupCurrentMemberAssignmentValue()
|
||||||
.setMemberEpoch(epoch)
|
.setMemberEpoch(epoch)
|
||||||
.setPreviousMemberEpoch(epoch - 1)
|
.setPreviousMemberEpoch(epoch - 1)
|
||||||
.setTargetMemberEpoch(epoch + 1)
|
|
||||||
.setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
.setAssignedPartitions(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
||||||
.setTopicId(topicId1)
|
.setTopicId(topicId1)
|
||||||
.setPartitions(assignedPartitions)))
|
.setPartitions(assignedPartitions)))
|
||||||
.setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
.setPartitionsPendingRevocation(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
||||||
.setTopicId(topicId2)
|
.setTopicId(topicId2)
|
||||||
.setPartitions(Arrays.asList(3, 4, 5))))
|
.setPartitions(Arrays.asList(3, 4, 5))));
|
||||||
.setPartitionsPendingAssignment(Collections.singletonList(new ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions()
|
|
||||||
.setTopicId(topicId3)
|
|
||||||
.setPartitions(Arrays.asList(6, 7, 8))));
|
|
||||||
String memberId = Uuid.randomUuid().toString();
|
String memberId = Uuid.randomUuid().toString();
|
||||||
String clientId = "clientId";
|
String clientId = "clientId";
|
||||||
String instanceId = "instanceId";
|
String instanceId = "instanceId";
|
||||||
|
|
|
@ -177,8 +177,6 @@ public class ConsumerGroupTest {
|
||||||
mkTopicAssignment(fooTopicId, 1, 2, 3)))
|
mkTopicAssignment(fooTopicId, 1, 2, 3)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(zarTopicId, 7, 8, 9)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member);
|
consumerGroup.updateMember(member);
|
||||||
|
@ -199,8 +197,6 @@ public class ConsumerGroupTest {
|
||||||
mkTopicAssignment(barTopicId, 1, 2, 3)))
|
mkTopicAssignment(barTopicId, 1, 2, 3)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(zarTopicId, 4, 5, 6)))
|
mkTopicAssignment(zarTopicId, 4, 5, 6)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(fooTopicId, 7, 8, 9)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member);
|
consumerGroup.updateMember(member);
|
||||||
|
@ -337,8 +333,6 @@ public class ConsumerGroupTest {
|
||||||
mkTopicAssignment(fooTopicId, 1, 2, 3)))
|
mkTopicAssignment(fooTopicId, 1, 2, 3)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
mkTopicAssignment(barTopicId, 4, 5, 6)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(zarTopicId, 7, 8, 9)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member);
|
consumerGroup.updateMember(member);
|
||||||
|
@ -373,27 +367,27 @@ public class ConsumerGroupTest {
|
||||||
assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, consumerGroup.state());
|
assertEquals(ConsumerGroup.ConsumerGroupState.EMPTY, consumerGroup.state());
|
||||||
|
|
||||||
ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1")
|
ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(1)
|
.setMemberEpoch(1)
|
||||||
.setPreviousMemberEpoch(0)
|
.setPreviousMemberEpoch(0)
|
||||||
.setTargetMemberEpoch(1)
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member1);
|
consumerGroup.updateMember(member1);
|
||||||
consumerGroup.setGroupEpoch(1);
|
consumerGroup.setGroupEpoch(1);
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member1.state());
|
assertEquals(MemberState.STABLE, member1.state());
|
||||||
assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state());
|
assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state());
|
||||||
|
|
||||||
ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2")
|
ConsumerGroupMember member2 = new ConsumerGroupMember.Builder("member2")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(1)
|
.setMemberEpoch(1)
|
||||||
.setPreviousMemberEpoch(0)
|
.setPreviousMemberEpoch(0)
|
||||||
.setTargetMemberEpoch(1)
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member2);
|
consumerGroup.updateMember(member2);
|
||||||
consumerGroup.setGroupEpoch(2);
|
consumerGroup.setGroupEpoch(2);
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member2.state());
|
assertEquals(MemberState.STABLE, member2.state());
|
||||||
assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state());
|
assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, consumerGroup.state());
|
||||||
|
|
||||||
consumerGroup.setTargetAssignmentEpoch(2);
|
consumerGroup.setTargetAssignmentEpoch(2);
|
||||||
|
@ -401,42 +395,37 @@ public class ConsumerGroupTest {
|
||||||
assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state());
|
assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state());
|
||||||
|
|
||||||
member1 = new ConsumerGroupMember.Builder(member1)
|
member1 = new ConsumerGroupMember.Builder(member1)
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(2)
|
.setMemberEpoch(2)
|
||||||
.setPreviousMemberEpoch(1)
|
.setPreviousMemberEpoch(1)
|
||||||
.setTargetMemberEpoch(2)
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member1);
|
consumerGroup.updateMember(member1);
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member1.state());
|
assertEquals(MemberState.STABLE, member1.state());
|
||||||
assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state());
|
assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state());
|
||||||
|
|
||||||
// Member 2 is not stable so the group stays in reconciling state.
|
// Member 2 is not stable so the group stays in reconciling state.
|
||||||
member2 = new ConsumerGroupMember.Builder(member2)
|
member2 = new ConsumerGroupMember.Builder(member2)
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(2)
|
.setMemberEpoch(2)
|
||||||
.setPreviousMemberEpoch(1)
|
.setPreviousMemberEpoch(1)
|
||||||
.setTargetMemberEpoch(2)
|
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(fooTopicId, 0)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member2);
|
consumerGroup.updateMember(member2);
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member2.state());
|
assertEquals(MemberState.UNREVOKED_PARTITIONS, member2.state());
|
||||||
assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state());
|
assertEquals(ConsumerGroup.ConsumerGroupState.RECONCILING, consumerGroup.state());
|
||||||
|
|
||||||
member2 = new ConsumerGroupMember.Builder(member2)
|
member2 = new ConsumerGroupMember.Builder(member2)
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(2)
|
.setMemberEpoch(2)
|
||||||
.setPreviousMemberEpoch(1)
|
.setPreviousMemberEpoch(1)
|
||||||
.setTargetMemberEpoch(2)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
|
||||||
mkTopicAssignment(fooTopicId, 0)))
|
|
||||||
.setPartitionsPendingAssignment(Collections.emptyMap())
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member2);
|
consumerGroup.updateMember(member2);
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member2.state());
|
assertEquals(MemberState.STABLE, member2.state());
|
||||||
assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, consumerGroup.state());
|
assertEquals(ConsumerGroup.ConsumerGroupState.STABLE, consumerGroup.state());
|
||||||
|
|
||||||
consumerGroup.removeMember("member1");
|
consumerGroup.removeMember("member1");
|
||||||
|
@ -882,7 +871,6 @@ public class ConsumerGroupTest {
|
||||||
ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1")
|
ConsumerGroupMember member1 = new ConsumerGroupMember.Builder("member1")
|
||||||
.setMemberEpoch(1)
|
.setMemberEpoch(1)
|
||||||
.setPreviousMemberEpoch(0)
|
.setPreviousMemberEpoch(0)
|
||||||
.setTargetMemberEpoch(1)
|
|
||||||
.build();
|
.build();
|
||||||
consumerGroup.updateMember(member1);
|
consumerGroup.updateMember(member1);
|
||||||
|
|
||||||
|
@ -1014,7 +1002,6 @@ public class ConsumerGroupTest {
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(1)
|
.setMemberEpoch(1)
|
||||||
.setPreviousMemberEpoch(0)
|
.setPreviousMemberEpoch(0)
|
||||||
.setTargetMemberEpoch(1)
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
consumerGroup.updateMember(member);
|
consumerGroup.updateMember(member);
|
||||||
|
|
|
@ -17,542 +17,522 @@
|
||||||
package org.apache.kafka.coordinator.group.consumer;
|
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.message.ConsumerGroupHeartbeatRequestData;
|
import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
|
||||||
import org.junit.jupiter.params.provider.Arguments;
|
|
||||||
import org.junit.jupiter.params.provider.MethodSource;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.stream.Stream;
|
|
||||||
|
|
||||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
|
|
||||||
public class CurrentAssignmentBuilderTest {
|
public class CurrentAssignmentBuilderTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTransitionFromNewTargetToRevoke() {
|
public void testStableToStable() {
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(11, targetAssignment)
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 4, 5, 6))))
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
|
assertEquals(
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
new ConsumerGroupMember.Builder("member")
|
||||||
assertEquals(10, updatedMember.memberEpoch());
|
.setState(MemberState.STABLE)
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
.setMemberEpoch(11)
|
||||||
assertEquals(mkAssignment(
|
.setPreviousMemberEpoch(10)
|
||||||
mkTopicAssignment(topicId1, 3),
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 6)
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
), updatedMember.assignedPartitions());
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
assertEquals(mkAssignment(
|
.build(),
|
||||||
mkTopicAssignment(topicId1, 1, 2),
|
updatedMember
|
||||||
mkTopicAssignment(topicId2, 4, 5)
|
);
|
||||||
), updatedMember.partitionsPendingRevocation());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)
|
|
||||||
), updatedMember.partitionsPendingAssignment());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTransitionFromNewTargetToAssigning() {
|
public void testStableToStableWithNewPartitions() {
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6, 7, 8)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(11, targetAssignment)
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
.build();
|
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
|
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
|
||||||
assertEquals(11, updatedMember.memberEpoch());
|
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)
|
|
||||||
), updatedMember.assignedPartitions());
|
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)
|
|
||||||
), updatedMember.partitionsPendingAssignment());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTransitionFromNewTargetToStable() {
|
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
|
||||||
.setMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
|
||||||
.setTargetMemberEpoch(10)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)))
|
|
||||||
.build();
|
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
|
||||||
.withTargetAssignment(11, targetAssignment)
|
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
|
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
|
||||||
assertEquals(11, updatedMember.memberEpoch());
|
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)
|
|
||||||
), updatedMember.assignedPartitions());
|
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
|
|
||||||
}
|
|
||||||
|
|
||||||
private static Stream<Arguments> ownedTopicPartitionsArguments() {
|
|
||||||
return Stream.of(
|
|
||||||
// Field not set in the heartbeat request.
|
|
||||||
null,
|
|
||||||
// Owned partitions does not match the assigned partitions.
|
|
||||||
Collections.emptyList()
|
|
||||||
).map(Arguments::of);
|
|
||||||
}
|
|
||||||
|
|
||||||
@ParameterizedTest
|
|
||||||
@MethodSource("ownedTopicPartitionsArguments")
|
|
||||||
public void testTransitionFromRevokeToRevoke(
|
|
||||||
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> ownedTopicPartitions
|
|
||||||
) {
|
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
|
||||||
.setMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(10)
|
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3),
|
|
||||||
mkTopicAssignment(topicId2, 6)))
|
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5)))
|
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)))
|
|
||||||
.build();
|
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
|
||||||
.withTargetAssignment(11, targetAssignment)
|
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||||
.withOwnedTopicPartitions(ownedTopicPartitions)
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
|
assertEquals(
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
new ConsumerGroupMember.Builder("member")
|
||||||
assertEquals(10, updatedMember.memberEpoch());
|
.setState(MemberState.STABLE)
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
.setMemberEpoch(11)
|
||||||
assertEquals(mkAssignment(
|
.setPreviousMemberEpoch(10)
|
||||||
mkTopicAssignment(topicId1, 3),
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 6)
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
), updatedMember.assignedPartitions());
|
mkTopicAssignment(topicId2, 4, 5, 6, 7)))
|
||||||
assertEquals(mkAssignment(
|
.build(),
|
||||||
mkTopicAssignment(topicId1, 1, 2),
|
updatedMember
|
||||||
mkTopicAssignment(topicId2, 4, 5)
|
);
|
||||||
), updatedMember.partitionsPendingRevocation());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)
|
|
||||||
), updatedMember.partitionsPendingAssignment());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTransitionFromRevokeToAssigning() {
|
public void testStableToUnrevokedPartitions() {
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5)))
|
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||||
|
.build();
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
assertEquals(
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
new ConsumerGroupMember.Builder("member")
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
));
|
.setMemberEpoch(10)
|
||||||
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 1),
|
||||||
|
mkTopicAssignment(topicId2, 4)))
|
||||||
|
.build(),
|
||||||
|
updatedMember
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testStableToUnreleasedPartitions() {
|
||||||
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
|
.setMemberEpoch(10)
|
||||||
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
|
.build();
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(11, targetAssignment)
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 1, 2, 3, 4),
|
||||||
|
mkTopicAssignment(topicId2, 4, 5, 6, 7))))
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
||||||
.withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
|
.build();
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
|
.setMemberEpoch(11)
|
||||||
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 4, 5, 6)))
|
||||||
|
.build(),
|
||||||
|
updatedMember
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnrevokedPartitionsToStable() {
|
||||||
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
|
.setMemberEpoch(10)
|
||||||
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 1),
|
||||||
|
mkTopicAssignment(topicId2, 4)))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||||
|
.withOwnedTopicPartitions(Arrays.asList(
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId1)
|
||||||
|
.setPartitions(Arrays.asList(2, 3)),
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId2)
|
||||||
|
.setPartitions(Arrays.asList(5, 6))))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
|
.setMemberEpoch(11)
|
||||||
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
.build(),
|
||||||
|
updatedMember
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRemainsInUnrevokedPartitions() {
|
||||||
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
|
.setMemberEpoch(10)
|
||||||
|
.setPreviousMemberEpoch(10)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 1),
|
||||||
|
mkTopicAssignment(topicId2, 4)))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
CurrentAssignmentBuilder currentAssignmentBuilder = new CurrentAssignmentBuilder(member)
|
||||||
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6))))
|
mkTopicAssignment(topicId2, 6))))
|
||||||
.build();
|
.withCurrentPartitionEpoch((topicId, partitionId) -> -1);
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
|
assertEquals(
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
member,
|
||||||
assertEquals(11, updatedMember.memberEpoch());
|
currentAssignmentBuilder
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
.withOwnedTopicPartitions(null)
|
||||||
assertEquals(mkAssignment(
|
.build()
|
||||||
mkTopicAssignment(topicId1, 3),
|
);
|
||||||
mkTopicAssignment(topicId2, 6)
|
|
||||||
), updatedMember.assignedPartitions());
|
assertEquals(
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
member,
|
||||||
assertEquals(mkAssignment(
|
currentAssignmentBuilder
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
.withOwnedTopicPartitions(Arrays.asList(
|
||||||
mkTopicAssignment(topicId2, 7, 8)
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
), updatedMember.partitionsPendingAssignment());
|
.setTopicId(topicId1)
|
||||||
|
.setPartitions(Arrays.asList(1, 2, 3)),
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId2)
|
||||||
|
.setPartitions(Arrays.asList(5, 6))))
|
||||||
|
.build()
|
||||||
|
);
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
member,
|
||||||
|
currentAssignmentBuilder
|
||||||
|
.withOwnedTopicPartitions(Arrays.asList(
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId1)
|
||||||
|
.setPartitions(Arrays.asList(2, 3)),
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId2)
|
||||||
|
.setPartitions(Arrays.asList(4, 5, 6))))
|
||||||
|
.build()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTransitionFromRevokeToStable() {
|
public void testUnrevokedPartitionsToUnrevokedPartitions() {
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
.setMemberEpoch(10)
|
.setMemberEpoch(10)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2),
|
mkTopicAssignment(topicId1, 1),
|
||||||
mkTopicAssignment(topicId2, 4, 5)))
|
mkTopicAssignment(topicId2, 4)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(11, targetAssignment)
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
|
||||||
.withOwnedTopicPartitions(requestFromAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6))))
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||||
|
.withOwnedTopicPartitions(Arrays.asList(
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId1)
|
||||||
|
.setPartitions(Arrays.asList(2, 3)),
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId2)
|
||||||
|
.setPartitions(Arrays.asList(5, 6))))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
|
assertEquals(
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
new ConsumerGroupMember.Builder("member")
|
||||||
assertEquals(11, updatedMember.memberEpoch());
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
.setMemberEpoch(11)
|
||||||
assertEquals(mkAssignment(
|
.setPreviousMemberEpoch(10)
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
mkTopicAssignment(topicId1, 3),
|
||||||
), updatedMember.assignedPartitions());
|
mkTopicAssignment(topicId2, 6)))
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
|
mkTopicAssignment(topicId1, 2),
|
||||||
|
mkTopicAssignment(topicId2, 5)))
|
||||||
|
.build(),
|
||||||
|
updatedMember
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTransitionFromRevokeToStableWhenPartitionsPendingRevocationAreReassignedBeforeBeingRevoked() {
|
public void testUnrevokedPartitionsToUnreleasedPartitions() {
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(10)
|
.setPreviousMemberEpoch(10)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5)))
|
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
|
|
||||||
|
|
||||||
// A new target assignment is computed (epoch 12) before the partitions
|
|
||||||
// pending revocation are revoked by the member and those partitions
|
|
||||||
// have been reassigned to the member. In this case, the member
|
|
||||||
// can keep them a jump to epoch 12.
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(12, targetAssignment)
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
||||||
|
.withOwnedTopicPartitions(Arrays.asList(
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId1)
|
||||||
|
.setPartitions(Arrays.asList(2, 3)),
|
||||||
|
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
||||||
|
.setTopicId(topicId2)
|
||||||
|
.setPartitions(Arrays.asList(5, 6))))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
|
assertEquals(
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
new ConsumerGroupMember.Builder("member")
|
||||||
assertEquals(12, updatedMember.memberEpoch());
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
assertEquals(12, updatedMember.targetMemberEpoch());
|
.setMemberEpoch(11)
|
||||||
assertEquals(mkAssignment(
|
.setPreviousMemberEpoch(11)
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
), updatedMember.assignedPartitions());
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
.build(),
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
|
updatedMember
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTransitionFromAssigningToAssigning() {
|
public void testUnreleasedPartitionsToStable() {
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
|
||||||
.setMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(11)
|
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3),
|
|
||||||
mkTopicAssignment(topicId2, 6)))
|
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)))
|
|
||||||
.build();
|
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
|
||||||
.withTargetAssignment(11, targetAssignment)
|
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> {
|
|
||||||
if (topicId.equals(topicId1))
|
|
||||||
return -1;
|
|
||||||
else
|
|
||||||
return 10;
|
|
||||||
})
|
|
||||||
.build();
|
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, updatedMember.state());
|
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
|
||||||
assertEquals(11, updatedMember.memberEpoch());
|
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6)
|
|
||||||
), updatedMember.assignedPartitions());
|
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)
|
|
||||||
), updatedMember.partitionsPendingAssignment());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTransitionFromAssigningToStable() {
|
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
|
||||||
.setMemberEpoch(10)
|
|
||||||
.setPreviousMemberEpoch(11)
|
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3),
|
|
||||||
mkTopicAssignment(topicId2, 6)))
|
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)))
|
|
||||||
.build();
|
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.ASSIGNING, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
|
||||||
.withTargetAssignment(11, targetAssignment)
|
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
|
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
|
||||||
assertEquals(11, updatedMember.memberEpoch());
|
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
|
||||||
assertEquals(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
|
||||||
), updatedMember.assignedPartitions());
|
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTransitionFromStableToStable() {
|
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setMemberEpoch(11)
|
.setMemberEpoch(11)
|
||||||
.setPreviousMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
.setTargetMemberEpoch(11)
|
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)))
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, member.state());
|
|
||||||
|
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
|
||||||
));
|
|
||||||
|
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(11, targetAssignment)
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.STABLE, updatedMember.state());
|
assertEquals(
|
||||||
assertEquals(11, updatedMember.previousMemberEpoch());
|
new ConsumerGroupMember.Builder("member")
|
||||||
assertEquals(11, updatedMember.memberEpoch());
|
.setState(MemberState.STABLE)
|
||||||
assertEquals(11, updatedMember.targetMemberEpoch());
|
.setMemberEpoch(12)
|
||||||
assertEquals(mkAssignment(
|
.setPreviousMemberEpoch(11)
|
||||||
mkTopicAssignment(topicId1, 3, 4, 5),
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 6, 7, 8)
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
), updatedMember.assignedPartitions());
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingRevocation());
|
.build(),
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.partitionsPendingAssignment());
|
updatedMember
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testNewTargetRestartReconciliation() {
|
public void testUnreleasedPartitionsToStableWithNewPartitions() {
|
||||||
Uuid topicId1 = Uuid.randomUuid();
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
Uuid topicId2 = Uuid.randomUuid();
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
.setMemberEpoch(10)
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
.setPreviousMemberEpoch(10)
|
.setMemberEpoch(11)
|
||||||
.setTargetMemberEpoch(11)
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.STABLE)
|
||||||
|
.setMemberEpoch(11)
|
||||||
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6, 7)))
|
||||||
|
.build(),
|
||||||
|
updatedMember
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnreleasedPartitionsToUnreleasedPartitions() {
|
||||||
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
|
.setMemberEpoch(11)
|
||||||
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withTargetAssignment(11, new Assignment(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3, 4),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6, 7))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
assertEquals(member, updatedMember);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnreleasedPartitionsToUnrevokedPartitions() {
|
||||||
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNRELEASED_PARTITIONS)
|
||||||
|
.setMemberEpoch(11)
|
||||||
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2, 3),
|
||||||
|
mkTopicAssignment(topicId2, 5, 6)))
|
||||||
|
.build();
|
||||||
|
|
||||||
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 3),
|
||||||
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNREVOKED_PARTITIONS)
|
||||||
|
.setMemberEpoch(11)
|
||||||
|
.setPreviousMemberEpoch(11)
|
||||||
|
.setAssignedPartitions(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 3),
|
||||||
|
mkTopicAssignment(topicId2, 6)))
|
||||||
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
|
mkTopicAssignment(topicId1, 2),
|
||||||
|
mkTopicAssignment(topicId2, 5)))
|
||||||
|
.build(),
|
||||||
|
updatedMember
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testUnknownState() {
|
||||||
|
Uuid topicId1 = Uuid.randomUuid();
|
||||||
|
Uuid topicId2 = Uuid.randomUuid();
|
||||||
|
|
||||||
|
ConsumerGroupMember member = new ConsumerGroupMember.Builder("member")
|
||||||
|
.setState(MemberState.UNKNOWN)
|
||||||
|
.setMemberEpoch(11)
|
||||||
|
.setPreviousMemberEpoch(11)
|
||||||
.setAssignedPartitions(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 6)))
|
mkTopicAssignment(topicId2, 6)))
|
||||||
.setPartitionsPendingRevocation(mkAssignment(
|
.setPartitionsPendingRevocation(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2),
|
mkTopicAssignment(topicId1, 2),
|
||||||
mkTopicAssignment(topicId2, 4, 5)))
|
mkTopicAssignment(topicId2, 5)))
|
||||||
.setPartitionsPendingAssignment(mkAssignment(
|
|
||||||
mkTopicAssignment(topicId1, 4, 5),
|
|
||||||
mkTopicAssignment(topicId2, 7, 8)))
|
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, member.state());
|
// When the member is in an unknown state, the member is first to force
|
||||||
|
// a reset of the client side member state.
|
||||||
Assignment targetAssignment = new Assignment(mkAssignment(
|
assertThrows(FencedMemberEpochException.class, () -> new CurrentAssignmentBuilder(member)
|
||||||
mkTopicAssignment(topicId1, 6, 7, 8),
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
mkTopicAssignment(topicId2, 9, 10, 11)
|
mkTopicAssignment(topicId1, 3),
|
||||||
));
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 10)
|
||||||
|
.build());
|
||||||
|
|
||||||
|
// Then the member rejoins with no owned partitions.
|
||||||
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
ConsumerGroupMember updatedMember = new CurrentAssignmentBuilder(member)
|
||||||
.withTargetAssignment(12, targetAssignment)
|
.withTargetAssignment(12, new Assignment(mkAssignment(
|
||||||
.withCurrentPartitionEpoch((topicId, partitionId) -> -1)
|
mkTopicAssignment(topicId1, 3),
|
||||||
|
mkTopicAssignment(topicId2, 6))))
|
||||||
|
.withCurrentPartitionEpoch((topicId, partitionId) -> 11)
|
||||||
|
.withOwnedTopicPartitions(Collections.emptyList())
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
assertEquals(ConsumerGroupMember.MemberState.REVOKING, updatedMember.state());
|
assertEquals(
|
||||||
assertEquals(10, updatedMember.previousMemberEpoch());
|
new ConsumerGroupMember.Builder("member")
|
||||||
assertEquals(10, updatedMember.memberEpoch());
|
.setState(MemberState.STABLE)
|
||||||
assertEquals(12, updatedMember.targetMemberEpoch());
|
.setMemberEpoch(12)
|
||||||
assertEquals(Collections.emptyMap(), updatedMember.assignedPartitions());
|
.setPreviousMemberEpoch(11)
|
||||||
assertEquals(mkAssignment(
|
.setAssignedPartitions(mkAssignment(
|
||||||
mkTopicAssignment(topicId1, 1, 2, 3),
|
mkTopicAssignment(topicId1, 3),
|
||||||
mkTopicAssignment(topicId2, 4, 5, 6)
|
mkTopicAssignment(topicId2, 6)))
|
||||||
), updatedMember.partitionsPendingRevocation());
|
.build(),
|
||||||
assertEquals(mkAssignment(
|
updatedMember
|
||||||
mkTopicAssignment(topicId1, 6, 7, 8),
|
);
|
||||||
mkTopicAssignment(topicId2, 9, 10, 11)
|
|
||||||
), updatedMember.partitionsPendingAssignment());
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<ConsumerGroupHeartbeatRequestData.TopicPartitions> requestFromAssignment(
|
|
||||||
Map<Uuid, Set<Integer>> assignment
|
|
||||||
) {
|
|
||||||
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> topicPartitions = new ArrayList<>();
|
|
||||||
|
|
||||||
assignment.forEach((topicId, partitions) -> {
|
|
||||||
ConsumerGroupHeartbeatRequestData.TopicPartitions topic = new ConsumerGroupHeartbeatRequestData.TopicPartitions()
|
|
||||||
.setTopicId(topicId)
|
|
||||||
.setPartitions(new ArrayList<>(partitions));
|
|
||||||
topicPartitions.add(topic);
|
|
||||||
});
|
|
||||||
|
|
||||||
return topicPartitions;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -214,7 +214,7 @@ public class GroupCoordinatorMetricsShardTest {
|
||||||
|
|
||||||
// Set member2 to ASSIGNING state.
|
// Set member2 to ASSIGNING state.
|
||||||
new ConsumerGroupMember.Builder(member2)
|
new ConsumerGroupMember.Builder(member2)
|
||||||
.setPartitionsPendingAssignment(Collections.singletonMap(Uuid.ZERO_UUID, Collections.singleton(0)))
|
.setPartitionsPendingRevocation(Collections.singletonMap(Uuid.ZERO_UUID, Collections.singleton(0)))
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
snapshotRegistry.getOrCreateSnapshot(4000);
|
snapshotRegistry.getOrCreateSnapshot(4000);
|
||||||
|
|
Loading…
Reference in New Issue