KAFKA-16040; Rename `Generic` to `Classic` (#15059)
People has raised concerned about using `Generic` as a name to designate the old rebalance protocol. We considered using `Legacy` but discarded it because there are still applications, such as Connect, using the old protocol. We settled on using `Classic` for the `Classic Rebalance Protocol`.
The changes in this patch are extremely mechanical. It basically replaces the occurrences of `generic` by `classic`.
Reviewers: Divij Vaidya <diviv@amazon.com>, Lucas Brutschy <lbrutschy@confluent.io>
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
index 213fa3e..512f8c4 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
@@ -109,10 +109,10 @@
* <code>group.protocol</code>
*/
public static final String GROUP_PROTOCOL_CONFIG = "group.protocol";
- public static final String DEFAULT_GROUP_PROTOCOL = GroupProtocol.GENERIC.name().toLowerCase(Locale.ROOT);
+ public static final String DEFAULT_GROUP_PROTOCOL = GroupProtocol.CLASSIC.name().toLowerCase(Locale.ROOT);
public static final String GROUP_PROTOCOL_DOC = "The group protocol consumer should use. We currently " +
- "support \"generic\" or \"consumer\". If \"consumer\" is specified, then the consumer group protocol will be " +
- "used. Otherwise, the generic group protocol will be used.";
+ "support \"classic\" or \"consumer\". If \"consumer\" is specified, then the consumer group protocol will be " +
+ "used. Otherwise, the classic group protocol will be used.";
/**
* <code>group.remote.assignor</code>
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/GroupProtocol.java b/clients/src/main/java/org/apache/kafka/clients/consumer/GroupProtocol.java
index 9c88ac3..9dd4674 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/GroupProtocol.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/GroupProtocol.java
@@ -19,8 +19,8 @@
import java.util.Locale;
public enum GroupProtocol {
- /** Generic group protocol. */
- GENERIC("GENERIC"),
+ /** Classic group protocol. */
+ CLASSIC("CLASSIC"),
/** Consumer group protocol */
CONSUMER("CONSUMER");
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java
index 563b437..bcc8cb4 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/LegacyKafkaConsumer.java
@@ -96,13 +96,13 @@
import static org.apache.kafka.common.utils.Utils.swallow;
/**
- * A client that consumes records from a Kafka cluster using the {@link GroupProtocol#GENERIC generic group protocol}.
+ * A client that consumes records from a Kafka cluster using the {@link GroupProtocol#CLASSIC classic group protocol}.
* In this implementation, all network I/O happens in the thread of the application making the call.
*
* <p/>
*
* <em>Note:</em> per its name, this implementation is left for backward compatibility purposes. The updated consumer
- * group protocol (from KIP-848) introduces allows users continue using the legacy "generic" group protocol.
+ * group protocol (from KIP-848) introduces allows users continue using the legacy "classic" group protocol.
* This class should not be invoked directly; users should instead create a {@link KafkaConsumer} as before.
*/
public class LegacyKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
diff --git a/clients/src/main/resources/common/message/OffsetCommitRequest.json b/clients/src/main/resources/common/message/OffsetCommitRequest.json
index 20484aa..5b3029a 100644
--- a/clients/src/main/resources/common/message/OffsetCommitRequest.json
+++ b/clients/src/main/resources/common/message/OffsetCommitRequest.json
@@ -41,7 +41,7 @@
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
"about": "The unique group identifier." },
{ "name": "GenerationIdOrMemberEpoch", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true,
- "about": "The generation of the group if using the generic group protocol or the member epoch if using the consumer protocol." },
+ "about": "The generation of the group if using the classic group protocol or the member epoch if using the consumer protocol." },
{ "name": "MemberId", "type": "string", "versions": "1+", "ignorable": true,
"about": "The member ID assigned by the group coordinator." },
{ "name": "GroupInstanceId", "type": "string", "versions": "7+",
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java
index 6aa8e09..8e9fa57 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java
@@ -173,7 +173,7 @@
configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass);
final ConsumerConfig consumerConfig = new ConsumerConfig(configs);
- assertEquals("generic", consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG));
+ assertEquals("classic", consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG));
assertNull(consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG));
}
@@ -192,7 +192,7 @@
}
@ParameterizedTest
- @CsvSource({"consumer, true", "generic, true", "Consumer, true", "Generic, true", "invalid, false"})
+ @CsvSource({"consumer, true", "classic, true", "Consumer, true", "Classic, true", "invalid, false"})
public void testProtocolConfigValidation(String protocol, boolean isValid) {
final Map<String, Object> configs = new HashMap<>();
configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
index 657d314..5595ed2 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
@@ -279,7 +279,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
@SuppressWarnings("unchecked")
public void testPollReturnsRecords(GroupProtocol groupProtocol) {
consumer = setUpConsumerWithRecordsToPoll(groupProtocol, tp0, 5);
@@ -294,7 +294,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
@SuppressWarnings("unchecked")
public void testSecondPollWithDeserializationErrorThrowsRecordDeserializationException(GroupProtocol groupProtocol) {
int invalidRecordNumber = 4;
@@ -674,7 +674,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void verifyHeartbeatSent(GroupProtocol groupProtocol) throws Exception {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -707,7 +707,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void verifyHeartbeatSentWhenFetchedDataReady(GroupProtocol groupProtocol) throws Exception {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -738,7 +738,7 @@
}
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void verifyPollTimesOutDuringMetadataUpdate(GroupProtocol groupProtocol) {
final ConsumerMetadata metadata = createMetadata(subscription);
final MockClient client = new MockClient(time, metadata);
@@ -764,7 +764,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
@SuppressWarnings("deprecation")
public void verifyDeprecatedPollDoesNotTimeOutDuringMetadataUpdate(GroupProtocol groupProtocol) {
final ConsumerMetadata metadata = createMetadata(subscription);
@@ -812,7 +812,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void verifyNoCoordinatorLookupForManualAssignmentWithOffsetCommit(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -851,7 +851,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupProtocol) {
// Verifies that we can make progress on one partition while we are awaiting
// a reset on another partition.
@@ -911,7 +911,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) {
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
ConsumerMetadata metadata = createMetadata(subscription);
@@ -935,7 +935,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testResetToCommittedOffset(GroupProtocol groupProtocol) {
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
ConsumerMetadata metadata = createMetadata(subscription);
@@ -960,7 +960,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testResetUsingAutoResetPolicy(GroupProtocol groupProtocol) {
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.LATEST);
ConsumerMetadata metadata = createMetadata(subscription);
@@ -1004,7 +1004,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCommitsFetchedDuringAssign(GroupProtocol groupProtocol) {
long offset1 = 10000;
long offset2 = 20000;
@@ -1043,7 +1043,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testFetchStableOffsetThrowInCommitted(GroupProtocol groupProtocol) {
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).committed(Collections.singleton(tp0)));
}
@@ -1051,7 +1051,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testFetchStableOffsetThrowInPoll(GroupProtocol groupProtocol) {
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).poll(Duration.ZERO));
}
@@ -1059,7 +1059,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testFetchStableOffsetThrowInPosition(GroupProtocol groupProtocol) {
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).position(tp0));
}
@@ -1090,7 +1090,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testNoCommittedOffsets(GroupProtocol groupProtocol) {
long offset1 = 10000;
@@ -1118,7 +1118,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testAutoCommitSentBeforePositionUpdate(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1152,7 +1152,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testRegexSubscription(GroupProtocol groupProtocol) {
String unmatchedTopic = "unmatched";
ConsumerMetadata metadata = createMetadata(subscription);
@@ -1181,7 +1181,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testChangingRegexSubscription(GroupProtocol groupProtocol) {
String otherTopic = "other";
TopicPartition otherTopicPartition = new TopicPartition(otherTopic, 0);
@@ -1218,7 +1218,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testWakeupWithFetchDataAvailable(GroupProtocol groupProtocol) throws Exception {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1259,7 +1259,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testPollThrowsInterruptExceptionIfInterrupted(GroupProtocol groupProtocol) {
final ConsumerMetadata metadata = createMetadata(subscription);
final MockClient client = new MockClient(time, metadata);
@@ -1285,7 +1285,7 @@
}
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void fetchResponseWithUnexpectedPartitionIsIgnored(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1320,7 +1320,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
@SuppressWarnings("unchecked")
public void testSubscriptionChangesWithAutoCommitEnabled(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
@@ -1436,7 +1436,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testSubscriptionChangesWithAutoCommitDisabled(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1493,7 +1493,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testUnsubscribeShouldTriggerPartitionsRevokedWithValidGeneration(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1519,7 +1519,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testUnsubscribeShouldTriggerPartitionsLostWithNoGeneration(GroupProtocol groupProtocol) throws Exception {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1557,7 +1557,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
@SuppressWarnings("unchecked")
public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
@@ -1615,7 +1615,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1673,7 +1673,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testOffsetOfPausedPartitions(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -1749,7 +1749,7 @@
// TODO: this test references RPCs to be sent that are not part of the CONSUMER group protocol.
// We are deferring any attempts at generalizing this test for both group protocols to the future.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testGracefulClose(GroupProtocol groupProtocol) throws Exception {
Map<TopicPartition, Errors> response = new HashMap<>();
response.put(tp0, Errors.NONE);
@@ -1762,7 +1762,7 @@
// TODO: this test references RPCs to be sent that are not part of the CONSUMER group protocol.
// We are deferring any attempts at generalizing this test for both group protocols to the future.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCloseTimeoutDueToNoResponseForCloseFetchRequest(GroupProtocol groupProtocol) throws Exception {
Map<TopicPartition, Errors> response = new HashMap<>();
response.put(tp0, Errors.NONE);
@@ -1781,7 +1781,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCloseTimeout(GroupProtocol groupProtocol) throws Exception {
consumerCloseTest(groupProtocol, 5000, Collections.emptyList(), 5000, false);
}
@@ -1789,7 +1789,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testLeaveGroupTimeout(GroupProtocol groupProtocol) throws Exception {
Map<TopicPartition, Errors> response = new HashMap<>();
response.put(tp0, Errors.NONE);
@@ -1800,7 +1800,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCloseNoWait(GroupProtocol groupProtocol) throws Exception {
consumerCloseTest(groupProtocol, 0, Collections.emptyList(), 0, false);
}
@@ -1808,7 +1808,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCloseInterrupt(GroupProtocol groupProtocol) throws Exception {
consumerCloseTest(groupProtocol, Long.MAX_VALUE, Collections.emptyList(), 0, true);
}
@@ -1816,7 +1816,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCloseShouldBeIdempotent(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = spy(new MockClient(time, metadata));
@@ -1918,7 +1918,7 @@
// TODO: this test references RPCs to be sent that are not part of the CONSUMER group protocol.
// We are deferring any attempts at generalizing this test for both group protocols to the future.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
@SuppressWarnings("unchecked")
public void testShouldAttemptToRejoinGroupAfterSyncGroupFailed(GroupProtocol groupProtocol) throws Exception {
ConsumerMetadata metadata = createMetadata(subscription);
@@ -2085,7 +2085,7 @@
// TODO: this test requires topic metadata logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testPartitionsForNonExistingTopic(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -2106,7 +2106,7 @@
// TODO: this test requires topic metadata logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testPartitionsForAuthenticationFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
assertThrows(AuthenticationException.class, () -> consumer.partitionsFor("some other topic"));
@@ -2115,7 +2115,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testBeginningOffsetsAuthenticationFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
assertThrows(AuthenticationException.class, () -> consumer.beginningOffsets(Collections.singleton(tp0)));
@@ -2124,7 +2124,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testEndOffsetsAuthenticationFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
assertThrows(AuthenticationException.class, () -> consumer.endOffsets(Collections.singleton(tp0)));
@@ -2133,7 +2133,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testPollAuthenticationFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
consumer.subscribe(singleton(topic));
@@ -2143,7 +2143,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testOffsetsForTimesAuthenticationFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
assertThrows(AuthenticationException.class, () -> consumer.offsetsForTimes(singletonMap(tp0, 0L)));
@@ -2152,7 +2152,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCommitSyncAuthenticationFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
@@ -2163,7 +2163,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testCommittedAuthenticationFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
assertThrows(AuthenticationException.class, () -> consumer.committed(Collections.singleton(tp0)).get(tp0));
@@ -2172,7 +2172,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testMeasureCommitSyncDurationOnFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer
= consumerWithPendingError(groupProtocol, new MockTime(Duration.ofSeconds(1).toMillis()));
@@ -2190,7 +2190,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testMeasureCommitSyncDuration(GroupProtocol groupProtocol) {
Time time = new MockTime(Duration.ofSeconds(1).toMillis());
SubscriptionState subscription = new SubscriptionState(new LogContext(),
@@ -2221,7 +2221,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testMeasureCommittedDurationOnFailure(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer
= consumerWithPendingError(groupProtocol, new MockTime(Duration.ofSeconds(1).toMillis()));
@@ -2239,7 +2239,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testMeasureCommittedDuration(GroupProtocol groupProtocol) {
long offset1 = 10000;
Time time = new MockTime(Duration.ofSeconds(1).toMillis());
@@ -2272,7 +2272,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testRebalanceException(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -2317,7 +2317,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testReturnRecordsDuringRebalance(GroupProtocol groupProtocol) throws InterruptedException {
Time time = new MockTime(1L);
ConsumerMetadata metadata = createMetadata(subscription);
@@ -2445,7 +2445,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testGetGroupMetadata(GroupProtocol groupProtocol) {
final ConsumerMetadata metadata = createMetadata(subscription);
final MockClient client = new MockClient(time, metadata);
@@ -2478,7 +2478,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testInvalidGroupMetadata(GroupProtocol groupProtocol) throws InterruptedException {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -2508,7 +2508,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
@SuppressWarnings("unchecked")
public void testCurrentLag(GroupProtocol groupProtocol) {
final ConsumerMetadata metadata = createMetadata(subscription);
@@ -2564,7 +2564,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testListOffsetShouldUpdateSubscriptions(GroupProtocol groupProtocol) {
final ConsumerMetadata metadata = createMetadata(subscription);
final MockClient client = new MockClient(time, metadata);
@@ -2998,7 +2998,7 @@
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
// Once it is implemented, this should use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) {
ConsumerMetadata metadata = createMetadata(subscription);
MockClient client = new MockClient(time, metadata);
@@ -3137,7 +3137,7 @@
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testEnforceRebalanceWithManualAssignment(GroupProtocol groupProtocol) {
consumer = newConsumer(groupProtocol, null);
consumer.assign(singleton(new TopicPartition("topic", 0)));
@@ -3146,7 +3146,7 @@
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testEnforceRebalanceTriggersRebalanceOnNextPoll(GroupProtocol groupProtocol) {
Time time = new MockTime(1L);
ConsumerMetadata metadata = createMetadata(subscription);
@@ -3177,7 +3177,7 @@
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testEnforceRebalanceReason(GroupProtocol groupProtocol) {
Time time = new MockTime(1L);
@@ -3241,7 +3241,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void configurableObjectsShouldSeeGeneratedClientId(GroupProtocol groupProtocol) {
Properties props = new Properties();
props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name());
@@ -3288,7 +3288,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testOffsetsForTimesTimeout(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
assertEquals(
@@ -3300,7 +3300,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testBeginningOffsetsTimeout(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
assertEquals(
@@ -3312,7 +3312,7 @@
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
// The bug will be investigated and fixed so this test can use both group protocols.
@ParameterizedTest
- @EnumSource(value = GroupProtocol.class, names = "GENERIC")
+ @EnumSource(value = GroupProtocol.class, names = "CLASSIC")
public void testEndOffsetsTimeout(GroupProtocol groupProtocol) {
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
assertEquals(
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java
index 05903d4..4a34fce 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java
@@ -1129,7 +1129,7 @@
public void testGroupRemoteAssignorUnusedInGenericProtocol() {
final Properties props = requiredConsumerProperties();
props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA");
- props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.GENERIC.name().toLowerCase(Locale.ROOT));
+ props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name().toLowerCase(Locale.ROOT));
props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor");
final ConsumerConfig config = new ConsumerConfig(props);
consumer = newConsumer(config);
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index 121162e..4686248 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -561,7 +561,7 @@
config.offsetMetadataMaxSize,
config.groupMaxSize,
config.groupInitialRebalanceDelay,
- GroupCoordinatorConfig.GENERIC_GROUP_NEW_MEMBER_JOIN_TIMEOUT_MS,
+ GroupCoordinatorConfig.CLASSIC_GROUP_NEW_MEMBER_JOIN_TIMEOUT_MS,
config.groupMinSessionTimeoutMs,
config.groupMaxSessionTimeoutMs,
config.offsetsRetentionCheckIntervalMs,
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index e3a694d..03d55c3 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -167,7 +167,7 @@
/** New group coordinator configs */
val NewGroupCoordinatorEnable = false
- val GroupCoordinatorRebalanceProtocols = List(GroupType.GENERIC.toString).asJava
+ val GroupCoordinatorRebalanceProtocols = List(GroupType.CLASSIC.toString).asJava
val GroupCoordinatorNumThreads = 1
/** Consumer group configs */
@@ -2013,8 +2013,8 @@
val groupCoordinatorRebalanceProtocols = {
val protocols = getList(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp)
.asScala.map(_.toUpperCase).map(GroupType.valueOf).toSet
- if (!protocols.contains(GroupType.GENERIC)) {
- throw new ConfigException(s"Disabling the '${GroupType.GENERIC}' protocol is not supported.")
+ if (!protocols.contains(GroupType.CLASSIC)) {
+ throw new ConfigException(s"Disabling the '${GroupType.CLASSIC}' protocol is not supported.")
}
if (protocols.contains(GroupType.CONSUMER)) {
warn(s"The new '${GroupType.CONSUMER}' rebalance protocol is enabled along with the new group coordinator. " +
diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
index 079d648..bb3259b 100644
--- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
@@ -112,15 +112,15 @@
object BaseConsumerTest {
// We want to test the following combinations:
- // * ZooKeeper and the generic group protocol
- // * KRaft and the generic group protocol
- // * KRaft with the new group coordinator enabled and the generic group protocol
+ // * ZooKeeper and the classic group protocol
+ // * KRaft and the classic group protocol
+ // * KRaft with the new group coordinator enabled and the classic group protocol
// * KRaft with the new group coordinator enabled and the consumer group protocol
def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = {
java.util.stream.Stream.of(
- Arguments.of("zk", "generic"),
- Arguments.of("kraft", "generic"),
- Arguments.of("kraft+kip848", "generic"),
+ Arguments.of("zk", "classic"),
+ Arguments.of("kraft", "classic"),
+ Arguments.of("kraft+kip848", "classic"),
Arguments.of("kraft+kip848", "consumer"))
}
@@ -130,18 +130,18 @@
// single combination are written using @CsvSource rather than the more elegant @MethodSource.
// def getTestQuorumAndGroupProtocolParametersZkOnly() : java.util.stream.Stream[Arguments] = {
// java.util.stream.Stream.of(
- // Arguments.of("zk", "generic"))
+ // Arguments.of("zk", "classic"))
// }
- // For tests that only work with the generic group protocol, we want to test the following combinations:
- // * ZooKeeper and the generic group protocol
- // * KRaft and the generic group protocol
- // * KRaft with the new group coordinator enabled and the generic group protocol
- def getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly() : java.util.stream.Stream[Arguments] = {
+ // For tests that only work with the classic group protocol, we want to test the following combinations:
+ // * ZooKeeper and the classic group protocol
+ // * KRaft and the classic group protocol
+ // * KRaft with the new group coordinator enabled and the classic group protocol
+ def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() : java.util.stream.Stream[Arguments] = {
java.util.stream.Stream.of(
- Arguments.of("zk", "generic"),
- Arguments.of("kraft", "generic"),
- Arguments.of("kraft+kip848", "generic"))
+ Arguments.of("zk", "classic"),
+ Arguments.of("kraft", "classic"),
+ Arguments.of("kraft+kip848", "classic"))
}
val updateProducerCount = new AtomicInteger()
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
index 01e9aec..707069c 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
@@ -134,7 +134,7 @@
// Deprecated poll(timeout) not supported for consumer group protocol
@deprecated("poll(Duration) is the replacement", since = "2.0")
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testDeprecatedPollBlocksForAssignment(quorum: String, groupProtocol: String): Unit = {
val consumer = createConsumer()
consumer.subscribe(Set(topic).asJava)
@@ -171,7 +171,7 @@
// TODO: enable this test for the consumer group protocol when KAFKA-16008 has been fixed.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMaxPollIntervalMs(quorum: String, groupProtocol: String): Unit = {
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 1000.toString)
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString)
@@ -198,7 +198,7 @@
// TODO: enable this test for the consumer group protocol when KAFKA-16009 has been fixed.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMaxPollIntervalMsDelayInRevocation(quorum: String, groupProtocol: String): Unit = {
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString)
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString)
@@ -290,7 +290,7 @@
// TODO: enable this test for the consumer group protocol when support for committing offsets on close is implemented.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testAutoCommitOnCloseAfterWakeup(quorum: String, groupProtocol: String): Unit = {
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
val consumer = createConsumer()
@@ -352,7 +352,7 @@
*/
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testPatternSubscription(quorum: String, groupProtocol: String): Unit = {
val numRecords = 10000
val producer = createProducer()
@@ -411,7 +411,7 @@
*/
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testSubsequentPatternSubscription(quorum: String, groupProtocol: String): Unit = {
this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "30000")
val consumer = createConsumer()
@@ -464,7 +464,7 @@
*/
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testPatternUnsubscription(quorum: String, groupProtocol: String): Unit = {
val numRecords = 10000
val producer = createProducer()
@@ -533,7 +533,7 @@
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testExpandingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = {
val otherTopic = "other"
val initialAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1))
@@ -548,7 +548,7 @@
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testShrinkingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = {
val otherTopic = "other"
createTopic(otherTopic, 2, brokerCount)
@@ -975,9 +975,9 @@
assertEquals(0, consumer.assignment().size)
}
- // Only the generic group protocol supports client-side assignors
+ // Only the classic group protocol supports client-side assignors
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testRoundRobinAssignment(quorum: String, groupProtocol: String): Unit = {
// 1 consumer using round-robin assignment
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
@@ -1013,9 +1013,9 @@
assertEquals(0, consumer.assignment().size)
}
- // Only the generic group protocol supports client-side assignors
+ // Only the classic group protocol supports client-side assignors
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMultiConsumerRoundRobinAssignor(quorum: String, groupProtocol: String): Unit = {
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName)
@@ -1052,9 +1052,9 @@
* - (#par / 10) partition per consumer, where one partition from each of the early (#par mod 9) consumers
* will move to consumer #10, leading to a total of (#par mod 9) partition movement
*/
- // Only the generic group protocol supports client-side assignors
+ // Only the classic group protocol supports client-side assignors
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMultiConsumerStickyAssignor(quorum: String, groupProtocol: String): Unit = {
def reverse(m: Map[Long, Set[TopicPartition]]) =
@@ -1100,9 +1100,9 @@
* This test re-uses BaseConsumerTest's consumers.
* As a result, it is testing the default assignment strategy set by BaseConsumerTest
*/
- // Only the generic group protocol supports client-side assignors
+ // Only the classic group protocol supports client-side assignors
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMultiConsumerDefaultAssignor(quorum: String, groupProtocol: String): Unit = {
// use consumers and topics defined in this class + one more topic
val producer = createProducer()
@@ -1137,7 +1137,7 @@
}
}
- // Only the generic group protocol supports client-side assignors
+ // Only the classic group protocol supports client-side assignors
@ParameterizedTest
@CsvSource(Array(
"org.apache.kafka.clients.consumer.CooperativeStickyAssignor, zk",
@@ -1147,7 +1147,7 @@
))
def testRebalanceAndRejoin(assignmentStrategy: String, quorum: String): Unit = {
// create 2 consumers
- this.consumerConfig.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, "generic")
+ this.consumerConfig.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, "classic")
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "rebalance-and-rejoin-group")
this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, assignmentStrategy)
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
@@ -1230,9 +1230,9 @@
* As a result, it is testing the default assignment strategy set by BaseConsumerTest
* It tests the assignment results is expected using default assignor (i.e. Range assignor)
*/
- // Only the generic group protocol supports client-side assignors
+ // Only the classic group protocol supports client-side assignors
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMultiConsumerDefaultAssignorAndVerifyAssignment(quorum: String, groupProtocol: String): Unit = {
// create two new topics, each having 3 partitions
val topic1 = "topic1"
@@ -1266,21 +1266,21 @@
// TODO: enable this test for the consumer group protocol when KAFKA-16010 has been fixed.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMultiConsumerSessionTimeoutOnStopPolling(quorum: String, groupProtocol: String): Unit = {
runMultiConsumerSessionTimeoutTest(false)
}
// TODO: enable this test for the consumer group protocol when KAFKA-16011 has been fixed.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testMultiConsumerSessionTimeoutOnClose(quorum: String, groupProtocol: String): Unit = {
runMultiConsumerSessionTimeoutTest(true)
}
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testInterceptors(quorum: String, groupProtocol: String): Unit = {
val appendStr = "mock"
MockConsumerInterceptor.resetCounters()
@@ -1341,7 +1341,7 @@
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testAutoCommitIntercept(quorum: String, groupProtocol: String): Unit = {
val topic2 = "topic2"
createTopic(topic2, 2, brokerCount)
@@ -1393,7 +1393,7 @@
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testInterceptorsWithWrongKeyValue(quorum: String, groupProtocol: String): Unit = {
val appendStr = "mock"
// create producer with interceptor that has different key and value types from the producer
@@ -1556,7 +1556,7 @@
// TODO: enable this test for the consumer group protocol when auto-commit support is implemented.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testAutoCommitOnRebalance(quorum: String, groupProtocol: String): Unit = {
val topic2 = "topic2"
createTopic(topic2, 2, brokerCount)
@@ -1596,7 +1596,7 @@
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testPerPartitionLeadMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = {
val numMessages = 1000
val topic2 = "topic2"
@@ -1636,7 +1636,7 @@
}
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testPerPartitionLagMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = {
val numMessages = 1000
val topic2 = "topic2"
@@ -2034,9 +2034,9 @@
assertThrows(classOf[InvalidGroupIdException], () => consumer1.commitSync())
}
- // Empty group ID only supported for generic group protocol
+ // Empty group ID only supported for classic group protocol
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testConsumingWithEmptyGroupId(quorum: String, groupProtocol: String): Unit = {
val topic = "test_topic"
val partition = 0
@@ -2097,7 +2097,7 @@
// TODO: enable this test for the consumer group protocol when static membership is implemented.
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
- @MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
+ @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
def testStaticConsumerDetectsNewPartitionCreatedAfterRestart(quorum:String, groupProtocol: String): Unit = {
val foo = "foo"
val foo0 = new TopicPartition(foo, 0)
@@ -2231,7 +2231,7 @@
// partitionsFor not implemented in consumer group protocol and this test requires ZK also
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@CsvSource(Array(
- "zk, generic"
+ "zk, classic"
))
def testAssignAndConsumeWithLeaderChangeValidatingPositions(quorum:String, groupProtocol: String): Unit = {
val numRecords = 10
diff --git a/core/src/test/scala/kafka/utils/TestInfoUtils.scala b/core/src/test/scala/kafka/utils/TestInfoUtils.scala
index c82a654..8e69bbf 100644
--- a/core/src/test/scala/kafka/utils/TestInfoUtils.scala
+++ b/core/src/test/scala/kafka/utils/TestInfoUtils.scala
@@ -61,8 +61,8 @@
}
def maybeGroupProtocolSpecified(testInfo: TestInfo): Option[GroupProtocol] = {
- if (testInfo.getDisplayName().contains("groupProtocol=generic"))
- Some(GroupProtocol.GENERIC)
+ if (testInfo.getDisplayName().contains("groupProtocol=classic"))
+ Some(GroupProtocol.CLASSIC)
else if (testInfo.getDisplayName().contains("groupProtocol=consumer"))
Some(GroupProtocol.CONSUMER)
else
diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala
index 4a33af5..250cc83 100644
--- a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala
@@ -50,7 +50,7 @@
}
@ClusterTest(clusterType = Type.KRAFT, serverProperties = Array(
- new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "generic,consumer"),
+ new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"),
new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"),
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1")
))
@@ -138,7 +138,7 @@
}
@ClusterTest(clusterType = Type.KRAFT, serverProperties = Array(
- new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "generic,consumer"),
+ new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"),
new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"),
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1")
))
@@ -252,7 +252,7 @@
}
@ClusterTest(clusterType = Type.KRAFT, serverProperties = Array(
- new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "generic,consumer"),
+ new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer"),
new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"),
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"),
new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "5000"),
diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala
index 27856a3..f4bff8c 100644
--- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala
@@ -21,7 +21,7 @@
import kafka.test.junit.ClusterTestExtensions
import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState
import org.junit.jupiter.api.Assertions.{assertEquals, fail}
import org.junit.jupiter.api.{Tag, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@@ -120,7 +120,7 @@
assertEquals(
List(new DescribedGroup()
.setGroupId("grp")
- .setGroupState(GenericGroupState.DEAD.toString)
+ .setGroupState(ClassicGroupState.DEAD.toString)
),
describeGroups(List("grp"))
)
diff --git a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala
index 6a1c9b4..8305fc4 100644
--- a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala
@@ -21,7 +21,7 @@
import kafka.test.junit.ClusterTestExtensions
import org.apache.kafka.common.message.DescribeGroupsResponseData.{DescribedGroup, DescribedGroupMember}
import org.apache.kafka.common.protocol.ApiKeys
-import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{Tag, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@@ -80,7 +80,7 @@
List(
new DescribedGroup()
.setGroupId("grp-1")
- .setGroupState(GenericGroupState.STABLE.toString)
+ .setGroupState(ClassicGroupState.STABLE.toString)
.setProtocolType("consumer")
.setProtocolData("consumer-range")
.setMembers(List(
@@ -94,7 +94,7 @@
).asJava),
new DescribedGroup()
.setGroupId("grp-2")
- .setGroupState(GenericGroupState.COMPLETING_REBALANCE.toString)
+ .setGroupState(ClassicGroupState.COMPLETING_REBALANCE.toString)
.setProtocolType("consumer")
.setMembers(List(
new DescribedGroupMember()
@@ -107,7 +107,7 @@
).asJava),
new DescribedGroup()
.setGroupId("grp-unknown")
- .setGroupState(GenericGroupState.DEAD.toString) // Return DEAD group when the group does not exist.
+ .setGroupState(ClassicGroupState.DEAD.toString) // Return DEAD group when the group does not exist.
),
describeGroups(
groupIds = List("grp-1", "grp-2", "grp-unknown"),
diff --git a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala
index 0d7ea8b..c3e9d6c 100644
--- a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala
@@ -24,7 +24,7 @@
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.common.message.SyncGroupRequestData
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState
import org.junit.jupiter.api.{Tag, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@@ -156,7 +156,7 @@
TestUtils.waitUntilTrue(() => {
val described = describeGroups(groupIds = List("grp"))
- GenericGroupState.PREPARING_REBALANCE.toString == described.head.groupState
+ ClassicGroupState.PREPARING_REBALANCE.toString == described.head.groupState
}, msg = s"The group is not in PREPARING_REBALANCE state.")
// Heartbeat PREPARING_REBALANCE group.
diff --git a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala
index 34e0431..21c0de1 100644
--- a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala
@@ -25,7 +25,7 @@
import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember
import org.apache.kafka.common.message.{JoinGroupResponseData, SyncGroupRequestData}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{Tag, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@@ -190,7 +190,7 @@
TestUtils.waitUntilTrue(() => {
val described = describeGroups(groupIds = List("grp"))
- GenericGroupState.PREPARING_REBALANCE.toString == described.head.groupState
+ ClassicGroupState.PREPARING_REBALANCE.toString == described.head.groupState
}, msg = s"The group is not in PREPARING_REBALANCE state.")
// The leader rejoins.
@@ -316,7 +316,7 @@
TestUtils.waitUntilTrue(() => {
val described = describeGroups(groupIds = List("grp"))
- GenericGroupState.PREPARING_REBALANCE.toString == described.head.groupState
+ ClassicGroupState.PREPARING_REBALANCE.toString == described.head.groupState
}, msg = s"The group is not in PREPARING_REBALANCE state.")
// A new follower with duplicated group instance id joins.
@@ -329,7 +329,7 @@
TestUtils.waitUntilTrue(() => {
val described = describeGroups(groupIds = List("grp"))
- GenericGroupState.COMPLETING_REBALANCE.toString == described.head.groupState
+ ClassicGroupState.COMPLETING_REBALANCE.toString == described.head.groupState
}, msg = s"The group is not in COMPLETING_REBALANCE state.")
// The old follower rejoin request should be fenced.
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index b1c3881..dd2419a 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -1816,18 +1816,18 @@
val props = new Properties()
props.putAll(kraftProps())
- // Only generic and consumer are supported.
+ // Only classic and consumer are supported.
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "foo")
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
- // generic cannot be disabled.
+ // classic cannot be disabled.
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "consumer")
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
// This is OK.
- props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "generic,consumer")
+ props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "classic,consumer")
val config = KafkaConfig.fromProps(props)
- assertEquals(Set(GroupType.GENERIC, GroupType.CONSUMER), config.groupCoordinatorRebalanceProtocols)
+ assertEquals(Set(GroupType.CLASSIC, GroupType.CONSUMER), config.groupCoordinatorRebalanceProtocols)
assertTrue(config.isNewGroupCoordinatorEnabled)
}
diff --git a/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala
index 3e04b26..1958c40 100644
--- a/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala
@@ -21,7 +21,7 @@
import kafka.test.junit.ClusterTestExtensions
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.JoinGroupRequest
-import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{Tag, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@@ -115,7 +115,7 @@
// grp-1 is empty.
assertEquals(
- GenericGroupState.EMPTY.toString,
+ ClassicGroupState.EMPTY.toString,
describeGroups(List("grp-1")).head.groupState
)
@@ -142,7 +142,7 @@
// grp-2 is empty.
assertEquals(
- GenericGroupState.EMPTY.toString,
+ ClassicGroupState.EMPTY.toString,
describeGroups(List("grp-2")).head.groupState
)
}
diff --git a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala
index eff2027..d7c15ff 100644
--- a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala
@@ -22,7 +22,7 @@
import org.apache.kafka.common.message.ListGroupsResponseData
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState
-import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState
import org.junit.jupiter.api.Assertions.{assertEquals, fail}
import org.junit.jupiter.api.{Tag, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@@ -81,14 +81,14 @@
val (memberId1InGroup1, _) = joinDynamicConsumerGroupWithOldProtocol(groupId = "grp-1")
val response1 = new ListGroupsResponseData.ListedGroup()
.setGroupId("grp-1")
- .setGroupState(if (version >= 4) GenericGroupState.STABLE.toString else "")
+ .setGroupState(if (version >= 4) ClassicGroupState.STABLE.toString else "")
.setProtocolType("consumer")
// Create grp-2 in old protocol without completing rebalance. Grp-2 is in COMPLETING_REBALANCE state.
val (memberId1InGroup2, _) = joinDynamicConsumerGroupWithOldProtocol(groupId = "grp-2", completeRebalance = false)
val response2 = new ListGroupsResponseData.ListedGroup()
.setGroupId("grp-2")
- .setGroupState(if (version >= 4) GenericGroupState.COMPLETING_REBALANCE.toString else "")
+ .setGroupState(if (version >= 4) ClassicGroupState.COMPLETING_REBALANCE.toString else "")
.setProtocolType("consumer")
// Create grp-3 in old protocol and complete a rebalance. Then memeber 1 leaves grp-3. Grp-3 is in EMPTY state.
@@ -96,7 +96,7 @@
leaveGroup(groupId = "grp-3", memberId = memberId1InGroup3, useNewProtocol = false, ApiKeys.LEAVE_GROUP.latestVersion(isUnstableApiEnabled))
val response3 = new ListGroupsResponseData.ListedGroup()
.setGroupId("grp-3")
- .setGroupState(if (version >= 4) GenericGroupState.EMPTY.toString else "")
+ .setGroupState(if (version >= 4) ClassicGroupState.EMPTY.toString else "")
.setProtocolType("consumer")
var memberId1InGroup4: String = null
@@ -157,7 +157,7 @@
if (useNewProtocol) List(response2, response5).toSet else List(response2).toSet,
listGroups(
statesFilter = List(
- GenericGroupState.COMPLETING_REBALANCE.toString,
+ ClassicGroupState.COMPLETING_REBALANCE.toString,
ConsumerGroupState.RECONCILING.toString,
),
version = version.toShort
@@ -168,8 +168,8 @@
if (useNewProtocol) List(response1, response3, response6).toSet else List(response1, response3).toSet,
listGroups(
statesFilter = List(
- GenericGroupState.STABLE.toString,
- GenericGroupState.EMPTY.toString,
+ ClassicGroupState.STABLE.toString,
+ ClassicGroupState.EMPTY.toString,
ConsumerGroupState.EMPTY.toString
),
version = version.toShort
diff --git a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala
index 5d13ce5..c1c528f 100644
--- a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala
@@ -24,7 +24,7 @@
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.common.message.SyncGroupRequestData
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.coordinator.group.generic.GenericGroupState
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState
import org.junit.jupiter.api.{Tag, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@@ -182,7 +182,7 @@
TestUtils.waitUntilTrue(() => {
val described = describeGroups(groupIds = List("grp"))
- GenericGroupState.PREPARING_REBALANCE.toString == described.head.groupState
+ ClassicGroupState.PREPARING_REBALANCE.toString == described.head.groupState
}, msg = s"The group is not in PREPARING_REBALANCE state.")
// The leader rejoins.
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java
index 9939b25..d0f9ad9 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Group.java
@@ -28,7 +28,7 @@
public interface Group {
enum GroupType {
CONSUMER("consumer"),
- GENERIC("generic");
+ CLASSIC("classic");
private final String name;
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java
index 33fef01..3a10e58 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java
@@ -73,7 +73,7 @@
);
/**
- * Join a Generic Group.
+ * Join a Classic Group.
*
* @param context The request context.
* @param request The JoinGroupRequest data.
@@ -89,7 +89,7 @@
);
/**
- * Sync a Generic Group.
+ * Sync a Classic Group.
*
* @param context The coordinator request context.
* @param request The SyncGroupRequest data.
@@ -105,7 +105,7 @@
);
/**
- * Heartbeat to a Generic Group.
+ * Heartbeat to a Classic Group.
*
* @param context The coordinator request context.
* @param request The HeartbeatRequest data.
@@ -119,7 +119,7 @@
);
/**
- * Leave a Generic Group.
+ * Leave a Classic Group.
*
* @param context The coordinator request context.
* @param request The LeaveGroupRequest data.
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java
index 9c67ac4..10fd969 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java
@@ -28,7 +28,7 @@
/**
* The timeout used to wait for a new member in milliseconds.
*/
- public static final int GENERIC_GROUP_NEW_MEMBER_JOIN_TIMEOUT_MS = 5 * 60 * 1000;
+ public static final int CLASSIC_GROUP_NEW_MEMBER_JOIN_TIMEOUT_MS = 5 * 60 * 1000;
/**
* The number of threads or event loops running.
@@ -67,29 +67,29 @@
public final int offsetMetadataMaxSize;
/**
- * The generic group maximum size.
+ * The classic group maximum size.
*/
- public final int genericGroupMaxSize;
+ public final int classicGroupMaxSize;
/**
- * The delay in milliseconds introduced for the first rebalance of a generic group.
+ * The delay in milliseconds introduced for the first rebalance of a classic group.
*/
- public final int genericGroupInitialRebalanceDelayMs;
+ public final int classicGroupInitialRebalanceDelayMs;
/**
* The timeout used to wait for a new member in milliseconds.
*/
- public final int genericGroupNewMemberJoinTimeoutMs;
+ public final int classicGroupNewMemberJoinTimeoutMs;
/**
- * The generic group minimum session timeout.
+ * The classic group minimum session timeout.
*/
- public final int genericGroupMinSessionTimeoutMs;
+ public final int classicGroupMinSessionTimeoutMs;
/**
- * The generic group maximum session timeout.
+ * The classic group maximum session timeout.
*/
- public final int genericGroupMaxSessionTimeoutMs;
+ public final int classicGroupMaxSessionTimeoutMs;
/**
* Frequency at which to check for expired offsets.
@@ -126,11 +126,11 @@
List<PartitionAssignor> consumerGroupAssignors,
int offsetsTopicSegmentBytes,
int offsetMetadataMaxSize,
- int genericGroupMaxSize,
- int genericGroupInitialRebalanceDelayMs,
- int genericGroupNewMemberJoinTimeoutMs,
- int genericGroupMinSessionTimeoutMs,
- int genericGroupMaxSessionTimeoutMs,
+ int classicGroupMaxSize,
+ int classicGroupInitialRebalanceDelayMs,
+ int classicGroupNewMemberJoinTimeoutMs,
+ int classicGroupMinSessionTimeoutMs,
+ int classicGroupMaxSessionTimeoutMs,
long offsetsRetentionCheckIntervalMs,
long offsetsRetentionMs,
int offsetCommitTimeoutMs
@@ -142,11 +142,11 @@
this.consumerGroupAssignors = consumerGroupAssignors;
this.offsetsTopicSegmentBytes = offsetsTopicSegmentBytes;
this.offsetMetadataMaxSize = offsetMetadataMaxSize;
- this.genericGroupMaxSize = genericGroupMaxSize;
- this.genericGroupInitialRebalanceDelayMs = genericGroupInitialRebalanceDelayMs;
- this.genericGroupNewMemberJoinTimeoutMs = genericGroupNewMemberJoinTimeoutMs;
- this.genericGroupMinSessionTimeoutMs = genericGroupMinSessionTimeoutMs;
- this.genericGroupMaxSessionTimeoutMs = genericGroupMaxSessionTimeoutMs;
+ this.classicGroupMaxSize = classicGroupMaxSize;
+ this.classicGroupInitialRebalanceDelayMs = classicGroupInitialRebalanceDelayMs;
+ this.classicGroupNewMemberJoinTimeoutMs = classicGroupNewMemberJoinTimeoutMs;
+ this.classicGroupMinSessionTimeoutMs = classicGroupMinSessionTimeoutMs;
+ this.classicGroupMaxSessionTimeoutMs = classicGroupMaxSessionTimeoutMs;
this.offsetsRetentionCheckIntervalMs = offsetsRetentionCheckIntervalMs;
this.offsetsRetentionMs = offsetsRetentionMs;
this.offsetCommitTimeoutMs = offsetCommitTimeoutMs;
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
index 3e007b7..2c95333 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
@@ -355,10 +355,10 @@
CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
runtime.scheduleWriteOperation(
- "generic-group-join",
+ "classic-group-join",
topicPartitionFor(request.groupId()),
Duration.ofMillis(config.offsetCommitTimeoutMs),
- coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+ coordinator -> coordinator.classicGroupJoin(context, request, responseFuture)
).exceptionally(exception -> {
if (!(exception instanceof KafkaException)) {
log.error("JoinGroup request {} hit an unexpected exception: {}",
@@ -399,10 +399,10 @@
CompletableFuture<SyncGroupResponseData> responseFuture = new CompletableFuture<>();
runtime.scheduleWriteOperation(
- "generic-group-sync",
+ "classic-group-sync",
topicPartitionFor(request.groupId()),
Duration.ofMillis(config.offsetCommitTimeoutMs),
- coordinator -> coordinator.genericGroupSync(context, request, responseFuture)
+ coordinator -> coordinator.classicGroupSync(context, request, responseFuture)
).exceptionally(exception -> {
if (!(exception instanceof KafkaException)) {
log.error("SyncGroup request {} hit an unexpected exception: {}",
@@ -441,9 +441,9 @@
// Using a read operation is okay here as we ignore the last committed offset in the snapshot registry.
// This means we will read whatever is in the latest snapshot, which is how the old coordinator behaves.
- return runtime.scheduleReadOperation("generic-group-heartbeat",
+ return runtime.scheduleReadOperation("classic-group-heartbeat",
topicPartitionFor(request.groupId()),
- (coordinator, __) -> coordinator.genericGroupHeartbeat(context, request)
+ (coordinator, __) -> coordinator.classicGroupHeartbeat(context, request)
).exceptionally(exception -> {
if (!(exception instanceof KafkaException)) {
log.error("Heartbeat request {} hit an unexpected exception: {}",
@@ -482,10 +482,10 @@
}
return runtime.scheduleWriteOperation(
- "generic-group-leave",
+ "classic-group-leave",
topicPartitionFor(request.groupId()),
Duration.ofMillis(config.offsetCommitTimeoutMs),
- coordinator -> coordinator.genericGroupLeave(context, request)
+ coordinator -> coordinator.classicGroupLeave(context, request)
).exceptionally(exception -> {
if (!(exception instanceof KafkaException)) {
log.error("LeaveGroup request {} hit an unexpected exception: {}",
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
index 66694af..9a7360a 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
@@ -83,7 +83,7 @@
/**
* The group coordinator shard is a replicated state machine that manages the metadata of all
- * generic and consumer groups. It holds the hard and the soft state of the groups. This class
+ * classic and consumer groups. It holds the hard and the soft state of the groups. This class
* has two kinds of methods:
* 1) The request handlers which handle the requests and generate a response and records to
* mutate the hard state. Those records will be written by the runtime and applied to the
@@ -182,11 +182,11 @@
.withConsumerGroupMaxSize(config.consumerGroupMaxSize)
.withConsumerGroupSessionTimeout(config.consumerGroupSessionTimeoutMs)
.withConsumerGroupHeartbeatInterval(config.consumerGroupHeartbeatIntervalMs)
- .withGenericGroupMaxSize(config.genericGroupMaxSize)
- .withGenericGroupInitialRebalanceDelayMs(config.genericGroupInitialRebalanceDelayMs)
- .withGenericGroupNewMemberJoinTimeoutMs(config.genericGroupNewMemberJoinTimeoutMs)
- .withGenericGroupMinSessionTimeoutMs(config.genericGroupMinSessionTimeoutMs)
- .withGenericGroupMaxSessionTimeoutMs(config.genericGroupMaxSessionTimeoutMs)
+ .withClassicGroupMaxSize(config.classicGroupMaxSize)
+ .withClassicGroupInitialRebalanceDelayMs(config.classicGroupInitialRebalanceDelayMs)
+ .withClassicGroupNewMemberJoinTimeoutMs(config.classicGroupNewMemberJoinTimeoutMs)
+ .withClassicGroupMinSessionTimeoutMs(config.classicGroupMinSessionTimeoutMs)
+ .withClassicGroupMaxSessionTimeoutMs(config.classicGroupMaxSessionTimeoutMs)
.withGroupCoordinatorMetricsShard(metricsShard)
.build();
@@ -313,12 +313,12 @@
* @return A Result containing the JoinGroup response and
* a list of records to update the state machine.
*/
- public CoordinatorResult<Void, Record> genericGroupJoin(
+ public CoordinatorResult<Void, Record> classicGroupJoin(
RequestContext context,
JoinGroupRequestData request,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
- return groupMetadataManager.genericGroupJoin(
+ return groupMetadataManager.classicGroupJoin(
context,
request,
responseFuture
@@ -334,12 +334,12 @@
* @return A Result containing the SyncGroup response and
* a list of records to update the state machine.
*/
- public CoordinatorResult<Void, Record> genericGroupSync(
+ public CoordinatorResult<Void, Record> classicGroupSync(
RequestContext context,
SyncGroupRequestData request,
CompletableFuture<SyncGroupResponseData> responseFuture
) {
- return groupMetadataManager.genericGroupSync(
+ return groupMetadataManager.classicGroupSync(
context,
request,
responseFuture
@@ -347,18 +347,18 @@
}
/**
- * Handles a generic group HeartbeatRequest.
+ * Handles a classic group HeartbeatRequest.
*
* @param context The request context.
* @param request The actual Heartbeat request.
*
* @return The HeartbeatResponse.
*/
- public HeartbeatResponseData genericGroupHeartbeat(
+ public HeartbeatResponseData classicGroupHeartbeat(
RequestContext context,
HeartbeatRequestData request
) {
- return groupMetadataManager.genericGroupHeartbeat(
+ return groupMetadataManager.classicGroupHeartbeat(
context,
request
);
@@ -529,11 +529,11 @@
* @return A Result containing the LeaveGroup response and
* a list of records to update the state machine.
*/
- public CoordinatorResult<LeaveGroupResponseData, Record> genericGroupLeave(
+ public CoordinatorResult<LeaveGroupResponseData, Record> classicGroupLeave(
RequestContext context,
LeaveGroupRequestData request
) throws ApiException {
- return groupMetadataManager.genericGroupLeave(context, request);
+ return groupMetadataManager.classicGroupLeave(context, request);
}
/**
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
index 323e44b..2d368ff 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
@@ -74,9 +74,9 @@
import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue;
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
-import org.apache.kafka.coordinator.group.generic.GenericGroup;
-import org.apache.kafka.coordinator.group.generic.GenericGroupMember;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroup;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
import org.apache.kafka.coordinator.group.runtime.CoordinatorTimer;
@@ -110,7 +110,7 @@
import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH;
import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID;
import static org.apache.kafka.coordinator.group.Group.GroupType.CONSUMER;
-import static org.apache.kafka.coordinator.group.Group.GroupType.GENERIC;
+import static org.apache.kafka.coordinator.group.Group.GroupType.CLASSIC;
import static org.apache.kafka.coordinator.group.RecordHelpers.newCurrentAssignmentRecord;
import static org.apache.kafka.coordinator.group.RecordHelpers.newCurrentAssignmentTombstoneRecord;
import static org.apache.kafka.coordinator.group.RecordHelpers.newGroupEpochRecord;
@@ -119,18 +119,18 @@
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.Utils.ofSentinel;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupMember.EMPTY_ASSIGNMENT;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
-import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupMember.EMPTY_ASSIGNMENT;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD;
+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.STABLE;
+import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CONSUMER_GROUP_REBALANCES_SENSOR_NAME;
-import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.GENERIC_GROUP_REBALANCES_SENSOR_NAME;
+import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_REBALANCES_SENSOR_NAME;
/**
- * The GroupMetadataManager manages the metadata of all generic and consumer groups. It holds
+ * The GroupMetadataManager manages the metadata of all classic and consumer groups. It holds
* the hard and the soft state of the groups. This class has two kinds of methods:
* 1) The request handlers which handle the requests and generate a response and records to
* mutate the hard state. Those records will be written by the runtime and applied to the
@@ -151,11 +151,11 @@
private int consumerGroupMetadataRefreshIntervalMs = Integer.MAX_VALUE;
private MetadataImage metadataImage = null;
private int consumerGroupSessionTimeoutMs = 45000;
- private int genericGroupMaxSize = Integer.MAX_VALUE;
- private int genericGroupInitialRebalanceDelayMs = 3000;
- private int genericGroupNewMemberJoinTimeoutMs = 5 * 60 * 1000;
- private int genericGroupMinSessionTimeoutMs;
- private int genericGroupMaxSessionTimeoutMs;
+ private int classicGroupMaxSize = Integer.MAX_VALUE;
+ private int classicGroupInitialRebalanceDelayMs = 3000;
+ private int classicGroupNewMemberJoinTimeoutMs = 5 * 60 * 1000;
+ private int classicGroupMinSessionTimeoutMs;
+ private int classicGroupMaxSessionTimeoutMs;
private GroupCoordinatorMetricsShard metrics;
Builder withLogContext(LogContext logContext) {
@@ -208,28 +208,28 @@
return this;
}
- Builder withGenericGroupMaxSize(int genericGroupMaxSize) {
- this.genericGroupMaxSize = genericGroupMaxSize;
+ Builder withClassicGroupMaxSize(int classicGroupMaxSize) {
+ this.classicGroupMaxSize = classicGroupMaxSize;
return this;
}
- Builder withGenericGroupInitialRebalanceDelayMs(int genericGroupInitialRebalanceDelayMs) {
- this.genericGroupInitialRebalanceDelayMs = genericGroupInitialRebalanceDelayMs;
+ Builder withClassicGroupInitialRebalanceDelayMs(int classicGroupInitialRebalanceDelayMs) {
+ this.classicGroupInitialRebalanceDelayMs = classicGroupInitialRebalanceDelayMs;
return this;
}
- Builder withGenericGroupNewMemberJoinTimeoutMs(int genericGroupNewMemberJoinTimeoutMs) {
- this.genericGroupNewMemberJoinTimeoutMs = genericGroupNewMemberJoinTimeoutMs;
+ Builder withClassicGroupNewMemberJoinTimeoutMs(int classicGroupNewMemberJoinTimeoutMs) {
+ this.classicGroupNewMemberJoinTimeoutMs = classicGroupNewMemberJoinTimeoutMs;
return this;
}
- Builder withGenericGroupMinSessionTimeoutMs(int genericGroupMinSessionTimeoutMs) {
- this.genericGroupMinSessionTimeoutMs = genericGroupMinSessionTimeoutMs;
+ Builder withClassicGroupMinSessionTimeoutMs(int classicGroupMinSessionTimeoutMs) {
+ this.classicGroupMinSessionTimeoutMs = classicGroupMinSessionTimeoutMs;
return this;
}
- Builder withGenericGroupMaxSessionTimeoutMs(int genericGroupMaxSessionTimeoutMs) {
- this.genericGroupMaxSessionTimeoutMs = genericGroupMaxSessionTimeoutMs;
+ Builder withClassicGroupMaxSessionTimeoutMs(int classicGroupMaxSessionTimeoutMs) {
+ this.classicGroupMaxSessionTimeoutMs = classicGroupMaxSessionTimeoutMs;
return this;
}
@@ -263,11 +263,11 @@
consumerGroupSessionTimeoutMs,
consumerGroupHeartbeatIntervalMs,
consumerGroupMetadataRefreshIntervalMs,
- genericGroupMaxSize,
- genericGroupInitialRebalanceDelayMs,
- genericGroupNewMemberJoinTimeoutMs,
- genericGroupMinSessionTimeoutMs,
- genericGroupMaxSessionTimeoutMs
+ classicGroupMaxSize,
+ classicGroupInitialRebalanceDelayMs,
+ classicGroupNewMemberJoinTimeoutMs,
+ classicGroupMinSessionTimeoutMs,
+ classicGroupMaxSessionTimeoutMs
);
}
}
@@ -312,7 +312,7 @@
private final PartitionAssignor defaultAssignor;
/**
- * The generic and consumer groups keyed by their name.
+ * The classic and consumer groups keyed by their name.
*/
private final TimelineHashMap<String, Group> groups;
@@ -356,29 +356,29 @@
new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
/**
- * The maximum number of members allowed in a single generic group.
+ * The maximum number of members allowed in a single classic group.
*/
- private final int genericGroupMaxSize;
+ private final int classicGroupMaxSize;
/**
- * Initial rebalance delay for members joining a generic group.
+ * Initial rebalance delay for members joining a classic group.
*/
- private final int genericGroupInitialRebalanceDelayMs;
+ private final int classicGroupInitialRebalanceDelayMs;
/**
* The timeout used to wait for a new member in milliseconds.
*/
- private final int genericGroupNewMemberJoinTimeoutMs;
+ private final int classicGroupNewMemberJoinTimeoutMs;
/**
* The group minimum session timeout.
*/
- private final int genericGroupMinSessionTimeoutMs;
+ private final int classicGroupMinSessionTimeoutMs;
/**
* The group maximum session timeout.
*/
- private final int genericGroupMaxSessionTimeoutMs;
+ private final int classicGroupMaxSessionTimeoutMs;
private GroupMetadataManager(
SnapshotRegistry snapshotRegistry,
@@ -392,11 +392,11 @@
int consumerGroupSessionTimeoutMs,
int consumerGroupHeartbeatIntervalMs,
int consumerGroupMetadataRefreshIntervalMs,
- int genericGroupMaxSize,
- int genericGroupInitialRebalanceDelayMs,
- int genericGroupNewMemberJoinTimeoutMs,
- int genericGroupMinSessionTimeoutMs,
- int genericGroupMaxSessionTimeoutMs
+ int classicGroupMaxSize,
+ int classicGroupInitialRebalanceDelayMs,
+ int classicGroupNewMemberJoinTimeoutMs,
+ int classicGroupMinSessionTimeoutMs,
+ int classicGroupMaxSessionTimeoutMs
) {
this.logContext = logContext;
this.log = logContext.logger(GroupMetadataManager.class);
@@ -413,11 +413,11 @@
this.consumerGroupSessionTimeoutMs = consumerGroupSessionTimeoutMs;
this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
this.consumerGroupMetadataRefreshIntervalMs = consumerGroupMetadataRefreshIntervalMs;
- this.genericGroupMaxSize = genericGroupMaxSize;
- this.genericGroupInitialRebalanceDelayMs = genericGroupInitialRebalanceDelayMs;
- this.genericGroupNewMemberJoinTimeoutMs = genericGroupNewMemberJoinTimeoutMs;
- this.genericGroupMinSessionTimeoutMs = genericGroupMinSessionTimeoutMs;
- this.genericGroupMaxSessionTimeoutMs = genericGroupMaxSessionTimeoutMs;
+ this.classicGroupMaxSize = classicGroupMaxSize;
+ this.classicGroupInitialRebalanceDelayMs = classicGroupInitialRebalanceDelayMs;
+ this.classicGroupNewMemberJoinTimeoutMs = classicGroupNewMemberJoinTimeoutMs;
+ this.classicGroupMinSessionTimeoutMs = classicGroupMinSessionTimeoutMs;
+ this.classicGroupMaxSessionTimeoutMs = classicGroupMaxSessionTimeoutMs;
}
/**
@@ -511,7 +511,7 @@
final List<DescribeGroupsResponseData.DescribedGroup> describedGroups = new ArrayList<>();
groupIds.forEach(groupId -> {
try {
- GenericGroup group = genericGroup(groupId, committedOffset);
+ ClassicGroup group = classicGroup(groupId, committedOffset);
if (group.isInState(STABLE)) {
if (!group.protocolName().isPresent()) {
@@ -589,66 +589,66 @@
}
/**
- * Gets or maybe creates a generic group.
+ * Gets or maybe creates a classic group.
*
* @param groupId The group id.
* @param createIfNotExists A boolean indicating whether the group should be
* created if it does not exist.
*
- * @return A GenericGroup.
+ * @return A ClassicGroup.
* @throws UnknownMemberIdException if the group does not exist and createIfNotExists is false.
- * @throws GroupIdNotFoundException if the group is not a generic group.
+ * @throws GroupIdNotFoundException if the group is not a classic group.
*
* Package private for testing.
*/
- GenericGroup getOrMaybeCreateGenericGroup(
+ ClassicGroup getOrMaybeCreateClassicGroup(
String groupId,
boolean createIfNotExists
) throws UnknownMemberIdException, GroupIdNotFoundException {
Group group = groups.get(groupId);
if (group == null && !createIfNotExists) {
- throw new UnknownMemberIdException(String.format("Generic group %s not found.", groupId));
+ throw new UnknownMemberIdException(String.format("Classic group %s not found.", groupId));
}
if (group == null) {
- GenericGroup genericGroup = new GenericGroup(logContext, groupId, GenericGroupState.EMPTY, time, metrics);
- groups.put(groupId, genericGroup);
- metrics.onGenericGroupStateTransition(null, genericGroup.currentState());
- return genericGroup;
+ ClassicGroup classicGroup = new ClassicGroup(logContext, groupId, ClassicGroupState.EMPTY, time, metrics);
+ groups.put(groupId, classicGroup);
+ metrics.onClassicGroupStateTransition(null, classicGroup.currentState());
+ return classicGroup;
} else {
- if (group.type() == GENERIC) {
- return (GenericGroup) group;
+ if (group.type() == CLASSIC) {
+ return (ClassicGroup) group;
} else {
// We don't support upgrading/downgrading between protocols at the moment so
// we throw an exception if a group exists with the wrong type.
- throw new GroupIdNotFoundException(String.format("Group %s is not a generic group.",
+ throw new GroupIdNotFoundException(String.format("Group %s is not a classic group.",
groupId));
}
}
}
/**
- * Gets a generic group by committed offset.
+ * Gets a classic group by committed offset.
*
* @param groupId The group id.
* @param committedOffset A specified committed offset corresponding to this shard.
*
- * @return A GenericGroup.
- * @throws GroupIdNotFoundException if the group does not exist or is not a generic group.
+ * @return A ClassicGroup.
+ * @throws GroupIdNotFoundException if the group does not exist or is not a classic group.
*/
- public GenericGroup genericGroup(
+ public ClassicGroup classicGroup(
String groupId,
long committedOffset
) throws GroupIdNotFoundException {
Group group = group(groupId, committedOffset);
- if (group.type() == GENERIC) {
- return (GenericGroup) group;
+ if (group.type() == CLASSIC) {
+ return (ClassicGroup) group;
} else {
// We don't support upgrading/downgrading between protocols at the moment so
// we throw an exception if a group exists with the wrong type.
- throw new GroupIdNotFoundException(String.format("Group %s is not a generic group.",
+ throw new GroupIdNotFoundException(String.format("Group %s is not a classic group.",
groupId));
}
}
@@ -693,9 +693,9 @@
ConsumerGroup consumerGroup = (ConsumerGroup) group;
metrics.onConsumerGroupStateTransition(consumerGroup.state(), null);
break;
- case GENERIC:
- GenericGroup genericGroup = (GenericGroup) group;
- metrics.onGenericGroupStateTransition(genericGroup.currentState(), null);
+ case CLASSIC:
+ ClassicGroup classicGroup = (ClassicGroup) group;
+ metrics.onClassicGroupStateTransition(classicGroup.currentState(), null);
break;
}
}
@@ -1774,18 +1774,18 @@
});
break;
- case GENERIC:
- GenericGroup genericGroup = (GenericGroup) group;
- log.info("Loaded generic group {} with {} members.", groupId, genericGroup.allMembers().size());
- genericGroup.allMembers().forEach(member -> {
- log.debug("Loaded member {} in generic group {}.", member.memberId(), groupId);
- rescheduleGenericGroupMemberHeartbeat(genericGroup, member);
+ case CLASSIC:
+ ClassicGroup classicGroup = (ClassicGroup) group;
+ log.info("Loaded classic group {} with {} members.", groupId, classicGroup.allMembers().size());
+ classicGroup.allMembers().forEach(member -> {
+ log.debug("Loaded member {} in classic group {}.", member.memberId(), groupId);
+ rescheduleClassicGroupMemberHeartbeat(classicGroup, member);
});
- if (genericGroup.size() > genericGroupMaxSize) {
+ if (classicGroup.size() > classicGroupMaxSize) {
// In case the max size config has changed.
- prepareRebalance(genericGroup, "Freshly-loaded group " + groupId +
- " (size " + genericGroup.size() + ") is over capacity " + genericGroupMaxSize +
+ prepareRebalance(classicGroup, "Freshly-loaded group " + groupId +
+ " (size " + classicGroup.size() + ") is over capacity " + classicGroupMaxSize +
". Rebalancing in order to give a chance for consumers to commit offsets");
}
break;
@@ -1803,7 +1803,7 @@
/**
* Replays GroupMetadataKey/Value to update the soft state of
- * the generic group.
+ * the classic group.
*
* @param key A GroupMetadataKey key.
* @param value A GroupMetadataValue record.
@@ -1818,7 +1818,7 @@
// Tombstone. Group should be removed.
removeGroup(groupId);
} else {
- List<GenericGroupMember> loadedMembers = new ArrayList<>();
+ List<ClassicGroupMember> loadedMembers = new ArrayList<>();
for (GroupMetadataValue.MemberMetadata member : value.members()) {
int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
member.sessionTimeout() : member.rebalanceTimeout();
@@ -1828,7 +1828,7 @@
.setName(value.protocol())
.setMetadata(member.subscription()));
- GenericGroupMember loadedMember = new GenericGroupMember(
+ ClassicGroupMember loadedMember = new ClassicGroupMember(
member.memberId(),
Optional.ofNullable(member.groupInstanceId()),
member.clientId(),
@@ -1845,7 +1845,7 @@
String protocolType = value.protocolType();
- GenericGroup genericGroup = new GenericGroup(
+ ClassicGroup classicGroup = new ClassicGroup(
this.logContext,
groupId,
loadedMembers.isEmpty() ? EMPTY : STABLE,
@@ -1858,14 +1858,14 @@
value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
);
- loadedMembers.forEach(member -> genericGroup.add(member, null));
- Group prevGroup = groups.put(groupId, genericGroup);
+ loadedMembers.forEach(member -> classicGroup.add(member, null));
+ Group prevGroup = groups.put(groupId, classicGroup);
if (prevGroup == null) {
- metrics.onGenericGroupStateTransition(null, genericGroup.currentState());
+ metrics.onClassicGroupStateTransition(null, classicGroup.currentState());
}
- genericGroup.setSubscribedTopics(
- genericGroup.computeSubscribedTopics()
+ classicGroup.setSubscribedTopics(
+ classicGroup.computeSubscribedTopics()
);
}
}
@@ -1879,7 +1879,7 @@
*
* @return The result that contains records to append if the join group phase completes.
*/
- public CoordinatorResult<Void, Record> genericGroupJoin(
+ public CoordinatorResult<Void, Record> classicGroupJoin(
RequestContext context,
JoinGroupRequestData request,
CompletableFuture<JoinGroupResponseData> responseFuture
@@ -1890,8 +1890,8 @@
String memberId = request.memberId();
int sessionTimeoutMs = request.sessionTimeoutMs();
- if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
- sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+ if (sessionTimeoutMs < classicGroupMinSessionTimeoutMs ||
+ sessionTimeoutMs > classicGroupMaxSessionTimeoutMs
) {
responseFuture.complete(new JoinGroupResponseData()
.setMemberId(memberId)
@@ -1901,10 +1901,10 @@
boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
// Group is created if it does not exist and the member id is UNKNOWN. if member
// is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
- GenericGroup group;
+ ClassicGroup group;
boolean isNewGroup = !groups.containsKey(groupId);
try {
- group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+ group = getOrMaybeCreateClassicGroup(groupId, isUnknownMember);
} catch (Throwable t) {
responseFuture.complete(new JoinGroupResponseData()
.setMemberId(memberId)
@@ -1920,14 +1920,14 @@
.setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
);
} else if (isUnknownMember) {
- result = genericGroupJoinNewMember(
+ result = classicGroupJoinNewMember(
context,
request,
group,
responseFuture
);
} else {
- result = genericGroupJoinExistingMember(
+ result = classicGroupJoinExistingMember(
context,
request,
group,
@@ -1968,19 +1968,19 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+ private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(ClassicGroup group) {
if (group.isInState(PREPARING_REBALANCE) &&
group.hasAllMembersJoined() &&
group.previousState() != EMPTY
) {
- return completeGenericGroupJoin(group);
+ return completeClassicGroupJoin(group);
}
return EMPTY_RESULT;
}
/**
- * Handle a new member generic group join.
+ * Handle a new member classic group join.
*
* @param context The request context.
* @param request The join group request.
@@ -1989,10 +1989,10 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+ private CoordinatorResult<Void, Record> classicGroupJoinNewMember(
RequestContext context,
JoinGroupRequestData request,
- GenericGroup group,
+ ClassicGroup group,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
if (group.isInState(DEAD)) {
@@ -2014,7 +2014,7 @@
String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
if (groupInstanceId.isPresent()) {
- return genericGroupJoinNewStaticMember(
+ return classicGroupJoinNewStaticMember(
context,
request,
group,
@@ -2022,7 +2022,7 @@
responseFuture
);
} else {
- return genericGroupJoinNewDynamicMember(
+ return classicGroupJoinNewDynamicMember(
context,
request,
group,
@@ -2047,10 +2047,10 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+ private CoordinatorResult<Void, Record> classicGroupJoinNewStaticMember(
RequestContext context,
JoinGroupRequestData request,
- GenericGroup group,
+ ClassicGroup group,
String newMemberId,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
@@ -2093,10 +2093,10 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+ private CoordinatorResult<Void, Record> classicGroupJoinNewDynamicMember(
RequestContext context,
JoinGroupRequestData request,
- GenericGroup group,
+ ClassicGroup group,
String newMemberId,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
@@ -2108,14 +2108,14 @@
group.groupId(), group.currentState(), newMemberId);
group.addPendingMember(newMemberId);
- String genericGroupHeartbeatKey = genericGroupHeartbeatKey(group.groupId(), newMemberId);
+ String classicGroupHeartbeatKey = classicGroupHeartbeatKey(group.groupId(), newMemberId);
timer.schedule(
- genericGroupHeartbeatKey,
+ classicGroupHeartbeatKey,
request.sessionTimeoutMs(),
TimeUnit.MILLISECONDS,
false,
- () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+ () -> expireClassicGroupMemberHeartbeat(group, newMemberId)
);
responseFuture.complete(new JoinGroupResponseData()
@@ -2143,10 +2143,10 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+ private CoordinatorResult<Void, Record> classicGroupJoinExistingMember(
RequestContext context,
JoinGroupRequestData request,
- GenericGroup group,
+ ClassicGroup group,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
String memberId = request.memberId();
@@ -2200,7 +2200,7 @@
return EMPTY_RESULT;
}
- GenericGroupMember member = group.member(memberId);
+ ClassicGroupMember member = group.member(memberId);
if (group.isInState(PREPARING_REBALANCE)) {
return updateMemberThenRebalanceOrCompleteJoin(
request,
@@ -2217,7 +2217,7 @@
// for the current generation.
responseFuture.complete(new JoinGroupResponseData()
.setMembers(group.isLeader(memberId) ?
- group.currentGenericGroupMembers() : Collections.emptyList())
+ group.currentClassicGroupMembers() : Collections.emptyList())
.setMemberId(memberId)
.setGenerationId(group.generationId())
.setProtocolName(group.protocolName().orElse(null))
@@ -2299,13 +2299,13 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> completeGenericGroupJoin(
- GenericGroup group
+ private CoordinatorResult<Void, Record> completeClassicGroupJoin(
+ ClassicGroup group
) {
- timer.cancel(genericGroupJoinKey(group.groupId()));
+ timer.cancel(classicGroupJoinKey(group.groupId()));
String groupId = group.groupId();
- Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+ Map<String, ClassicGroupMember> notYetRejoinedDynamicMembers =
group.notYetRejoinedMembers().entrySet().stream()
.filter(entry -> !entry.getValue().isStaticMember())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
@@ -2313,7 +2313,7 @@
if (!notYetRejoinedDynamicMembers.isEmpty()) {
notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
group.remove(failedMember.memberId());
- timer.cancel(genericGroupHeartbeatKey(group.groupId(), failedMember.memberId()));
+ timer.cancel(classicGroupHeartbeatKey(group.groupId(), failedMember.memberId()));
});
log.info("Group {} removed dynamic members who haven't joined: {}",
@@ -2329,11 +2329,11 @@
log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
timer.schedule(
- genericGroupJoinKey(groupId),
+ classicGroupJoinKey(groupId),
group.rebalanceTimeoutMs(),
TimeUnit.MILLISECONDS,
false,
- () -> completeGenericGroupJoin(group)
+ () -> completeClassicGroupJoin(group)
);
return EMPTY_RESULT;
@@ -2366,7 +2366,7 @@
group.allMembers().forEach(member -> {
List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
if (group.isLeader(member.memberId())) {
- members = group.currentGenericGroupMembers();
+ members = group.currentClassicGroupMembers();
}
JoinGroupResponseData response = new JoinGroupResponseData()
@@ -2380,7 +2380,7 @@
.setErrorCode(Errors.NONE.code());
group.completeJoinFuture(member, response);
- rescheduleGenericGroupMemberHeartbeat(group, member);
+ rescheduleClassicGroupMemberHeartbeat(group, member);
member.setIsNew(false);
group.addPendingSyncMember(member.memberId());
@@ -2398,8 +2398,9 @@
*
* @param group The group.
*/
- private void schedulePendingSync(GenericGroup group) {
- timer.schedule(genericGroupSyncKey(group.groupId()),
+ private void schedulePendingSync(ClassicGroup group) {
+ timer.schedule(
+ classicGroupSyncKey(group.groupId()),
group.rebalanceTimeoutMs(),
TimeUnit.MILLISECONDS,
false,
@@ -2415,8 +2416,8 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
- GenericGroup group,
+ private CoordinatorResult<Void, Record> expireClassicGroupMemberHeartbeat(
+ ClassicGroup group,
String memberId
) {
if (group.isInState(DEAD)) {
@@ -2427,16 +2428,16 @@
log.info("Pending member {} in group {} has been removed after session timeout expiration.",
memberId, group.groupId());
- return removePendingMemberAndUpdateGenericGroup(group, memberId);
+ return removePendingMemberAndUpdateClassicGroup(group, memberId);
} else if (!group.hasMemberId(memberId)) {
log.debug("Member {} has already been removed from the group.", memberId);
} else {
- GenericGroupMember member = group.member(memberId);
+ ClassicGroupMember member = group.member(memberId);
if (!member.hasSatisfiedHeartbeat()) {
log.info("Member {} in group {} has failed, removing it from the group.",
member.memberId(), group.groupId());
- return removeMemberAndUpdateGenericGroup(
+ return removeMemberAndUpdateClassicGroup(
group,
member,
"removing member " + member.memberId() + " on heartbeat expiration."
@@ -2456,9 +2457,9 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
- GenericGroup group,
- GenericGroupMember member,
+ private CoordinatorResult<Void, Record> removeMemberAndUpdateClassicGroup(
+ ClassicGroup group,
+ ClassicGroupMember member,
String reason
) {
// New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
@@ -2473,7 +2474,7 @@
if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
return maybePrepareRebalanceOrCompleteJoin(group, reason);
} else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
- return completeGenericGroupJoin(group);
+ return completeClassicGroupJoin(group);
}
return EMPTY_RESULT;
@@ -2487,14 +2488,14 @@
*
* @return The coordinator result that will be appended to the log.
*/
- private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
- GenericGroup group,
+ private CoordinatorResult<Void, Record> removePendingMemberAndUpdateClassicGroup(
+ ClassicGroup group,
String memberId
) {
group.remove(memberId);
if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
- return completeGenericGroupJoin(group);
+ return completeClassicGroupJoin(group);
}
return EMPTY_RESULT;
@@ -2513,8 +2514,8 @@
*/
private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
JoinGroupRequestData request,
- GenericGroup group,
- GenericGroupMember member,
+ ClassicGroup group,
+ ClassicGroupMember member,
String joinReason,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
@@ -2543,12 +2544,12 @@
private CoordinatorResult<Void, Record> addMemberThenRebalanceOrCompleteJoin(
RequestContext context,
JoinGroupRequestData request,
- GenericGroup group,
+ ClassicGroup group,
String memberId,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
groupInstanceId,
context.clientId(),
@@ -2574,7 +2575,7 @@
// timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
// members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
// for new members. If the new member is still there, we expect it to retry.
- rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+ rescheduleClassicGroupMemberHeartbeat(group, member, classicGroupNewMemberJoinTimeoutMs);
return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
request.groupInstanceId() + "; client reason: " + JoinGroupRequest.joinReason(request));
@@ -2590,7 +2591,7 @@
* @return The coordinator result that will be appended to the log.
*/
private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
- GenericGroup group,
+ ClassicGroup group,
String reason
) {
if (group.canRebalance()) {
@@ -2611,7 +2612,7 @@
* Package private for testing.
*/
CoordinatorResult<Void, Record> prepareRebalance(
- GenericGroup group,
+ ClassicGroup group,
String reason
) {
// If any members are awaiting sync, cancel their request and have them rejoin.
@@ -2625,11 +2626,11 @@
boolean isInitialRebalance = group.isInState(EMPTY);
if (isInitialRebalance) {
// The group is new. Provide more time for the members to join.
- int delayMs = genericGroupInitialRebalanceDelayMs;
- int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+ int delayMs = classicGroupInitialRebalanceDelayMs;
+ int remainingMs = Math.max(group.rebalanceTimeoutMs() - classicGroupInitialRebalanceDelayMs, 0);
timer.schedule(
- genericGroupJoinKey(group.groupId()),
+ classicGroupJoinKey(group.groupId()),
delayMs,
TimeUnit.MILLISECONDS,
false,
@@ -2638,7 +2639,7 @@
}
group.transitionTo(PREPARING_REBALANCE);
- metrics.record(GENERIC_GROUP_REBALANCES_SENSOR_NAME);
+ metrics.record(CLASSIC_GROUP_REBALANCES_SENSOR_NAME);
log.info("Preparing to rebalance group {} in state {} with old generation {} (reason: {}).",
group.groupId(), group.currentState(), group.generationId(), reason);
@@ -2654,19 +2655,19 @@
* @return The coordinator result that will be appended to the log.
*/
private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
- GenericGroup group
+ ClassicGroup group
) {
- String genericGroupJoinKey = genericGroupJoinKey(group.groupId());
+ String classicGroupJoinKey = classicGroupJoinKey(group.groupId());
if (group.hasAllMembersJoined()) {
// All members have joined. Proceed to sync phase.
- return completeGenericGroupJoin(group);
+ return completeClassicGroupJoin(group);
} else {
timer.schedule(
- genericGroupJoinKey,
+ classicGroupJoinKey,
group.rebalanceTimeoutMs(),
TimeUnit.MILLISECONDS,
false,
- () -> completeGenericGroupJoin(group)
+ () -> completeClassicGroupJoin(group)
);
return EMPTY_RESULT;
}
@@ -2681,18 +2682,18 @@
* @return The coordinator result that will be appended to the log.
*/
private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
- GenericGroup group,
+ ClassicGroup group,
int delayMs,
int remainingMs
) {
if (group.newMemberAdded() && remainingMs != 0) {
// A new member was added. Extend the delay.
group.setNewMemberAdded(false);
- int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+ int newDelayMs = Math.min(classicGroupInitialRebalanceDelayMs, remainingMs);
int newRemainingMs = Math.max(remainingMs - delayMs, 0);
timer.schedule(
- genericGroupJoinKey(group.groupId()),
+ classicGroupJoinKey(group.groupId()),
newDelayMs,
TimeUnit.MILLISECONDS,
false,
@@ -2700,7 +2701,7 @@
);
} else {
// No more time remaining. Complete the join phase.
- return completeGenericGroupJoin(group);
+ return completeClassicGroupJoin(group);
}
return EMPTY_RESULT;
@@ -2712,7 +2713,7 @@
* @param group The group.
* @param error The error to propagate.
*/
- private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+ private void resetAndPropagateAssignmentWithError(ClassicGroup group, Errors error) {
if (!group.isInState(COMPLETING_REBALANCE)) {
throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
" state but is in " + group.currentState() + ".");
@@ -2728,7 +2729,7 @@
* @param group The group.
* @param assignment The assignment for all members.
*/
- private void setAndPropagateAssignment(GenericGroup group, Map<String, byte[]> assignment) {
+ private void setAndPropagateAssignment(ClassicGroup group, Map<String, byte[]> assignment) {
if (!group.isInState(COMPLETING_REBALANCE)) {
throw new IllegalStateException("The group must be in CompletingRebalance state " +
"to set and propagate assignment.");
@@ -2746,7 +2747,7 @@
* @param group The group.
* @param error The error to propagate.
*/
- private void propagateAssignment(GenericGroup group, Errors error) {
+ private void propagateAssignment(ClassicGroup group, Errors error) {
Optional<String> protocolName = Optional.empty();
Optional<String> protocolType = Optional.empty();
if (error == Errors.NONE) {
@@ -2754,7 +2755,7 @@
protocolType = group.protocolType();
}
- for (GenericGroupMember member : group.allMembers()) {
+ for (ClassicGroupMember member : group.allMembers()) {
if (!member.hasAssignment() && error == Errors.NONE) {
log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
member.memberId(), group.groupId(), group.generationId());
@@ -2771,7 +2772,7 @@
// This is because if any member's session expired while we were still awaiting either
// the leader sync group or the append future, its expiration will be ignored and no
// future heartbeat expectations will not be scheduled.
- rescheduleGenericGroupMemberHeartbeat(group, member);
+ rescheduleClassicGroupMemberHeartbeat(group, member);
}
}
}
@@ -2782,11 +2783,11 @@
* @param group The group.
* @param member The member.
*/
- public void rescheduleGenericGroupMemberHeartbeat(
- GenericGroup group,
- GenericGroupMember member
+ public void rescheduleClassicGroupMemberHeartbeat(
+ ClassicGroup group,
+ ClassicGroupMember member
) {
- rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+ rescheduleClassicGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
}
/**
@@ -2796,19 +2797,19 @@
* @param member The member.
* @param timeoutMs The timeout for the new heartbeat.
*/
- private void rescheduleGenericGroupMemberHeartbeat(
- GenericGroup group,
- GenericGroupMember member,
+ private void rescheduleClassicGroupMemberHeartbeat(
+ ClassicGroup group,
+ ClassicGroupMember member,
long timeoutMs
) {
- String genericGroupHeartbeatKey = genericGroupHeartbeatKey(group.groupId(), member.memberId());
+ String classicGroupHeartbeatKey = classicGroupHeartbeatKey(group.groupId(), member.memberId());
// Reschedule the next heartbeat expiration deadline
- timer.schedule(genericGroupHeartbeatKey,
+ timer.schedule(classicGroupHeartbeatKey,
timeoutMs,
TimeUnit.MILLISECONDS,
false,
- () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+ () -> expireClassicGroupMemberHeartbeat(group, member.memberId()));
}
/**
@@ -2817,9 +2818,9 @@
*
* @param group The group.
*/
- private void removeSyncExpiration(GenericGroup group) {
+ private void removeSyncExpiration(ClassicGroup group) {
group.clearPendingSyncMembers();
- timer.cancel(genericGroupSyncKey(group.groupId()));
+ timer.cancel(classicGroupSyncKey(group.groupId()));
}
/**
@@ -2831,7 +2832,7 @@
* @return The coordinator result that will be appended to the log.
* */
private CoordinatorResult<Void, Record> expirePendingSync(
- GenericGroup group,
+ ClassicGroup group,
int generationId
) {
if (generationId != group.generationId()) {
@@ -2846,7 +2847,7 @@
Set<String> pendingSyncMembers = new HashSet<>(group.allPendingSyncMembers());
pendingSyncMembers.forEach(memberId -> {
group.remove(memberId);
- timer.cancel(genericGroupHeartbeatKey(group.groupId(), memberId));
+ timer.cancel(classicGroupHeartbeatKey(group.groupId(), memberId));
});
log.debug("Group {} removed members who haven't sent their sync requests: {}",
@@ -2868,7 +2869,7 @@
*
* @return whether the group can accept a joining member.
*/
- private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+ private boolean acceptJoiningMember(ClassicGroup group, String memberId) {
switch (group.currentState()) {
case EMPTY:
case DEAD:
@@ -2883,13 +2884,13 @@
// 2) using the number of awaiting members allows to kick out the last rejoining
// members of the group.
return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
- group.numAwaitingJoinResponse() < genericGroupMaxSize;
+ group.numAwaitingJoinResponse() < classicGroupMaxSize;
case COMPLETING_REBALANCE:
case STABLE:
// An existing member is accepted. New members are accepted up to the max group size.
// Note that the group size is used here. When the group transitions to CompletingRebalance,
// members who haven't rejoined are removed.
- return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+ return group.hasMemberId(memberId) || group.size() < classicGroupMaxSize;
default:
throw new IllegalStateException("Unknown group state: " + group.stateAsString());
}
@@ -2910,17 +2911,17 @@
private CoordinatorResult<Void, Record> updateStaticMemberThenRebalanceOrCompleteJoin(
RequestContext context,
JoinGroupRequestData request,
- GenericGroup group,
+ ClassicGroup group,
String oldMemberId,
String newMemberId,
CompletableFuture<JoinGroupResponseData> responseFuture
) {
String currentLeader = group.leaderOrNull();
- GenericGroupMember newMember = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+ ClassicGroupMember newMember = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
// Heartbeat of old member id will expire without effect since the group no longer contains that member id.
// New heartbeat shall be scheduled with new member id.
- rescheduleGenericGroupMemberHeartbeat(group, newMember);
+ rescheduleClassicGroupMemberHeartbeat(group, newMember);
int oldRebalanceTimeoutMs = newMember.rebalanceTimeoutMs();
int oldSessionTimeoutMs = newMember.sessionTimeoutMs();
@@ -2954,8 +2955,8 @@
// Failed to persist the member id of the given static member, revert the update of the static member in the group.
group.updateMember(newMember, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
- GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
- rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+ ClassicGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+ rescheduleClassicGroupMemberHeartbeat(group, oldMember);
responseFuture.complete(
new JoinGroupResponseData()
@@ -2972,7 +2973,7 @@
boolean isLeader = group.isLeader(newMemberId);
group.completeJoinFuture(newMember, new JoinGroupResponseData()
- .setMembers(isLeader ? group.currentGenericGroupMembers() : Collections.emptyList())
+ .setMembers(isLeader ? group.currentClassicGroupMembers() : Collections.emptyList())
.setMemberId(newMemberId)
.setGenerationId(group.generationId())
.setProtocolName(group.protocolName().orElse(null))
@@ -3033,16 +3034,16 @@
*
* @return The result that contains records to append if the group metadata manager received assignments.
*/
- public CoordinatorResult<Void, Record> genericGroupSync(
+ public CoordinatorResult<Void, Record> classicGroupSync(
RequestContext context,
SyncGroupRequestData request,
CompletableFuture<SyncGroupResponseData> responseFuture
) throws UnknownMemberIdException, GroupIdNotFoundException {
String groupId = request.groupId();
String memberId = request.memberId();
- GenericGroup group;
+ ClassicGroup group;
try {
- group = getOrMaybeCreateGenericGroup(groupId, false);
+ group = getOrMaybeCreateClassicGroup(groupId, false);
} catch (Throwable t) {
responseFuture.complete(new SyncGroupResponseData()
.setErrorCode(Errors.forException(t).code())
@@ -3105,7 +3106,7 @@
// Update group's assignment and propagate to all members.
setAndPropagateAssignment(group, assignment);
group.transitionTo(STABLE);
- metrics.record(GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
+ metrics.record(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
}
}
});
@@ -3119,7 +3120,7 @@
removePendingSyncMember(group, memberId);
// If the group is stable, we just return the current assignment
- GenericGroupMember member = group.member(memberId);
+ ClassicGroupMember member = group.member(memberId);
responseFuture.complete(new SyncGroupResponseData()
.setProtocolType(group.protocolType().orElse(null))
.setProtocolName(group.protocolName().orElse(null))
@@ -3155,7 +3156,7 @@
}
private Optional<Errors> validateSyncGroup(
- GenericGroup group,
+ ClassicGroup group,
SyncGroupRequestData request
) {
if (group.isInState(DEAD)) {
@@ -3187,11 +3188,11 @@
}
private void removePendingSyncMember(
- GenericGroup group,
+ ClassicGroup group,
String memberId
) {
group.removePendingSyncMember(memberId);
- String syncKey = genericGroupSyncKey(group.groupId());
+ String syncKey = classicGroupSyncKey(group.groupId());
switch (group.currentState()) {
case DEAD:
case EMPTY:
@@ -3210,27 +3211,27 @@
}
/**
- * Handle a generic group HeartbeatRequest.
+ * Handle a classic group HeartbeatRequest.
*
* @param context The request context.
* @param request The actual Heartbeat request.
*
* @return The Heartbeat response.
*/
- public HeartbeatResponseData genericGroupHeartbeat(
+ public HeartbeatResponseData classicGroupHeartbeat(
RequestContext context,
HeartbeatRequestData request
) {
- GenericGroup group = getOrMaybeCreateGenericGroup(request.groupId(), false);
+ ClassicGroup group = getOrMaybeCreateClassicGroup(request.groupId(), false);
- validateGenericGroupHeartbeat(group, request.memberId(), request.groupInstanceId(), request.generationId());
+ validateClassicGroupHeartbeat(group, request.memberId(), request.groupInstanceId(), request.generationId());
switch (group.currentState()) {
case EMPTY:
return new HeartbeatResponseData().setErrorCode(Errors.UNKNOWN_MEMBER_ID.code());
case PREPARING_REBALANCE:
- rescheduleGenericGroupMemberHeartbeat(group, group.member(request.memberId()));
+ rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId()));
return new HeartbeatResponseData().setErrorCode(Errors.REBALANCE_IN_PROGRESS.code());
case COMPLETING_REBALANCE:
@@ -3238,7 +3239,7 @@
// Consumers may start sending heartbeats after join-group response, while the group
// is in CompletingRebalance state. In this case, we should treat them as
// normal heartbeat requests and reset the timer
- rescheduleGenericGroupMemberHeartbeat(group, group.member(request.memberId()));
+ rescheduleClassicGroupMemberHeartbeat(group, group.member(request.memberId()));
return new HeartbeatResponseData();
default:
@@ -3248,7 +3249,7 @@
}
/**
- * Validates a generic group heartbeat request.
+ * Validates a classic group heartbeat request.
*
* @param group The group.
* @param memberId The member id.
@@ -3259,8 +3260,8 @@
* @throws IllegalGenerationException If the generation id in the request and the generation id of the
* group does not match.
*/
- private void validateGenericGroupHeartbeat(
- GenericGroup group,
+ private void validateClassicGroupHeartbeat(
+ ClassicGroup group,
String memberId,
String groupInstanceId,
int generationId
@@ -3281,7 +3282,7 @@
}
/**
- * Handle a generic LeaveGroupRequest.
+ * Handle a classic LeaveGroupRequest.
*
* @param context The request context.
* @param request The actual LeaveGroup request.
@@ -3289,11 +3290,11 @@
* @return The LeaveGroup response and the GroupMetadata record to append if the group
* no longer has any members.
*/
- public CoordinatorResult<LeaveGroupResponseData, Record> genericGroupLeave(
+ public CoordinatorResult<LeaveGroupResponseData, Record> classicGroupLeave(
RequestContext context,
LeaveGroupRequestData request
) throws UnknownMemberIdException, GroupIdNotFoundException {
- GenericGroup group = getOrMaybeCreateGenericGroup(request.groupId(), false);
+ ClassicGroup group = getOrMaybeCreateClassicGroup(request.groupId(), false);
if (group.isInState(DEAD)) {
return new CoordinatorResult<>(
Collections.emptyList(),
@@ -3310,7 +3311,7 @@
// in which case we expect the MemberId to be undefined.
if (UNKNOWN_MEMBER_ID.equals(member.memberId())) {
if (member.groupInstanceId() != null && group.hasStaticMember(member.groupInstanceId())) {
- removeCurrentMemberFromGenericGroup(
+ removeCurrentMemberFromClassicGroup(
group,
group.staticMemberId(member.groupInstanceId()),
reason
@@ -3330,7 +3331,7 @@
}
} else if (group.isPendingMember(member.memberId())) {
group.remove(member.memberId());
- timer.cancel(genericGroupHeartbeatKey(group.groupId(), member.memberId()));
+ timer.cancel(classicGroupHeartbeatKey(group.groupId(), member.memberId()));
log.info("[Group {}] Pending member {} has left group through explicit `LeaveGroup` request; client reason: {}",
group.groupId(), member.memberId(), reason);
@@ -3342,7 +3343,7 @@
} else {
try {
group.validateMember(member.memberId(), member.groupInstanceId(), "leave-group");
- removeCurrentMemberFromGenericGroup(
+ removeCurrentMemberFromClassicGroup(
group,
member.memberId(),
reason
@@ -3396,18 +3397,18 @@
* Remove a member from the group. Cancel member's heartbeat, and prepare rebalance
* or complete the join phase if necessary.
*
- * @param group The generic group.
+ * @param group The classic group.
* @param memberId The member id.
* @param reason The reason for the LeaveGroup request.
*
*/
- private void removeCurrentMemberFromGenericGroup(
- GenericGroup group,
+ private void removeCurrentMemberFromClassicGroup(
+ ClassicGroup group,
String memberId,
String reason
) {
- GenericGroupMember member = group.member(memberId);
- timer.cancel(genericGroupHeartbeatKey(group.groupId(), memberId));
+ ClassicGroupMember member = group.member(memberId);
+ timer.cancel(classicGroupHeartbeatKey(group.groupId(), memberId));
log.info("[Group {}] Member {} has left group through explicit `LeaveGroup` request; client reason: {}",
group.groupId(), memberId, reason);
@@ -3488,7 +3489,7 @@
}
/**
- * Generate a generic group heartbeat key for the timer.
+ * Generate a classic group heartbeat key for the timer.
*
* Package private for testing.
*
@@ -3497,12 +3498,12 @@
*
* @return the heartbeat key.
*/
- static String genericGroupHeartbeatKey(String groupId, String memberId) {
+ static String classicGroupHeartbeatKey(String groupId, String memberId) {
return "heartbeat-" + groupId + "-" + memberId;
}
/**
- * Generate a generic group join key for the timer.
+ * Generate a classic group join key for the timer.
*
* Package private for testing.
*
@@ -3510,12 +3511,12 @@
*
* @return the join key.
*/
- static String genericGroupJoinKey(String groupId) {
+ static String classicGroupJoinKey(String groupId) {
return "join-" + groupId;
}
/**
- * Generate a generic group sync key for the timer.
+ * Generate a classic group sync key for the timer.
*
* Package private for testing.
*
@@ -3523,7 +3524,7 @@
*
* @return the sync key.
*/
- static String genericGroupSyncKey(String groupId) {
+ static String classicGroupSyncKey(String groupId) {
return "sync-" + groupId;
}
}
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationCondition.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationCondition.java
index ce3f299..ccb8ccf 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationCondition.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationCondition.java
@@ -18,7 +18,7 @@
/**
* An offset is considered expired based on different factors, such as the state of the group
- * and/or the GroupMetadata record version (for generic groups). This class is used to check
+ * and/or the GroupMetadata record version (for classic groups). This class is used to check
* how offsets for the group should be expired.
*/
public interface OffsetExpirationCondition {
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationConditionImpl.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationConditionImpl.java
index cd82310..07b6ea0 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationConditionImpl.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetExpirationConditionImpl.java
@@ -33,7 +33,7 @@
/**
* Determine whether an offset is expired. Older versions have an expire timestamp per partition. If this
* exists, compare against the current timestamp. Otherwise, use the base timestamp (either commit timestamp
- * or current state timestamp if group is empty for generic groups) and check whether the offset has
+ * or current state timestamp if group is empty for classic groups) and check whether the offset has
* exceeded the offset retention.
*
* @param offset The offset and metadata.
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java
index b4f90bd..b060800 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java
@@ -41,8 +41,8 @@
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
-import org.apache.kafka.coordinator.group.generic.GenericGroup;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroup;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics;
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
@@ -300,7 +300,7 @@
// either the admin client or a consumer which does not use the group management
// facility. In this case, a so-called simple group is created and the request
// is accepted.
- group = groupMetadataManager.getOrMaybeCreateGenericGroup(request.groupId(), true);
+ group = groupMetadataManager.getOrMaybeCreateClassicGroup(request.groupId(), true);
} else {
if (context.header.apiVersion() >= 9) {
// Starting from version 9 of the OffsetCommit API, we return GROUP_ID_NOT_FOUND
@@ -354,7 +354,7 @@
// either the admin client or a consumer which does not use the group management
// facility. In this case, a so-called simple group is created and the request
// is accepted.
- group = groupMetadataManager.getOrMaybeCreateGenericGroup(request.groupId(), true);
+ group = groupMetadataManager.getOrMaybeCreateClassicGroup(request.groupId(), true);
} else {
throw Errors.ILLEGAL_GENERATION.exception();
}
@@ -450,12 +450,12 @@
// In the old consumer group protocol, the offset commits maintain the session if
// the group is in Stable or PreparingRebalance state.
- if (group.type() == Group.GroupType.GENERIC) {
- GenericGroup genericGroup = (GenericGroup) group;
- if (genericGroup.isInState(GenericGroupState.STABLE) || genericGroup.isInState(GenericGroupState.PREPARING_REBALANCE)) {
- groupMetadataManager.rescheduleGenericGroupMemberHeartbeat(
- genericGroup,
- genericGroup.member(request.memberId())
+ if (group.type() == Group.GroupType.CLASSIC) {
+ ClassicGroup classicGroup = (ClassicGroup) group;
+ if (classicGroup.isInState(ClassicGroupState.STABLE) || classicGroup.isInState(ClassicGroupState.PREPARING_REBALANCE)) {
+ groupMetadataManager.rescheduleClassicGroupMemberHeartbeat(
+ classicGroup,
+ classicGroup.member(request.memberId())
);
}
}
@@ -850,15 +850,15 @@
final int partition = key.partition();
if (value != null) {
- // The generic or consumer group should exist when offsets are committed or
+ // The classic or consumer group should exist when offsets are committed or
// replayed. However, it won't if the consumer commits offsets but does not
// use the membership functionality. In this case, we automatically create
- // a so-called "simple consumer group". This is an empty generic group
+ // a so-called "simple consumer group". This is an empty classic group
// without a protocol type.
try {
groupMetadataManager.group(groupId);
} catch (GroupIdNotFoundException ex) {
- groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, true);
+ groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, true);
}
if (producerId == RecordBatch.NO_PRODUCER_ID) {
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java
index c9bae67..5e7e6fa 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java
@@ -37,7 +37,7 @@
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
-import org.apache.kafka.coordinator.group.generic.GenericGroup;
+import org.apache.kafka.coordinator.group.classic.ClassicGroup;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
@@ -380,13 +380,13 @@
/**
* Creates a GroupMetadata record.
*
- * @param group The generic group.
- * @param assignment The generic group assignment.
+ * @param group The classic group.
+ * @param assignment The classic group assignment.
* @param metadataVersion The metadata version.
* @return The record.
*/
public static Record newGroupMetadataRecord(
- GenericGroup group,
+ ClassicGroup group,
Map<String, byte[]> assignment,
MetadataVersion metadataVersion
) {
@@ -457,12 +457,12 @@
/**
* Creates an empty GroupMetadata record.
*
- * @param group The generic group.
+ * @param group The classic group.
* @param metadataVersion The metadata version.
* @return The record.
*/
public static Record newEmptyGroupMetadataRecord(
- GenericGroup group,
+ ClassicGroup group,
MetadataVersion metadataVersion
) {
return new Record(
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java
similarity index 93%
rename from group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java
rename to group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java
index 9ff3c16..188ad23 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.kafka.coordinator.group.generic;
+package org.apache.kafka.coordinator.group.classic;
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
import org.apache.kafka.common.errors.ApiException;
@@ -56,20 +56,20 @@
import java.util.concurrent.CompletableFuture;
import java.util.stream.Collectors;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+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.EMPTY;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.STABLE;
/**
- * This class holds metadata for a generic group where the
+ * This class holds metadata for a classic group where the
* member assignment is driven solely from the client side.
*
* The APIs members use to make changes to the group membership
* consist of JoinGroup, SyncGroup, and LeaveGroup.
*/
-public class GenericGroup implements Group {
+public class ClassicGroup implements Group {
/**
* Empty generation.
@@ -110,12 +110,12 @@
/**
* The current group state.
*/
- private GenericGroupState state;
+ private ClassicGroupState state;
/**
* The previous group state.
*/
- private GenericGroupState previousState;
+ private ClassicGroupState previousState;
/**
* The timestamp of when the group transitioned
@@ -146,7 +146,7 @@
/**
* The members of the group.
*/
- private final Map<String, GenericGroupMember> members = new HashMap<>();
+ private final Map<String, ClassicGroupMember> members = new HashMap<>();
/**
* The static members of the group.
@@ -191,10 +191,10 @@
*/
private final GroupCoordinatorMetricsShard metrics;
- public GenericGroup(
+ public ClassicGroup(
LogContext logContext,
String groupId,
- GenericGroupState initialState,
+ ClassicGroupState initialState,
Time time,
GroupCoordinatorMetricsShard metrics
) {
@@ -212,10 +212,10 @@
);
}
- public GenericGroup(
+ public ClassicGroup(
LogContext logContext,
String groupId,
- GenericGroupState initialState,
+ ClassicGroupState initialState,
Time time,
GroupCoordinatorMetricsShard metrics,
int generationId,
@@ -225,7 +225,7 @@
Optional<Long> currentStateTimestamp
) {
Objects.requireNonNull(logContext);
- this.log = logContext.logger(GenericGroup.class);
+ this.log = logContext.logger(ClassicGroup.class);
this.groupId = Objects.requireNonNull(groupId);
this.state = Objects.requireNonNull(initialState);
this.previousState = DEAD;
@@ -241,11 +241,11 @@
/**
* The type of this group.
*
- * @return The group type (Generic).
+ * @return The group type (Classic).
*/
@Override
public GroupType type() {
- return GroupType.GENERIC;
+ return GroupType.CLASSIC;
}
/**
@@ -299,11 +299,11 @@
/**
* @return the current group state.
*/
- public GenericGroupState currentState() {
+ public ClassicGroupState currentState() {
return this.state;
}
- public GenericGroupState previousState() {
+ public ClassicGroupState previousState() {
return this.previousState;
}
@@ -320,7 +320,7 @@
* @param groupState the state to match against.
* @return true if the state matches, false otherwise.
*/
- public boolean isInState(GenericGroupState groupState) {
+ public boolean isInState(ClassicGroupState groupState) {
return this.state == groupState;
}
@@ -340,7 +340,7 @@
* @param memberId the member id.
* @return the member metadata if it exists, null otherwise.
*/
- public GenericGroupMember member(String memberId) {
+ public ClassicGroupMember member(String memberId) {
return members.get(memberId);
}
@@ -407,7 +407,7 @@
*
* @param member the member to add.
*/
- public void add(GenericGroupMember member) {
+ public void add(ClassicGroupMember member) {
add(member, null);
}
@@ -417,7 +417,7 @@
* @param member the member to add.
* @param future the future to complete once the join group phase completes.
*/
- public void add(GenericGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
+ public void add(ClassicGroupMember member, CompletableFuture<JoinGroupResponseData> future) {
member.groupInstanceId().ifPresent(instanceId -> {
if (staticMembers.containsKey(instanceId)) {
throw new IllegalStateException("Static member with groupInstanceId=" +
@@ -460,7 +460,7 @@
* @param memberId the member id to remove.
*/
public void remove(String memberId) {
- GenericGroupMember removedMember = members.remove(memberId);
+ ClassicGroupMember removedMember = members.remove(memberId);
if (removedMember != null) {
decrementSupportedProtocols(removedMember);
if (removedMember.isAwaitingJoin()) {
@@ -493,9 +493,9 @@
*/
public boolean maybeElectNewJoinedLeader() {
if (leaderId.isPresent()) {
- GenericGroupMember currentLeader = member(leaderId.get());
+ ClassicGroupMember currentLeader = member(leaderId.get());
if (!currentLeader.isAwaitingJoin()) {
- for (GenericGroupMember member : members.values()) {
+ for (ClassicGroupMember member : members.values()) {
if (member.isAwaitingJoin()) {
leaderId = Optional.of(member.memberId());
log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
@@ -531,12 +531,12 @@
* @param newMemberId the new member id that will replace the old member id.
* @return the member with the new id.
*/
- public GenericGroupMember replaceStaticMember(
+ public ClassicGroupMember replaceStaticMember(
String groupInstanceId,
String oldMemberId,
String newMemberId
) {
- GenericGroupMember removedMember = members.remove(oldMemberId);
+ ClassicGroupMember removedMember = members.remove(oldMemberId);
if (removedMember == null) {
throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
}
@@ -560,7 +560,7 @@
.setErrorCode(Errors.FENCED_INSTANCE_ID.code());
completeSyncFuture(removedMember, syncGroupResponse);
- GenericGroupMember newMember = new GenericGroupMember(
+ ClassicGroupMember newMember = new ClassicGroupMember(
newMemberId,
removedMember.groupInstanceId(),
removedMember.clientId(),
@@ -693,8 +693,8 @@
* @return members who have yet to rejoin during the
* join group phase.
*/
- public Map<String, GenericGroupMember> notYetRejoinedMembers() {
- Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
+ public Map<String, ClassicGroupMember> notYetRejoinedMembers() {
+ Map<String, ClassicGroupMember> notYetRejoinedMembers = new HashMap<>();
members.values().forEach(member -> {
if (!member.isAwaitingJoin()) {
notYetRejoinedMembers.put(member.memberId(), member);
@@ -741,7 +741,7 @@
/**
* @return all members.
*/
- public Collection<GenericGroupMember> allMembers() {
+ public Collection<ClassicGroupMember> allMembers() {
return members.values();
}
@@ -751,7 +751,7 @@
*/
public int rebalanceTimeoutMs() {
int maxRebalanceTimeoutMs = 0;
- for (GenericGroupMember member : members.values()) {
+ for (ClassicGroupMember member : members.values()) {
maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
}
return maxRebalanceTimeoutMs;
@@ -856,8 +856,8 @@
/**
* Validates the OffsetFetch request.
*
- * @param memberId The member id. This is not provided for generic groups.
- * @param memberEpoch The member epoch for consumer groups. This is not provided for generic groups.
+ * @param memberId The member id. This is not provided for classic groups.
+ * @param memberEpoch The member epoch for consumer groups. This is not provided for classic groups.
* @param lastCommittedOffset The last committed offsets in the timeline.
*/
@Override
@@ -984,12 +984,12 @@
* Transition to a group state.
* @param groupState the group state.
*/
- public void transitionTo(GenericGroupState groupState) {
+ public void transitionTo(ClassicGroupState groupState) {
assertValidTransition(groupState);
previousState = state;
state = groupState;
currentStateTimestamp = Optional.of(time.milliseconds());
- metrics.onGenericGroupStateTransition(previousState, state);
+ metrics.onClassicGroupStateTransition(previousState, state);
}
/**
@@ -1028,7 +1028,7 @@
*
* @param member the member.
*/
- private void incrementSupportedProtocols(GenericGroupMember member) {
+ private void incrementSupportedProtocols(ClassicGroupMember member) {
member.supportedProtocols().forEach(protocol -> {
int count = supportedProtocols.getOrDefault(protocol.name(), 0);
supportedProtocols.put(protocol.name(), count + 1);
@@ -1041,7 +1041,7 @@
*
* @param member the member.
*/
- private void decrementSupportedProtocols(GenericGroupMember member) {
+ private void decrementSupportedProtocols(ClassicGroupMember member) {
member.supportedProtocols().forEach(protocol -> {
int count = supportedProtocols.getOrDefault(protocol.name(), 0);
supportedProtocols.put(protocol.name(), count - 1);
@@ -1069,10 +1069,10 @@
*
* @return a boolean based on the condition mentioned above.
*/
- public boolean supportsProtocols(GenericGroupMember member) {
+ public boolean supportsProtocols(ClassicGroupMember member) {
return supportsProtocols(
member.protocolType(),
- GenericGroupMember.plainProtocolSet(member.supportedProtocols())
+ ClassicGroupMember.plainProtocolSet(member.supportedProtocols())
);
}
@@ -1091,7 +1091,7 @@
) {
return supportsProtocols(
memberProtocolType,
- GenericGroupMember.plainProtocolSet(memberProtocols)
+ ClassicGroupMember.plainProtocolSet(memberProtocols)
);
}
@@ -1122,7 +1122,7 @@
}
/**
- * Returns true if the generic group is actively subscribed to the topic. When the generic group does not know,
+ * Returns true if the classic group is actively subscribed to the topic. When the classic group does not know,
* because the information is not available yet or because it has failed to parse the Consumer Protocol, we
* consider the group not subscribed to the topic if the group is not using any protocol or not using the
* consumer group protocol.
@@ -1191,7 +1191,7 @@
* @param future the future that is invoked once the join phase is complete.
*/
public void updateMember(
- GenericGroupMember member,
+ ClassicGroupMember member,
JoinGroupRequestProtocolCollection protocols,
int rebalanceTimeoutMs,
int sessionTimeoutMs,
@@ -1219,7 +1219,7 @@
* @return true if a join future actually completes.
*/
public boolean completeJoinFuture(
- GenericGroupMember member,
+ ClassicGroupMember member,
JoinGroupResponseData response
) {
if (member.isAwaitingJoin()) {
@@ -1239,7 +1239,7 @@
* @return true if a sync future actually completes.
*/
public boolean completeSyncFuture(
- GenericGroupMember member,
+ ClassicGroupMember member,
SyncGroupResponseData response
) {
if (member.isAwaitingSync()) {
@@ -1272,9 +1272,9 @@
*
* @return the members.
*/
- public List<JoinGroupResponseMember> currentGenericGroupMembers() {
+ public List<JoinGroupResponseMember> currentClassicGroupMembers() {
if (isInState(DEAD) || isInState(PREPARING_REBALANCE)) {
- throw new IllegalStateException("Cannot obtain generic member metadata for group " +
+ throw new IllegalStateException("Cannot obtain classic member metadata for group " +
groupId + " in state " + state);
}
@@ -1301,7 +1301,7 @@
*/
public Map<String, byte[]> groupAssignment() {
return allMembers().stream().collect(Collectors.toMap(
- GenericGroupMember::memberId, GenericGroupMember::assignment
+ ClassicGroupMember::memberId, ClassicGroupMember::assignment
));
}
@@ -1310,7 +1310,7 @@
*
* @param targetState the target state to transition to.
*/
- private void assertValidTransition(GenericGroupState targetState) {
+ private void assertValidTransition(ClassicGroupState targetState) {
if (!targetState.validPreviousStates().contains(state)) {
throw new IllegalStateException("Group " + groupId + " should be in one of " +
targetState.validPreviousStates() + " states before moving to " + targetState +
@@ -1320,7 +1320,7 @@
@Override
public String toString() {
- return "GenericGroupMetadata(" +
+ return "ClassicGroupMetadata(" +
"groupId=" + groupId + ", " +
"generation=" + generationId + ", " +
"protocolType=" + protocolType + ", " +
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMember.java
similarity index 97%
rename from group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java
rename to group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMember.java
index 702bea6..5d20e64 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupMember.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMember.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.kafka.coordinator.group.generic;
+package org.apache.kafka.coordinator.group.classic;
import org.apache.kafka.common.message.DescribeGroupsResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol;
@@ -31,7 +31,7 @@
import java.util.stream.Collectors;
/**
- * This class encapsulates a generic group member's metadata.
+ * This class encapsulates a classic group member's metadata.
*
* Member metadata contains the following:
*
@@ -52,7 +52,7 @@
* is kept in metadata until the leader provides the group assignment
* and the group transitions to stable
*/
-public class GenericGroupMember {
+public class ClassicGroupMember {
/**
* An empty assignment.
@@ -119,7 +119,7 @@
*/
private boolean isNew = false;
- public GenericGroupMember(
+ public ClassicGroupMember(
String memberId,
Optional<String> groupInstanceId,
String clientId,
@@ -142,7 +142,7 @@
);
}
- public GenericGroupMember(
+ public ClassicGroupMember(
String memberId,
Optional<String> groupInstanceId,
String clientId,
@@ -420,7 +420,7 @@
@Override
public String toString() {
- return "GenericGroupMember(" +
+ return "ClassicGroupMember(" +
"memberId='" + memberId + '\'' +
", groupInstanceId='" + groupInstanceId.orElse("") + '\'' +
", clientId='" + clientId + '\'' +
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupState.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupState.java
similarity index 93%
rename from group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupState.java
rename to group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupState.java
index 74730b5..ac9269d 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroupState.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroupState.java
@@ -15,17 +15,17 @@
* limitations under the License.
*/
-package org.apache.kafka.coordinator.group.generic;
+package org.apache.kafka.coordinator.group.classic;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
/**
- * Represents all states that a generic group can be in, as well as the states that a group must
+ * Represents all states that a classic group can be in, as well as the states that a group must
* be in to transition to a particular state.
*/
-public enum GenericGroupState {
+public enum ClassicGroupState {
/**
* Group has no more members, but lingers until all offsets have expired. This state
@@ -104,7 +104,7 @@
DEAD("Dead");
private final String name;
- private Set<GenericGroupState> validPreviousStates;
+ private Set<ClassicGroupState> validPreviousStates;
static {
EMPTY.addValidPreviousStates(PREPARING_REBALANCE);
@@ -114,7 +114,7 @@
DEAD.addValidPreviousStates(STABLE, PREPARING_REBALANCE, COMPLETING_REBALANCE, EMPTY, DEAD);
}
- GenericGroupState(String name) {
+ ClassicGroupState(String name) {
this.name = name;
}
@@ -123,14 +123,14 @@
return name;
}
- private void addValidPreviousStates(GenericGroupState... validPreviousStates) {
+ private void addValidPreviousStates(ClassicGroupState... validPreviousStates) {
this.validPreviousStates = new HashSet<>(Arrays.asList(validPreviousStates));
}
/**
* @return valid previous states a group must be in to transition to this state.
*/
- public Set<GenericGroupState> validPreviousStates() {
+ public Set<ClassicGroupState> validPreviousStates() {
return this.validPreviousStates;
}
}
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java
index 6ba1c68..da66df9 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetrics.java
@@ -26,7 +26,7 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.server.metrics.KafkaYammerMetrics;
import org.apache.kafka.timeline.SnapshotRegistry;
@@ -45,21 +45,21 @@
public static final String METRICS_GROUP = "group-coordinator-metrics";
/**
- * Old generic group count metric. To be deprecated.
+ * Old classic group count metric. To be deprecated.
*/
- public final static com.yammer.metrics.core.MetricName NUM_GENERIC_GROUPS = getMetricName(
+ public final static com.yammer.metrics.core.MetricName NUM_CLASSIC_GROUPS = getMetricName(
"GroupMetadataManager", "NumGroups");
public final static com.yammer.metrics.core.MetricName NUM_OFFSETS = getMetricName(
"GroupMetadataManager", "NumOffsets");
- public final static com.yammer.metrics.core.MetricName NUM_GENERIC_GROUPS_PREPARING_REBALANCE = getMetricName(
+ public final static com.yammer.metrics.core.MetricName NUM_CLASSIC_GROUPS_PREPARING_REBALANCE = getMetricName(
"GroupMetadataManager", "NumGroupsPreparingRebalance");
- public final static com.yammer.metrics.core.MetricName NUM_GENERIC_GROUPS_COMPLETING_REBALANCE = getMetricName(
+ public final static com.yammer.metrics.core.MetricName NUM_CLASSIC_GROUPS_COMPLETING_REBALANCE = getMetricName(
"GroupMetadataManager", "NumGroupsCompletingRebalance");
- public final static com.yammer.metrics.core.MetricName NUM_GENERIC_GROUPS_STABLE = getMetricName(
+ public final static com.yammer.metrics.core.MetricName NUM_CLASSIC_GROUPS_STABLE = getMetricName(
"GroupMetadataManager", "NumGroupsStable");
- public final static com.yammer.metrics.core.MetricName NUM_GENERIC_GROUPS_DEAD = getMetricName(
+ public final static com.yammer.metrics.core.MetricName NUM_CLASSIC_GROUPS_DEAD = getMetricName(
"GroupMetadataManager", "NumGroupsDead");
- public final static com.yammer.metrics.core.MetricName NUM_GENERIC_GROUPS_EMPTY = getMetricName(
+ public final static com.yammer.metrics.core.MetricName NUM_CLASSIC_GROUPS_EMPTY = getMetricName(
"GroupMetadataManager", "NumGroupsEmpty");
public final static String GROUP_COUNT_METRIC_NAME = "group-count";
@@ -70,11 +70,11 @@
public static final String OFFSET_COMMITS_SENSOR_NAME = "OffsetCommits";
public static final String OFFSET_EXPIRED_SENSOR_NAME = "OffsetExpired";
public static final String OFFSET_DELETIONS_SENSOR_NAME = "OffsetDeletions";
- public static final String GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME = "CompletedRebalances";
- public static final String GENERIC_GROUP_REBALANCES_SENSOR_NAME = "GenericGroupRebalances";
+ public static final String CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME = "CompletedRebalances";
+ public static final String CLASSIC_GROUP_REBALANCES_SENSOR_NAME = "ClassicGroupRebalances";
public static final String CONSUMER_GROUP_REBALANCES_SENSOR_NAME = "ConsumerGroupRebalances";
- private final MetricName genericGroupCountMetricName;
+ private final MetricName classicGroupCountMetricName;
private final MetricName consumerGroupCountMetricName;
private final MetricName consumerGroupCountEmptyMetricName;
private final MetricName consumerGroupCountAssigningMetricName;
@@ -99,11 +99,11 @@
this.registry = Objects.requireNonNull(registry);
this.metrics = Objects.requireNonNull(metrics);
- genericGroupCountMetricName = metrics.metricName(
+ classicGroupCountMetricName = metrics.metricName(
GROUP_COUNT_METRIC_NAME,
METRICS_GROUP,
- "The total number of groups using the generic rebalance protocol.",
- Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.GENERIC.toString())
+ "The total number of groups using the classic rebalance protocol.",
+ Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.CLASSIC.toString())
);
consumerGroupCountMetricName = metrics.metricName(
@@ -177,23 +177,23 @@
METRICS_GROUP,
"The total number of administrative deleted offsets")));
- Sensor genericGroupCompletedRebalancesSensor = metrics.sensor(GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
- genericGroupCompletedRebalancesSensor.add(new Meter(
+ Sensor classicGroupCompletedRebalancesSensor = metrics.sensor(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
+ classicGroupCompletedRebalancesSensor.add(new Meter(
metrics.metricName("group-completed-rebalance-rate",
METRICS_GROUP,
- "The rate of generic group completed rebalances"),
+ "The rate of classic group completed rebalances"),
metrics.metricName("group-completed-rebalance-count",
METRICS_GROUP,
- "The total number of generic group completed rebalances")));
+ "The total number of classic group completed rebalances")));
- Sensor genericGroupPreparingRebalancesSensor = metrics.sensor(GENERIC_GROUP_REBALANCES_SENSOR_NAME);
- genericGroupPreparingRebalancesSensor.add(new Meter(
+ Sensor classicGroupPreparingRebalancesSensor = metrics.sensor(CLASSIC_GROUP_REBALANCES_SENSOR_NAME);
+ classicGroupPreparingRebalancesSensor.add(new Meter(
metrics.metricName("group-rebalance-rate",
METRICS_GROUP,
- "The rate of generic group preparing rebalances"),
+ "The rate of classic group preparing rebalances"),
metrics.metricName("group-rebalance-count",
METRICS_GROUP,
- "The total number of generic group preparing rebalances")));
+ "The total number of classic group preparing rebalances")));
Sensor consumerGroupRebalanceSensor = metrics.sensor(CONSUMER_GROUP_REBALANCES_SENSOR_NAME);
consumerGroupRebalanceSensor.add(new Meter(
@@ -208,8 +208,8 @@
Utils.mkEntry(OFFSET_COMMITS_SENSOR_NAME, offsetCommitsSensor),
Utils.mkEntry(OFFSET_EXPIRED_SENSOR_NAME, offsetExpiredSensor),
Utils.mkEntry(OFFSET_DELETIONS_SENSOR_NAME, offsetDeletionsSensor),
- Utils.mkEntry(GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME, genericGroupCompletedRebalancesSensor),
- Utils.mkEntry(GENERIC_GROUP_REBALANCES_SENSOR_NAME, genericGroupPreparingRebalancesSensor),
+ Utils.mkEntry(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME, classicGroupCompletedRebalancesSensor),
+ Utils.mkEntry(CLASSIC_GROUP_REBALANCES_SENSOR_NAME, classicGroupPreparingRebalancesSensor),
Utils.mkEntry(CONSUMER_GROUP_REBALANCES_SENSOR_NAME, consumerGroupRebalanceSensor)
));
}
@@ -218,12 +218,12 @@
return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numOffsets).sum();
}
- private Long numGenericGroups() {
- return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numGenericGroups).sum();
+ private Long numClassicGroups() {
+ return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numClassicGroups).sum();
}
- private Long numGenericGroups(GenericGroupState state) {
- return shards.values().stream().mapToLong(shard -> shard.numGenericGroups(state)).sum();
+ private Long numClassicGroups(ClassicGroupState state) {
+ return shards.values().stream().mapToLong(shard -> shard.numClassicGroups(state)).sum();
}
private long numConsumerGroups() {
@@ -238,16 +238,16 @@
public void close() {
Arrays.asList(
NUM_OFFSETS,
- NUM_GENERIC_GROUPS,
- NUM_GENERIC_GROUPS_PREPARING_REBALANCE,
- NUM_GENERIC_GROUPS_COMPLETING_REBALANCE,
- NUM_GENERIC_GROUPS_STABLE,
- NUM_GENERIC_GROUPS_DEAD,
- NUM_GENERIC_GROUPS_EMPTY
+ NUM_CLASSIC_GROUPS,
+ NUM_CLASSIC_GROUPS_PREPARING_REBALANCE,
+ NUM_CLASSIC_GROUPS_COMPLETING_REBALANCE,
+ NUM_CLASSIC_GROUPS_STABLE,
+ NUM_CLASSIC_GROUPS_DEAD,
+ NUM_CLASSIC_GROUPS_EMPTY
).forEach(registry::removeMetric);
Arrays.asList(
- genericGroupCountMetricName,
+ classicGroupCountMetricName,
consumerGroupCountMetricName,
consumerGroupCountEmptyMetricName,
consumerGroupCountAssigningMetricName,
@@ -260,8 +260,8 @@
OFFSET_COMMITS_SENSOR_NAME,
OFFSET_EXPIRED_SENSOR_NAME,
OFFSET_DELETIONS_SENSOR_NAME,
- GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME,
- GENERIC_GROUP_REBALANCES_SENSOR_NAME,
+ CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME,
+ CLASSIC_GROUP_REBALANCES_SENSOR_NAME,
CONSUMER_GROUP_REBALANCES_SENSOR_NAME
).forEach(metrics::removeSensor);
}
@@ -309,51 +309,51 @@
}
});
- registry.newGauge(NUM_GENERIC_GROUPS, new com.yammer.metrics.core.Gauge<Long>() {
+ registry.newGauge(NUM_CLASSIC_GROUPS, new com.yammer.metrics.core.Gauge<Long>() {
@Override
public Long value() {
- return numGenericGroups();
+ return numClassicGroups();
}
});
- registry.newGauge(NUM_GENERIC_GROUPS_PREPARING_REBALANCE, new com.yammer.metrics.core.Gauge<Long>() {
+ registry.newGauge(NUM_CLASSIC_GROUPS_PREPARING_REBALANCE, new com.yammer.metrics.core.Gauge<Long>() {
@Override
public Long value() {
- return numGenericGroups(GenericGroupState.PREPARING_REBALANCE);
+ return numClassicGroups(ClassicGroupState.PREPARING_REBALANCE);
}
});
- registry.newGauge(NUM_GENERIC_GROUPS_COMPLETING_REBALANCE, new com.yammer.metrics.core.Gauge<Long>() {
+ registry.newGauge(NUM_CLASSIC_GROUPS_COMPLETING_REBALANCE, new com.yammer.metrics.core.Gauge<Long>() {
@Override
public Long value() {
- return numGenericGroups(GenericGroupState.COMPLETING_REBALANCE);
+ return numClassicGroups(ClassicGroupState.COMPLETING_REBALANCE);
}
});
- registry.newGauge(NUM_GENERIC_GROUPS_STABLE, new com.yammer.metrics.core.Gauge<Long>() {
+ registry.newGauge(NUM_CLASSIC_GROUPS_STABLE, new com.yammer.metrics.core.Gauge<Long>() {
@Override
public Long value() {
- return numGenericGroups(GenericGroupState.STABLE);
+ return numClassicGroups(ClassicGroupState.STABLE);
}
});
- registry.newGauge(NUM_GENERIC_GROUPS_DEAD, new com.yammer.metrics.core.Gauge<Long>() {
+ registry.newGauge(NUM_CLASSIC_GROUPS_DEAD, new com.yammer.metrics.core.Gauge<Long>() {
@Override
public Long value() {
- return numGenericGroups(GenericGroupState.DEAD);
+ return numClassicGroups(ClassicGroupState.DEAD);
}
});
- registry.newGauge(NUM_GENERIC_GROUPS_EMPTY, new com.yammer.metrics.core.Gauge<Long>() {
+ registry.newGauge(NUM_CLASSIC_GROUPS_EMPTY, new com.yammer.metrics.core.Gauge<Long>() {
@Override
public Long value() {
- return numGenericGroups(GenericGroupState.EMPTY);
+ return numClassicGroups(ClassicGroupState.EMPTY);
}
});
metrics.addMetric(
- genericGroupCountMetricName,
- (Gauge<Long>) (config, now) -> numGenericGroups()
+ classicGroupCountMetricName,
+ (Gauge<Long>) (config, now) -> numClassicGroups()
);
metrics.addMetric(
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java
index d5d9eeb..3fb4b42 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShard.java
@@ -20,7 +20,7 @@
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineLong;
@@ -56,9 +56,9 @@
}
}
/**
- * Consumer group size gauge counters keyed by the metric name.
+ * Classic group size gauge counters keyed by the metric name.
*/
- private final Map<GenericGroupState, AtomicLong> genericGroupGauges;
+ private final Map<ClassicGroupState, AtomicLong> classicGroupGauges;
/**
* Consumer group size gauge counters keyed by the metric name.
@@ -76,9 +76,9 @@
private final TimelineGaugeCounter numOffsetsTimelineGaugeCounter;
/**
- * The number of generic groups metric counter.
+ * The number of classic groups metric counter.
*/
- private final TimelineGaugeCounter numGenericGroupsTimelineCounter;
+ private final TimelineGaugeCounter numClassicGroupsTimelineCounter;
/**
* The topic partition.
@@ -92,14 +92,14 @@
) {
Objects.requireNonNull(snapshotRegistry);
numOffsetsTimelineGaugeCounter = new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0));
- numGenericGroupsTimelineCounter = new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0));
+ numClassicGroupsTimelineCounter = new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0));
- this.genericGroupGauges = Utils.mkMap(
- Utils.mkEntry(GenericGroupState.PREPARING_REBALANCE, new AtomicLong(0)),
- Utils.mkEntry(GenericGroupState.COMPLETING_REBALANCE, new AtomicLong(0)),
- Utils.mkEntry(GenericGroupState.STABLE, new AtomicLong(0)),
- Utils.mkEntry(GenericGroupState.DEAD, new AtomicLong(0)),
- Utils.mkEntry(GenericGroupState.EMPTY, new AtomicLong(0))
+ this.classicGroupGauges = Utils.mkMap(
+ Utils.mkEntry(ClassicGroupState.PREPARING_REBALANCE, new AtomicLong(0)),
+ Utils.mkEntry(ClassicGroupState.COMPLETING_REBALANCE, new AtomicLong(0)),
+ Utils.mkEntry(ClassicGroupState.STABLE, new AtomicLong(0)),
+ Utils.mkEntry(ClassicGroupState.DEAD, new AtomicLong(0)),
+ Utils.mkEntry(ClassicGroupState.EMPTY, new AtomicLong(0))
);
this.consumerGroupGauges = Utils.mkMap(
@@ -119,8 +119,8 @@
this.topicPartition = Objects.requireNonNull(topicPartition);
}
- public void incrementNumGenericGroups(GenericGroupState state) {
- AtomicLong counter = genericGroupGauges.get(state);
+ public void incrementNumClassicGroups(ClassicGroupState state) {
+ AtomicLong counter = classicGroupGauges.get(state);
if (counter != null) {
counter.incrementAndGet();
}
@@ -159,12 +159,12 @@
}
/**
- * Decrement the number of consumer groups.
+ * Decrement the number of classic groups.
*
- * @param state the consumer group state.
+ * @param state the classic group state.
*/
- public void decrementNumGenericGroups(GenericGroupState state) {
- AtomicLong counter = genericGroupGauges.get(state);
+ public void decrementNumClassicGroups(ClassicGroupState state) {
+ AtomicLong counter = classicGroupGauges.get(state);
if (counter != null) {
counter.decrementAndGet();
}
@@ -192,14 +192,14 @@
}
/**
- * Obtain the number of generic groups in the specified state.
+ * Obtain the number of classic groups in the specified state.
*
- * @param state The generic group state.
+ * @param state The classic group state.
*
- * @return The number of generic groups in `state`.
+ * @return The number of classic groups in `state`.
*/
- public long numGenericGroups(GenericGroupState state) {
- AtomicLong counter = genericGroupGauges.get(state);
+ public long numClassicGroups(ClassicGroupState state) {
+ AtomicLong counter = classicGroupGauges.get(state);
if (counter != null) {
return counter.get();
}
@@ -207,11 +207,11 @@
}
/**
- * @return The total number of generic groups.
+ * @return The total number of classic groups.
*/
- public long numGenericGroups() {
- return genericGroupGauges.values().stream()
- .mapToLong(AtomicLong::get).sum();
+ public long numClassicGroups() {
+ return classicGroupGauges.values().stream()
+ .mapToLong(AtomicLong::get).sum();
}
/**
@@ -230,7 +230,7 @@
}
/**
- * @return The total number of generic groups.
+ * @return The total number of consumer groups.
*/
public long numConsumerGroups() {
return consumerGroupGauges.values().stream()
@@ -268,9 +268,9 @@
gaugeCounter.atomicLong.set(value);
});
- synchronized (numGenericGroupsTimelineCounter.timelineLong) {
- long value = numGenericGroupsTimelineCounter.timelineLong.get(offset);
- numGenericGroupsTimelineCounter.atomicLong.set(value);
+ synchronized (numClassicGroupsTimelineCounter.timelineLong) {
+ long value = numClassicGroupsTimelineCounter.timelineLong.get(offset);
+ numClassicGroupsTimelineCounter.atomicLong.set(value);
}
synchronized (numOffsetsTimelineGaugeCounter.timelineLong) {
@@ -280,51 +280,51 @@
}
/**
- * Called when a generic group's state has changed. Increment/decrement
+ * Called when a classic group's state has changed. Increment/decrement
* the counter accordingly.
*
* @param oldState The previous state. null value means that it's a new group.
* @param newState The next state. null value means that the group has been removed.
*/
- public void onGenericGroupStateTransition(
- GenericGroupState oldState,
- GenericGroupState newState
+ public void onClassicGroupStateTransition(
+ ClassicGroupState oldState,
+ ClassicGroupState newState
) {
if (newState != null) {
switch (newState) {
case PREPARING_REBALANCE:
- incrementNumGenericGroups(GenericGroupState.PREPARING_REBALANCE);
+ incrementNumClassicGroups(ClassicGroupState.PREPARING_REBALANCE);
break;
case COMPLETING_REBALANCE:
- incrementNumGenericGroups(GenericGroupState.COMPLETING_REBALANCE);
+ incrementNumClassicGroups(ClassicGroupState.COMPLETING_REBALANCE);
break;
case STABLE:
- incrementNumGenericGroups(GenericGroupState.STABLE);
+ incrementNumClassicGroups(ClassicGroupState.STABLE);
break;
case DEAD:
- incrementNumGenericGroups(GenericGroupState.DEAD);
+ incrementNumClassicGroups(ClassicGroupState.DEAD);
break;
case EMPTY:
- incrementNumGenericGroups(GenericGroupState.EMPTY);
+ incrementNumClassicGroups(ClassicGroupState.EMPTY);
}
}
if (oldState != null) {
switch (oldState) {
case PREPARING_REBALANCE:
- decrementNumGenericGroups(GenericGroupState.PREPARING_REBALANCE);
+ decrementNumClassicGroups(ClassicGroupState.PREPARING_REBALANCE);
break;
case COMPLETING_REBALANCE:
- decrementNumGenericGroups(GenericGroupState.COMPLETING_REBALANCE);
+ decrementNumClassicGroups(ClassicGroupState.COMPLETING_REBALANCE);
break;
case STABLE:
- decrementNumGenericGroups(GenericGroupState.STABLE);
+ decrementNumClassicGroups(ClassicGroupState.STABLE);
break;
case DEAD:
- decrementNumGenericGroups(GenericGroupState.DEAD);
+ decrementNumClassicGroups(ClassicGroupState.DEAD);
break;
case EMPTY:
- decrementNumGenericGroups(GenericGroupState.EMPTY);
+ decrementNumClassicGroups(ClassicGroupState.EMPTY);
}
}
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java
index 5d2a3bf..10ce9f5 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfigTest.java
@@ -53,11 +53,11 @@
assertEquals(Collections.singletonList(assignor), config.consumerGroupAssignors);
assertEquals(2222, config.offsetsTopicSegmentBytes);
assertEquals(3333, config.offsetMetadataMaxSize);
- assertEquals(60, config.genericGroupMaxSize);
- assertEquals(3000, config.genericGroupInitialRebalanceDelayMs);
- assertEquals(5 * 60 * 1000, config.genericGroupNewMemberJoinTimeoutMs);
- assertEquals(120, config.genericGroupMinSessionTimeoutMs);
- assertEquals(10 * 60 * 1000, config.genericGroupMaxSessionTimeoutMs);
+ assertEquals(60, config.classicGroupMaxSize);
+ assertEquals(3000, config.classicGroupInitialRebalanceDelayMs);
+ assertEquals(5 * 60 * 1000, config.classicGroupNewMemberJoinTimeoutMs);
+ assertEquals(120, config.classicGroupMinSessionTimeoutMs);
+ assertEquals(10 * 60 * 1000, config.classicGroupMaxSessionTimeoutMs);
assertEquals(10 * 60 * 1000, config.offsetsRetentionCheckIntervalMs);
assertEquals(24 * 60 * 60 * 1000L, config.offsetsRetentionMs);
assertEquals(5000, config.offsetCommitTimeoutMs);
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java
index b9b162d..0d27cdb 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java
@@ -355,7 +355,7 @@
service.startup(() -> 1);
when(runtime.scheduleWriteOperation(
- ArgumentMatchers.eq("generic-group-join"),
+ ArgumentMatchers.eq("classic-group-join"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.eq(Duration.ofMillis(5000)),
ArgumentMatchers.any()
@@ -388,7 +388,7 @@
service.startup(() -> 1);
when(runtime.scheduleWriteOperation(
- ArgumentMatchers.eq("generic-group-join"),
+ ArgumentMatchers.eq("classic-group-join"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.eq(Duration.ofMillis(5000)),
ArgumentMatchers.any()
@@ -495,7 +495,7 @@
service.startup(() -> 1);
when(runtime.scheduleWriteOperation(
- ArgumentMatchers.eq("generic-group-sync"),
+ ArgumentMatchers.eq("classic-group-sync"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.eq(Duration.ofMillis(5000)),
ArgumentMatchers.any()
@@ -528,7 +528,7 @@
service.startup(() -> 1);
when(runtime.scheduleWriteOperation(
- ArgumentMatchers.eq("generic-group-sync"),
+ ArgumentMatchers.eq("classic-group-sync"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.eq(Duration.ofMillis(5000)),
ArgumentMatchers.any()
@@ -619,7 +619,7 @@
service.startup(() -> 1);
when(runtime.scheduleReadOperation(
- ArgumentMatchers.eq("generic-group-heartbeat"),
+ ArgumentMatchers.eq("classic-group-heartbeat"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.any()
)).thenReturn(CompletableFuture.completedFuture(
@@ -651,7 +651,7 @@
service.startup(() -> 1);
when(runtime.scheduleReadOperation(
- ArgumentMatchers.eq("generic-group-heartbeat"),
+ ArgumentMatchers.eq("classic-group-heartbeat"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.any()
)).thenReturn(FutureUtils.failedFuture(
@@ -683,7 +683,7 @@
service.startup(() -> 1);
when(runtime.scheduleReadOperation(
- ArgumentMatchers.eq("generic-group-heartbeat"),
+ ArgumentMatchers.eq("classic-group-heartbeat"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.any()
)).thenReturn(FutureUtils.failedFuture(
@@ -1240,7 +1240,7 @@
service.startup(() -> 1);
when(runtime.scheduleWriteOperation(
- ArgumentMatchers.eq("generic-group-leave"),
+ ArgumentMatchers.eq("classic-group-leave"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.eq(Duration.ofMillis(5000)),
ArgumentMatchers.any()
@@ -1281,7 +1281,7 @@
service.startup(() -> 1);
when(runtime.scheduleWriteOperation(
- ArgumentMatchers.eq("generic-group-leave"),
+ ArgumentMatchers.eq("classic-group-leave"),
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
ArgumentMatchers.eq(Duration.ofMillis(5000)),
ArgumentMatchers.any()
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
index d7f2a57..832398f 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
@@ -90,9 +90,9 @@
import org.apache.kafka.coordinator.group.generated.ConsumerGroupTargetAssignmentMetadataValue;
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
-import org.apache.kafka.coordinator.group.generic.GenericGroup;
-import org.apache.kafka.coordinator.group.generic.GenericGroupMember;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroup;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
import org.apache.kafka.image.MetadataDelta;
@@ -138,17 +138,17 @@
import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupRevocationTimeoutKey;
import static org.apache.kafka.coordinator.group.GroupMetadataManager.consumerGroupSessionTimeoutKey;
import static org.apache.kafka.coordinator.group.GroupMetadataManager.EMPTY_RESULT;
-import static org.apache.kafka.coordinator.group.GroupMetadataManager.genericGroupHeartbeatKey;
-import static org.apache.kafka.coordinator.group.GroupMetadataManager.genericGroupSyncKey;
+import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupHeartbeatKey;
+import static org.apache.kafka.coordinator.group.GroupMetadataManager.classicGroupSyncKey;
import static org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
-import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME;
+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.EMPTY;
+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.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.GENERIC_GROUP_REBALANCES_SENSOR_NAME;
+import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_REBALANCES_SENSOR_NAME;
import static org.junit.jupiter.api.AssertionFailureBuilder.assertionFailure;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -322,11 +322,11 @@
private List<ConsumerGroupBuilder> consumerGroupBuilders = new ArrayList<>();
private int consumerGroupMaxSize = Integer.MAX_VALUE;
private int consumerGroupMetadataRefreshIntervalMs = Integer.MAX_VALUE;
- private int genericGroupMaxSize = Integer.MAX_VALUE;
- private int genericGroupInitialRebalanceDelayMs = 3000;
- final private int genericGroupNewMemberJoinTimeoutMs = 5 * 60 * 1000;
- private int genericGroupMinSessionTimeoutMs = 10;
- private int genericGroupMaxSessionTimeoutMs = 10 * 60 * 1000;
+ private int classicGroupMaxSize = Integer.MAX_VALUE;
+ private int classicGroupInitialRebalanceDelayMs = 3000;
+ final private int classicGroupNewMemberJoinTimeoutMs = 5 * 60 * 1000;
+ private int classicGroupMinSessionTimeoutMs = 10;
+ private int classicGroupMaxSessionTimeoutMs = 10 * 60 * 1000;
private GroupCoordinatorMetricsShard metrics = mock(GroupCoordinatorMetricsShard.class);
public Builder withMetadataImage(MetadataImage metadataImage) {
@@ -354,23 +354,23 @@
return this;
}
- public Builder withGenericGroupMaxSize(int genericGroupMaxSize) {
- this.genericGroupMaxSize = genericGroupMaxSize;
+ public Builder withClassicGroupMaxSize(int classicGroupMaxSize) {
+ this.classicGroupMaxSize = classicGroupMaxSize;
return this;
}
- public Builder withGenericGroupInitialRebalanceDelayMs(int genericGroupInitialRebalanceDelayMs) {
- this.genericGroupInitialRebalanceDelayMs = genericGroupInitialRebalanceDelayMs;
+ public Builder withClassicGroupInitialRebalanceDelayMs(int classicGroupInitialRebalanceDelayMs) {
+ this.classicGroupInitialRebalanceDelayMs = classicGroupInitialRebalanceDelayMs;
return this;
}
- public Builder withGenericGroupMinSessionTimeoutMs(int genericGroupMinSessionTimeoutMs) {
- this.genericGroupMinSessionTimeoutMs = genericGroupMinSessionTimeoutMs;
+ public Builder withClassicGroupMinSessionTimeoutMs(int classicGroupMinSessionTimeoutMs) {
+ this.classicGroupMinSessionTimeoutMs = classicGroupMinSessionTimeoutMs;
return this;
}
- public Builder withGenericGroupMaxSessionTimeoutMs(int genericGroupMaxSessionTimeoutMs) {
- this.genericGroupMaxSessionTimeoutMs = genericGroupMaxSessionTimeoutMs;
+ public Builder withClassicGroupMaxSessionTimeoutMs(int classicGroupMaxSessionTimeoutMs) {
+ this.classicGroupMaxSessionTimeoutMs = classicGroupMaxSessionTimeoutMs;
return this;
}
@@ -394,15 +394,15 @@
.withConsumerGroupMaxSize(consumerGroupMaxSize)
.withConsumerGroupAssignors(consumerGroupAssignors)
.withConsumerGroupMetadataRefreshIntervalMs(consumerGroupMetadataRefreshIntervalMs)
- .withGenericGroupMaxSize(genericGroupMaxSize)
- .withGenericGroupMinSessionTimeoutMs(genericGroupMinSessionTimeoutMs)
- .withGenericGroupMaxSessionTimeoutMs(genericGroupMaxSessionTimeoutMs)
- .withGenericGroupInitialRebalanceDelayMs(genericGroupInitialRebalanceDelayMs)
- .withGenericGroupNewMemberJoinTimeoutMs(genericGroupNewMemberJoinTimeoutMs)
+ .withClassicGroupMaxSize(classicGroupMaxSize)
+ .withClassicGroupMinSessionTimeoutMs(classicGroupMinSessionTimeoutMs)
+ .withClassicGroupMaxSessionTimeoutMs(classicGroupMaxSessionTimeoutMs)
+ .withClassicGroupInitialRebalanceDelayMs(classicGroupInitialRebalanceDelayMs)
+ .withClassicGroupNewMemberJoinTimeoutMs(classicGroupNewMemberJoinTimeoutMs)
.withGroupCoordinatorMetricsShard(metrics)
.build(),
- genericGroupInitialRebalanceDelayMs,
- genericGroupNewMemberJoinTimeoutMs
+ classicGroupInitialRebalanceDelayMs,
+ classicGroupNewMemberJoinTimeoutMs
);
consumerGroupBuilders.forEach(builder -> {
@@ -420,8 +420,8 @@
final SnapshotRegistry snapshotRegistry;
final GroupCoordinatorMetricsShard metrics;
final GroupMetadataManager groupMetadataManager;
- final int genericGroupInitialRebalanceDelayMs;
- final int genericGroupNewMemberJoinTimeoutMs;
+ final int classicGroupInitialRebalanceDelayMs;
+ final int classicGroupNewMemberJoinTimeoutMs;
long lastCommittedOffset = 0L;
long lastWrittenOffset = 0L;
@@ -432,16 +432,16 @@
SnapshotRegistry snapshotRegistry,
GroupCoordinatorMetricsShard metrics,
GroupMetadataManager groupMetadataManager,
- int genericGroupInitialRebalanceDelayMs,
- int genericGroupNewMemberJoinTimeoutMs
+ int classicGroupInitialRebalanceDelayMs,
+ int classicGroupNewMemberJoinTimeoutMs
) {
this.time = time;
this.timer = timer;
this.snapshotRegistry = snapshotRegistry;
this.metrics = metrics;
this.groupMetadataManager = groupMetadataManager;
- this.genericGroupInitialRebalanceDelayMs = genericGroupInitialRebalanceDelayMs;
- this.genericGroupNewMemberJoinTimeoutMs = genericGroupNewMemberJoinTimeoutMs;
+ this.classicGroupInitialRebalanceDelayMs = classicGroupInitialRebalanceDelayMs;
+ this.classicGroupNewMemberJoinTimeoutMs = classicGroupNewMemberJoinTimeoutMs;
snapshotRegistry.getOrCreateSnapshot(lastWrittenOffset);
}
@@ -555,24 +555,24 @@
assertNull(timeout);
}
- GenericGroup createGenericGroup(String groupId) {
- return groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, true);
+ ClassicGroup createClassicGroup(String groupId) {
+ return groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, true);
}
- public JoinResult sendGenericGroupJoin(
+ public JoinResult sendClassicGroupJoin(
JoinGroupRequestData request
) {
- return sendGenericGroupJoin(request, false);
+ return sendClassicGroupJoin(request, false);
}
- public JoinResult sendGenericGroupJoin(
+ public JoinResult sendClassicGroupJoin(
JoinGroupRequestData request,
boolean requireKnownMemberId
) {
- return sendGenericGroupJoin(request, requireKnownMemberId, false);
+ return sendClassicGroupJoin(request, requireKnownMemberId, false);
}
- public JoinResult sendGenericGroupJoin(
+ public JoinResult sendClassicGroupJoin(
JoinGroupRequestData request,
boolean requireKnownMemberId,
boolean supportSkippingAssignment
@@ -605,7 +605,7 @@
);
CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
- CoordinatorResult<Void, Record> coordinatorResult = groupMetadataManager.genericGroupJoin(
+ CoordinatorResult<Void, Record> coordinatorResult = groupMetadataManager.classicGroupJoin(
context,
request,
responseFuture
@@ -614,13 +614,13 @@
return new JoinResult(responseFuture, coordinatorResult);
}
- public JoinGroupResponseData joinGenericGroupAsDynamicMemberAndCompleteRebalance(
+ public JoinGroupResponseData joinClassicGroupAsDynamicMemberAndCompleteRebalance(
String groupId
) throws Exception {
- GenericGroup group = createGenericGroup(groupId);
+ ClassicGroup group = createClassicGroup(groupId);
JoinGroupResponseData leaderJoinResponse =
- joinGenericGroupAsDynamicMemberAndCompleteJoin(new JoinGroupRequestBuilder()
+ joinClassicGroupAsDynamicMemberAndCompleteJoin(new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
.withDefaultProtocolTypeAndProtocols()
@@ -631,7 +631,7 @@
assertEquals(1, leaderJoinResponse.generationId());
assertTrue(group.isInState(COMPLETING_REBALANCE));
- SyncResult syncResult = sendGenericGroupSync(new SyncGroupRequestBuilder()
+ SyncResult syncResult = sendClassicGroupSync(new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(leaderJoinResponse.memberId())
.withGenerationId(leaderJoinResponse.generationId())
@@ -651,7 +651,7 @@
return leaderJoinResponse;
}
- public JoinGroupResponseData joinGenericGroupAsDynamicMemberAndCompleteJoin(
+ public JoinGroupResponseData joinClassicGroupAsDynamicMemberAndCompleteJoin(
JoinGroupRequestData request
) throws ExecutionException, InterruptedException {
boolean requireKnownMemberId = true;
@@ -659,7 +659,7 @@
if (request.memberId().equals(UNKNOWN_MEMBER_ID)) {
// Since member id is required, we need another round to get the successful join group result.
- JoinResult firstJoinResult = sendGenericGroupJoin(
+ JoinResult firstJoinResult = sendClassicGroupJoin(
request,
requireKnownMemberId
);
@@ -679,13 +679,13 @@
.setRebalanceTimeoutMs(request.rebalanceTimeoutMs())
.setReason(request.reason());
- JoinResult secondJoinResult = sendGenericGroupJoin(
+ JoinResult secondJoinResult = sendClassicGroupJoin(
secondRequest,
requireKnownMemberId
);
assertTrue(secondJoinResult.records.isEmpty());
- List<ExpiredTimeout<Void, Record>> timeouts = sleep(genericGroupInitialRebalanceDelayMs);
+ List<ExpiredTimeout<Void, Record>> timeouts = sleep(classicGroupInitialRebalanceDelayMs);
assertEquals(1, timeouts.size());
assertTrue(secondJoinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), secondJoinResult.joinFuture.get().errorCode());
@@ -693,31 +693,31 @@
return secondJoinResult.joinFuture.get();
}
- public JoinGroupResponseData joinGenericGroupAndCompleteJoin(
+ public JoinGroupResponseData joinClassicGroupAndCompleteJoin(
JoinGroupRequestData request,
boolean requireKnownMemberId,
boolean supportSkippingAssignment
) throws ExecutionException, InterruptedException {
- return joinGenericGroupAndCompleteJoin(
+ return joinClassicGroupAndCompleteJoin(
request,
requireKnownMemberId,
supportSkippingAssignment,
- genericGroupInitialRebalanceDelayMs
+ classicGroupInitialRebalanceDelayMs
);
}
- public JoinGroupResponseData joinGenericGroupAndCompleteJoin(
+ public JoinGroupResponseData joinClassicGroupAndCompleteJoin(
JoinGroupRequestData request,
boolean requireKnownMemberId,
boolean supportSkippingAssignment,
int advanceClockMs
) throws ExecutionException, InterruptedException {
if (requireKnownMemberId && request.groupInstanceId().isEmpty()) {
- return joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ return joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
}
try {
- JoinResult joinResult = sendGenericGroupJoin(
+ JoinResult joinResult = sendClassicGroupJoin(
request,
requireKnownMemberId,
supportSkippingAssignment
@@ -733,7 +733,7 @@
return null;
}
- public SyncResult sendGenericGroupSync(SyncGroupRequestData request) {
+ public SyncResult sendClassicGroupSync(SyncGroupRequestData request) {
RequestContext context = new RequestContext(
new RequestHeader(
ApiKeys.SYNC_GROUP,
@@ -752,7 +752,7 @@
CompletableFuture<SyncGroupResponseData> responseFuture = new CompletableFuture<>();
- CoordinatorResult<Void, Record> coordinatorResult = groupMetadataManager.genericGroupSync(
+ CoordinatorResult<Void, Record> coordinatorResult = groupMetadataManager.classicGroupSync(
context,
request,
responseFuture
@@ -782,7 +782,7 @@
int rebalanceTimeoutMs,
int sessionTimeoutMs
) throws Exception {
- GenericGroup group = createGenericGroup("group-id");
+ ClassicGroup group = createClassicGroup("group-id");
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId(groupId)
@@ -794,8 +794,8 @@
.withSessionTimeoutMs(sessionTimeoutMs)
.build();
- JoinResult leaderJoinResult = sendGenericGroupJoin(joinRequest);
- JoinResult followerJoinResult = sendGenericGroupJoin(joinRequest.setGroupInstanceId(followerInstanceId));
+ JoinResult leaderJoinResult = sendClassicGroupJoin(joinRequest);
+ JoinResult followerJoinResult = sendClassicGroupJoin(joinRequest.setGroupInstanceId(followerInstanceId));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(followerJoinResult.records.isEmpty());
@@ -805,8 +805,8 @@
// The goal for two timer advance is to let first group initial join complete and set newMemberAdded flag to false. Next advance is
// to trigger the rebalance as needed for follower delayed join. One large time advance won't help because we could only populate one
// delayed join from purgatory and the new delayed op is created at that time and never be triggered.
- assertNoOrEmptyResult(sleep(genericGroupInitialRebalanceDelayMs));
- assertNoOrEmptyResult(sleep(genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(sleep(classicGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(sleep(classicGroupInitialRebalanceDelayMs));
assertTrue(leaderJoinResult.joinFuture.isDone());
assertTrue(followerJoinResult.joinFuture.isDone());
@@ -834,7 +834,7 @@
.withAssignment(assignment)
.build();
- SyncResult leaderSyncResult = sendGenericGroupSync(syncRequest);
+ SyncResult leaderSyncResult = sendClassicGroupSync(syncRequest);
// The generated record should contain the new assignment.
Map<String, byte[]> groupAssignment = assignment.stream().collect(Collectors.toMap(
@@ -853,7 +853,7 @@
assertEquals(Errors.NONE.code(), leaderSyncResult.syncFuture.get().errorCode());
assertTrue(group.isInState(STABLE));
- SyncResult followerSyncResult = sendGenericGroupSync(
+ SyncResult followerSyncResult = sendClassicGroupSync(
syncRequest.setGroupInstanceId(followerInstanceId)
.setMemberId(followerId)
.setAssignments(Collections.emptyList())
@@ -876,7 +876,7 @@
);
}
- public PendingMemberGroupResult setupGroupWithPendingMember(GenericGroup group) throws Exception {
+ public PendingMemberGroupResult setupGroupWithPendingMember(ClassicGroup group) throws Exception {
// Add the first member
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -887,7 +887,7 @@
.build();
JoinGroupResponseData leaderJoinResponse =
- joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest);
+ joinClassicGroupAsDynamicMemberAndCompleteJoin(joinRequest);
List<SyncGroupRequestAssignment> assignment = new ArrayList<>();
assignment.add(new SyncGroupRequestAssignment().setMemberId(leaderJoinResponse.memberId()));
@@ -898,7 +898,7 @@
.withAssignment(assignment)
.build();
- SyncResult syncResult = sendGenericGroupSync(syncRequest);
+ SyncResult syncResult = sendClassicGroupSync(syncRequest);
// Now the group is stable, with the one member that joined above
assertEquals(
@@ -912,14 +912,14 @@
assertEquals(Errors.NONE.code(), syncResult.syncFuture.get().errorCode());
// Start the join for the second member
- JoinResult followerJoinResult = sendGenericGroupJoin(
+ JoinResult followerJoinResult = sendClassicGroupJoin(
joinRequest.setMemberId(UNKNOWN_MEMBER_ID)
);
assertTrue(followerJoinResult.records.isEmpty());
assertFalse(followerJoinResult.joinFuture.isDone());
- JoinResult leaderJoinResult = sendGenericGroupJoin(
+ JoinResult leaderJoinResult = sendClassicGroupJoin(
joinRequest.setMemberId(leaderJoinResponse.memberId())
);
@@ -937,7 +937,7 @@
String followerId = followerJoinResult.joinFuture.get().memberId();
// Stabilize the group
- syncResult = sendGenericGroupSync(syncRequest.setGenerationId(nextGenerationId));
+ syncResult = sendClassicGroupSync(syncRequest.setGenerationId(nextGenerationId));
assertEquals(
Collections.singletonList(newGroupMetadataRecordWithCurrentState(group, MetadataVersion.latest())),
@@ -951,7 +951,7 @@
assertTrue(group.isInState(STABLE));
// Re-join an existing member, to transition the group to PreparingRebalance state.
- leaderJoinResult = sendGenericGroupJoin(
+ leaderJoinResult = sendClassicGroupJoin(
joinRequest.setMemberId(leaderJoinResponse.memberId()));
assertTrue(leaderJoinResult.records.isEmpty());
@@ -959,7 +959,7 @@
assertTrue(group.isInState(PREPARING_REBALANCE));
// Create a pending member in the group
- JoinResult pendingMemberJoinResult = sendGenericGroupJoin(
+ JoinResult pendingMemberJoinResult = sendClassicGroupJoin(
joinRequest
.setMemberId(UNKNOWN_MEMBER_ID)
.setSessionTimeoutMs(2500),
@@ -972,7 +972,7 @@
assertEquals(1, group.numPendingJoinMembers());
// Re-join the second existing member
- followerJoinResult = sendGenericGroupJoin(
+ followerJoinResult = sendClassicGroupJoin(
joinRequest.setMemberId(followerId).setSessionTimeoutMs(5000)
);
@@ -989,9 +989,9 @@
);
}
- public void verifySessionExpiration(GenericGroup group, int timeoutMs) {
+ public void verifySessionExpiration(ClassicGroup group, int timeoutMs) {
Set<String> expectedHeartbeatKeys = group.allMembers().stream()
- .map(member -> genericGroupHeartbeatKey(group.groupId(), member.memberId())).collect(Collectors.toSet());
+ .map(member -> classicGroupHeartbeatKey(group.groupId(), member.memberId())).collect(Collectors.toSet());
// Member should be removed as session expires.
List<ExpiredTimeout<Void, Record>> timeouts = sleep(timeoutMs);
@@ -1018,7 +1018,7 @@
assertEquals(0, group.size());
}
- public HeartbeatResponseData sendGenericGroupHeartbeat(
+ public HeartbeatResponseData sendClassicGroupHeartbeat(
HeartbeatRequestData request
) {
RequestContext context = new RequestContext(
@@ -1037,7 +1037,7 @@
false
);
- return groupMetadataManager.genericGroupHeartbeat(
+ return groupMetadataManager.classicGroupHeartbeat(
context,
request
);
@@ -1066,9 +1066,9 @@
.setGenerationId(joinResponse.generationId());
if (expectedError == Errors.UNKNOWN_MEMBER_ID) {
- assertThrows(UnknownMemberIdException.class, () -> sendGenericGroupHeartbeat(request));
+ assertThrows(UnknownMemberIdException.class, () -> sendClassicGroupHeartbeat(request));
} else {
- HeartbeatResponseData response = sendGenericGroupHeartbeat(request);
+ HeartbeatResponseData response = sendClassicGroupHeartbeat(request);
assertEquals(expectedError.code(), response.errorCode());
}
}
@@ -1079,7 +1079,7 @@
int rebalanceTimeoutMs,
int sessionTimeoutMs
) {
- GenericGroup group = createGenericGroup(groupId);
+ ClassicGroup group = createClassicGroup(groupId);
boolean requireKnownMemberId = true;
// First join requests
@@ -1092,7 +1092,7 @@
.build();
List<String> memberIds = IntStream.range(0, numMembers).mapToObj(i -> {
- JoinResult joinResult = sendGenericGroupJoin(request, requireKnownMemberId);
+ JoinResult joinResult = sendClassicGroupJoin(request, requireKnownMemberId);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -1107,7 +1107,7 @@
// Second join requests
List<CompletableFuture<JoinGroupResponseData>> secondJoinFutures = IntStream.range(0, numMembers).mapToObj(i -> {
- JoinResult joinResult = sendGenericGroupJoin(request.setMemberId(memberIds.get(i)), requireKnownMemberId);
+ JoinResult joinResult = sendClassicGroupJoin(request.setMemberId(memberIds.get(i)), requireKnownMemberId);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
@@ -1116,7 +1116,7 @@
}).collect(Collectors.toList());
// Advance clock by initial rebalance delay.
- assertNoOrEmptyResult(sleep(genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(sleep(classicGroupInitialRebalanceDelayMs));
secondJoinFutures.forEach(future -> assertFalse(future.isDone()));
// Advance clock by rebalance timeout to complete join phase.
assertNoOrEmptyResult(sleep(rebalanceTimeoutMs));
@@ -1138,7 +1138,7 @@
return joinResponses;
}
- public CoordinatorResult<LeaveGroupResponseData, Record> sendGenericGroupLeave(
+ public CoordinatorResult<LeaveGroupResponseData, Record> sendClassicGroupLeave(
LeaveGroupRequestData request
) {
RequestContext context = new RequestContext(
@@ -1157,7 +1157,7 @@
false
);
- return groupMetadataManager.genericGroupLeave(context, request);
+ return groupMetadataManager.classicGroupLeave(context, request);
}
private void verifyDescribeGroupsReturnsDeadGroup(String groupId) {
@@ -4693,7 +4693,7 @@
}
@Test
- public void testGenerateRecordsOnNewGenericGroup() throws Exception {
+ public void testGenerateRecordsOnNewClassicGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
@@ -4703,11 +4703,11 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(request, true);
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.MEMBER_ID_REQUIRED.code(), joinResult.joinFuture.get().errorCode());
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
assertEquals(
Collections.singletonList(RecordHelpers.newEmptyGroupMetadataRecord(group, MetadataVersion.latest())),
@@ -4716,7 +4716,7 @@
}
@Test
- public void testGenerateRecordsOnNewGenericGroupFailureTransformsError() throws Exception {
+ public void testGenerateRecordsOnNewClassicGroupFailureTransformsError() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
@@ -4727,7 +4727,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(request, true);
assertFalse(joinResult.joinFuture.isDone());
// Simulate a failed write to the log.
@@ -4745,7 +4745,7 @@
byte[] subscription = ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
Collections.singletonList("foo"))).array();
List<GroupMetadataValue.MemberMetadata> members = new ArrayList<>();
- List<GenericGroupMember> expectedMembers = new ArrayList<>();
+ List<ClassicGroupMember> expectedMembers = new ArrayList<>();
JoinGroupRequestProtocolCollection expectedProtocols = new JoinGroupRequestProtocolCollection(0);
expectedProtocols.add(new JoinGroupRequestProtocol()
.setName("range")
@@ -4763,7 +4763,7 @@
.setRebalanceTimeout(useDefaultRebalanceTimeout ? -1 : 9000)
);
- expectedMembers.add(new GenericGroupMember(
+ expectedMembers.add(new ClassicGroupMember(
"member-" + i,
Optional.of("group-instance-id-" + i),
"client-" + i,
@@ -4787,9 +4787,9 @@
MetadataVersion.latest());
context.replay(groupMetadataRecord);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
- GenericGroup expectedGroup = new GenericGroup(
+ ClassicGroup expectedGroup = new ClassicGroup(
new LogContext(),
"group-id",
STABLE,
@@ -4814,13 +4814,13 @@
assertEquals(expectedGroup.leaderOrNull(), group.leaderOrNull());
assertEquals(expectedGroup.currentState(), group.currentState());
assertEquals(expectedGroup.currentStateTimestampOrDefault(), group.currentStateTimestampOrDefault());
- assertEquals(expectedGroup.currentGenericGroupMembers(), group.currentGenericGroupMembers());
+ assertEquals(expectedGroup.currentClassicGroupMembers(), group.currentClassicGroupMembers());
}
@Test
public void testOnLoadedExceedGroupMaxSizeTriggersRebalance() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSize(1)
+ .withClassicGroupMaxSize(1)
.build();
byte[] subscription = ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
@@ -4852,14 +4852,14 @@
context.replay(groupMetadataRecord);
context.groupMetadataManager.onLoaded();
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
assertTrue(group.isInState(PREPARING_REBALANCE));
assertEquals(2, group.size());
}
@Test
- public void testOnLoadedSchedulesGenericGroupMemberHeartbeats() {
+ public void testOnLoadedSchedulesClassicGroupMemberHeartbeats() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
@@ -4895,7 +4895,7 @@
IntStream.range(0, 2).forEach(i -> {
ScheduledTimeout<Void, Record> timeout = context.timer.timeout(
- genericGroupHeartbeatKey("group-id", "member-1"));
+ classicGroupHeartbeatKey("group-id", "member-1"));
assertNotNull(timeout);
assertEquals(context.time.milliseconds() + 4000, timeout.deadlineMs);
@@ -4905,9 +4905,9 @@
@Test
public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSize(10)
+ .withClassicGroupMaxSize(10)
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -4917,12 +4917,12 @@
.build();
IntStream.range(0, 10).forEach(i -> {
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertFalse(joinResult.joinFuture.isDone());
assertTrue(joinResult.records.isEmpty());
});
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), joinResult.joinFuture.get().errorCode());
@@ -4933,11 +4933,11 @@
boolean requiredKnownMemberId = true;
int groupMaxSize = 10;
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSize(groupMaxSize)
- .withGenericGroupInitialRebalanceDelayMs(50)
+ .withClassicGroupMaxSize(groupMaxSize)
+ .withClassicGroupInitialRebalanceDelayMs(50)
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -4947,12 +4947,12 @@
// First round of join requests. Generate member ids. All requests will be accepted
// as the group is still Empty.
- List<JoinResult> firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendGenericGroupJoin(
+ List<JoinResult> firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendClassicGroupJoin(
request,
requiredKnownMemberId
)).collect(Collectors.toList());
- List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundJoinResults, 0, Errors.MEMBER_ID_REQUIRED);
+ List<String> memberIds = verifyClassicGroupJoinResponses(firstRoundJoinResults, 0, Errors.MEMBER_ID_REQUIRED);
assertEquals(groupMaxSize + 1, memberIds.size());
assertEquals(0, group.size());
assertTrue(group.isInState(EMPTY));
@@ -4960,7 +4960,7 @@
// Second round of join requests with the generated member ids.
// One of them will fail, reaching group max size.
- List<JoinResult> secondRoundJoinResults = memberIds.stream().map(memberId -> context.sendGenericGroupJoin(
+ List<JoinResult> secondRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin(
request.setMemberId(memberId),
requiredKnownMemberId
)).collect(Collectors.toList());
@@ -4973,18 +4973,18 @@
// the join group phase will complete.
assertNoOrEmptyResult(context.sleep(50));
- verifyGenericGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+ verifyClassicGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
assertEquals(groupMaxSize, group.size());
assertEquals(0, group.numPendingJoinMembers());
assertTrue(group.isInState(COMPLETING_REBALANCE));
// Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
- List<JoinResult> thirdRoundJoinResults = memberIds.stream().map(memberId -> context.sendGenericGroupJoin(
+ List<JoinResult> thirdRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin(
request.setMemberId(memberId),
requiredKnownMemberId
)).collect(Collectors.toList());
- verifyGenericGroupJoinResponses(thirdRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+ verifyClassicGroupJoinResponses(thirdRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
}
@Test
@@ -4992,11 +4992,11 @@
boolean requiredKnownMemberId = false;
int groupMaxSize = 10;
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSize(groupMaxSize)
- .withGenericGroupInitialRebalanceDelayMs(50)
+ .withClassicGroupMaxSize(groupMaxSize)
+ .withClassicGroupInitialRebalanceDelayMs(50)
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5005,7 +5005,7 @@
.build();
// First round of join requests. This will trigger a rebalance.
- List<JoinResult> firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendGenericGroupJoin(
+ List<JoinResult> firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendClassicGroupJoin(
request,
requiredKnownMemberId
)).collect(Collectors.toList());
@@ -5022,15 +5022,15 @@
// we will complete the rebalance.
assertNoOrEmptyResult(context.sleep(50));
- List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+ List<String> memberIds = verifyClassicGroupJoinResponses(firstRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
// Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
- List<JoinResult> secondRoundJoinResults = memberIds.stream().map(memberId -> context.sendGenericGroupJoin(
+ List<JoinResult> secondRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin(
request.setMemberId(memberId),
requiredKnownMemberId
)).collect(Collectors.toList());
- verifyGenericGroupJoinResponses(secondRoundJoinResults, 10, Errors.GROUP_MAX_SIZE_REACHED);
+ verifyClassicGroupJoinResponses(secondRoundJoinResults, 10, Errors.GROUP_MAX_SIZE_REACHED);
assertEquals(groupMaxSize, group.size());
assertEquals(0, group.numAwaitingJoinResponse());
assertTrue(group.isInState(COMPLETING_REBALANCE));
@@ -5045,11 +5045,11 @@
.collect(Collectors.toList());
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSize(groupMaxSize)
- .withGenericGroupInitialRebalanceDelayMs(50)
+ .withClassicGroupMaxSize(groupMaxSize)
+ .withClassicGroupInitialRebalanceDelayMs(50)
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5059,7 +5059,7 @@
// First round of join requests. This will trigger a rebalance.
List<JoinResult> firstRoundJoinResults = groupInstanceIds.stream()
- .map(instanceId -> context.sendGenericGroupJoin(request.setGroupInstanceId(instanceId)))
+ .map(instanceId -> context.sendClassicGroupJoin(request.setGroupInstanceId(instanceId)))
.collect(Collectors.toList());
assertEquals(groupMaxSize, group.size());
@@ -5074,17 +5074,17 @@
// we will complete the rebalance.
assertNoOrEmptyResult(context.sleep(50));
- List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+ List<String> memberIds = verifyClassicGroupJoinResponses(firstRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
// Members which were accepted can rejoin, others are rejected, while
// completing rebalance
- List<JoinResult> secondRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendGenericGroupJoin(
+ List<JoinResult> secondRoundJoinResults = IntStream.range(0, groupMaxSize + 1).mapToObj(i -> context.sendClassicGroupJoin(
request
.setMemberId(memberIds.get(i))
.setGroupInstanceId(groupInstanceIds.get(i))
)).collect(Collectors.toList());
- verifyGenericGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+ verifyClassicGroupJoinResponses(secondRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
assertEquals(groupMaxSize, group.size());
assertEquals(0, group.numAwaitingJoinResponse());
assertTrue(group.isInState(COMPLETING_REBALANCE));
@@ -5095,11 +5095,11 @@
boolean requiredKnownMemberId = true;
int groupMaxSize = 10;
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSize(groupMaxSize)
- .withGenericGroupInitialRebalanceDelayMs(50)
+ .withClassicGroupMaxSize(groupMaxSize)
+ .withClassicGroupInitialRebalanceDelayMs(50)
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5109,20 +5109,20 @@
// First round of join requests. Generate member ids.
List<JoinResult> firstRoundJoinResults = IntStream.range(0, groupMaxSize + 1)
- .mapToObj(__ -> context.sendGenericGroupJoin(request, requiredKnownMemberId))
+ .mapToObj(__ -> context.sendClassicGroupJoin(request, requiredKnownMemberId))
.collect(Collectors.toList());
assertEquals(0, group.size());
assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
assertTrue(group.isInState(EMPTY));
- List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundJoinResults, 0, Errors.MEMBER_ID_REQUIRED);
+ List<String> memberIds = verifyClassicGroupJoinResponses(firstRoundJoinResults, 0, Errors.MEMBER_ID_REQUIRED);
assertEquals(groupMaxSize + 1, memberIds.size());
// Second round of join requests with the generated member ids.
// One of them will fail, reaching group max size.
memberIds.forEach(memberId -> {
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request.setMemberId(memberId),
requiredKnownMemberId
);
@@ -5134,7 +5134,7 @@
assertTrue(group.isInState(PREPARING_REBALANCE));
// Members can rejoin while rebalancing
- List<JoinResult> thirdRoundJoinResults = memberIds.stream().map(memberId -> context.sendGenericGroupJoin(
+ List<JoinResult> thirdRoundJoinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin(
request.setMemberId(memberId),
requiredKnownMemberId
)).collect(Collectors.toList());
@@ -5147,7 +5147,7 @@
// we will complete the rebalance.
assertNoOrEmptyResult(context.sleep(50));
- verifyGenericGroupJoinResponses(thirdRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+ verifyClassicGroupJoinResponses(thirdRoundJoinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
assertEquals(groupMaxSize, group.size());
assertEquals(0, group.numAwaitingJoinResponse());
assertTrue(group.isInState(COMPLETING_REBALANCE));
@@ -5158,12 +5158,12 @@
int groupMaxSize = 10;
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSize(groupMaxSize)
- .withGenericGroupInitialRebalanceDelayMs(50)
+ .withClassicGroupMaxSize(groupMaxSize)
+ .withClassicGroupInitialRebalanceDelayMs(50)
.build();
// Create a group and add members that exceed the group max size.
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
.mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
@@ -5171,7 +5171,7 @@
memberIds.forEach(memberId -> {
group.add(
- new GenericGroupMember(
+ new ClassicGroupMember(
memberId,
Optional.empty(),
"client-id",
@@ -5186,7 +5186,7 @@
context.groupMetadataManager.prepareRebalance(group, "test");
- List<JoinResult> joinResults = memberIds.stream().map(memberId -> context.sendGenericGroupJoin(
+ List<JoinResult> joinResults = memberIds.stream().map(memberId -> context.sendClassicGroupJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(memberId)
@@ -5202,7 +5202,7 @@
// Advance clock by rebalance timeout to complete join phase.
assertNoOrEmptyResult(context.sleep(10000));
- verifyGenericGroupJoinResponses(joinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+ verifyClassicGroupJoinResponses(joinResults, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
assertEquals(groupMaxSize, group.size());
assertTrue(group.isInState(COMPLETING_REBALANCE));
@@ -5218,7 +5218,7 @@
public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
int minSessionTimeout = 50;
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+ .withClassicGroupMinSessionTimeoutMs(minSessionTimeout)
.build();
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -5227,7 +5227,7 @@
.withSessionTimeoutMs(minSessionTimeout - 1)
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.joinFuture.isDone());
assertTrue(joinResult.records.isEmpty());
assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), joinResult.joinFuture.get().errorCode());
@@ -5237,7 +5237,7 @@
public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
int maxSessionTimeout = 50;
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+ .withClassicGroupMaxSessionTimeoutMs(maxSessionTimeout)
.build();
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -5246,7 +5246,7 @@
.withSessionTimeoutMs(maxSessionTimeout + 1)
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5263,7 +5263,7 @@
.withMemberId("member-id")
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5276,7 +5276,7 @@
.withGroupInstanceId("group-instance-id")
.build();
- joinResult = context.sendGenericGroupJoin(request);
+ joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5284,10 +5284,10 @@
}
@Test
- public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+ public void testClassicGroupJoinInconsistentProtocolType() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5295,7 +5295,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5304,7 +5304,7 @@
.withProtocols(toProtocols("range"))
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5315,7 +5315,7 @@
public void testJoinGroupWithEmptyProtocolType() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5324,13 +5324,13 @@
.withProtocols(toProtocols("range"))
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), joinResult.joinFuture.get().errorCode());
// Send as static member join.
- joinResult = context.sendGenericGroupJoin(request.setGroupInstanceId("group-instance-id"), true, true);
+ joinResult = context.sendClassicGroupJoin(request.setGroupInstanceId("group-instance-id"), true, true);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), joinResult.joinFuture.get().errorCode());
@@ -5340,7 +5340,7 @@
public void testJoinGroupWithEmptyGroupProtocol() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5349,7 +5349,7 @@
.withProtocols(new JoinGroupRequestProtocolCollection(0))
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), joinResult.joinFuture.get().errorCode());
@@ -5365,26 +5365,26 @@
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
.withDefaultProtocolTypeAndProtocols()
- .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
- .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+ .withSessionTimeoutMs(5000 + context.classicGroupNewMemberJoinTimeoutMs)
+ .withRebalanceTimeoutMs(2 * context.classicGroupNewMemberJoinTimeoutMs)
.build();
- JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData firstResponse = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
String firstMemberId = firstResponse.memberId();
assertEquals(firstResponse.leader(), firstMemberId);
assertEquals(Errors.NONE.code(), firstResponse.errorCode());
assertNotNull(group);
- assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+ assertEquals(0, group.allMembers().stream().filter(ClassicGroupMember::isNew).count());
// Send second join group request for a new dynamic member.
- JoinResult secondJoinResult = context.sendGenericGroupJoin(request
+ JoinResult secondJoinResult = context.sendClassicGroupJoin(request
.setSessionTimeoutMs(5000)
.setRebalanceTimeoutMs(5000)
);
@@ -5392,20 +5392,20 @@
assertFalse(secondJoinResult.joinFuture.isDone());
assertEquals(2, group.allMembers().size());
- assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+ assertEquals(1, group.allMembers().stream().filter(ClassicGroupMember::isNew).count());
- GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+ ClassicGroupMember newMember = group.allMembers().stream().filter(ClassicGroupMember::isNew).findFirst().get();
assertNotEquals(firstMemberId, newMember.memberId());
// Advance clock by new member join timeout to expire the second member.
- assertNoOrEmptyResult(context.sleep(context.genericGroupNewMemberJoinTimeoutMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupNewMemberJoinTimeoutMs));
assertTrue(secondJoinResult.joinFuture.isDone());
JoinGroupResponseData secondResponse = secondJoinResult.joinFuture.get();
assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
assertEquals(1, group.allMembers().size());
- assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+ assertEquals(0, group.allMembers().stream().filter(ClassicGroupMember::isNew).count());
assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
}
@@ -5413,7 +5413,7 @@
public void testJoinGroupInconsistentGroupProtocol() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5422,17 +5422,17 @@
.withProtocols(toProtocols("range"))
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
- JoinResult otherJoinResult = context.sendGenericGroupJoin(request.setProtocols(toProtocols("roundrobin")));
+ JoinResult otherJoinResult = context.sendClassicGroupJoin(request.setProtocols(toProtocols("roundrobin")));
assertTrue(joinResult.records.isEmpty());
assertTrue(otherJoinResult.joinFuture.isDone());
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherJoinResult.joinFuture.get().errorCode());
@@ -5442,7 +5442,7 @@
public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5450,7 +5450,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(request, true);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5462,19 +5462,19 @@
request = request.setMemberId(memberId)
.setProtocols(emptyProtocols);
- joinResult = context.sendGenericGroupJoin(request, true);
+ joinResult = context.sendClassicGroupJoin(request, true);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), joinResult.joinFuture.get().errorCode());
// Sending consistent protocol should be accepted
- joinResult = context.sendGenericGroupJoin(request.setProtocols(toProtocols("range")), true);
+ joinResult = context.sendClassicGroupJoin(request.setProtocols(toProtocols("range")), true);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
@@ -5484,7 +5484,7 @@
public void testStaticMemberJoinAsFirstMember() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5493,14 +5493,14 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- context.joinGenericGroupAndCompleteJoin(request, false, true);
+ context.joinClassicGroupAndCompleteJoin(request, false, true);
}
@Test
public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5511,10 +5511,10 @@
.withRebalanceTimeoutMs(5000)
.build();
- JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, false, true);
+ JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, false, true);
assertEquals(Errors.NONE.code(), response.errorCode());
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request.setMemberId("unknown-member-id")
);
@@ -5527,7 +5527,7 @@
public void testJoinGroupUnknownConsumerExistingGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5537,10 +5537,10 @@
.withRebalanceTimeoutMs(5000)
.build();
- JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData response = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), response.errorCode());
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request.setMemberId("other-member-id")
);
@@ -5553,7 +5553,7 @@
public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
group.transitionTo(DEAD);
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -5562,7 +5562,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5580,7 +5580,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5592,7 +5592,7 @@
public void testJoinGroupReturnsTheProtocolType() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
// Leader joins
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -5601,19 +5601,19 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(request);
assertTrue(leaderJoinResult.records.isEmpty());
assertFalse(leaderJoinResult.joinFuture.isDone());
// Member joins
- JoinResult memberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult memberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(memberJoinResult.records.isEmpty());
assertFalse(memberJoinResult.joinFuture.isDone());
// Complete join group phase
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
assertTrue(leaderJoinResult.joinFuture.isDone());
assertTrue(memberJoinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), leaderJoinResult.joinFuture.get().errorCode());
@@ -5626,7 +5626,7 @@
public void testDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5634,15 +5634,15 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs / 2));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs / 2));
assertFalse(joinResult.joinFuture.isDone());
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs / 2 + 1));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs / 2 + 1));
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
}
@@ -5651,34 +5651,34 @@
public void testResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
.withDefaultProtocolTypeAndProtocols()
- .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+ .withRebalanceTimeoutMs(context.classicGroupInitialRebalanceDelayMs * 3)
.build();
- JoinResult firstMemberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult firstMemberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(firstMemberJoinResult.records.isEmpty());
assertFalse(firstMemberJoinResult.joinFuture.isDone());
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs - 1));
- JoinResult secondMemberJoinResult = context.sendGenericGroupJoin(request);
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs - 1));
+ JoinResult secondMemberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(secondMemberJoinResult.records.isEmpty());
assertFalse(secondMemberJoinResult.joinFuture.isDone());
assertNoOrEmptyResult(context.sleep(2));
// Advance clock past initial rebalance delay and verify futures are not completed.
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs / 2 + 1));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs / 2 + 1));
assertFalse(firstMemberJoinResult.joinFuture.isDone());
assertFalse(secondMemberJoinResult.joinFuture.isDone());
// Advance clock beyond recomputed delay and make sure the futures have completed.
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs / 2));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs / 2));
assertTrue(firstMemberJoinResult.joinFuture.isDone());
assertTrue(secondMemberJoinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), firstMemberJoinResult.joinFuture.get().errorCode());
@@ -5689,34 +5689,34 @@
public void testDelayRebalanceUptoRebalanceTimeout() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
.withDefaultProtocolTypeAndProtocols()
- .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+ .withRebalanceTimeoutMs(context.classicGroupInitialRebalanceDelayMs * 2)
.build();
- JoinResult firstMemberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult firstMemberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(firstMemberJoinResult.records.isEmpty());
assertFalse(firstMemberJoinResult.joinFuture.isDone());
- JoinResult secondMemberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult secondMemberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(secondMemberJoinResult.records.isEmpty());
assertFalse(secondMemberJoinResult.joinFuture.isDone());
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs + 1));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs + 1));
- JoinResult thirdMemberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult thirdMemberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(thirdMemberJoinResult.records.isEmpty());
assertFalse(thirdMemberJoinResult.joinFuture.isDone());
// Advance clock right before rebalance timeout.
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs - 1));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs - 1));
assertFalse(firstMemberJoinResult.joinFuture.isDone());
assertFalse(secondMemberJoinResult.joinFuture.isDone());
assertFalse(thirdMemberJoinResult.joinFuture.isDone());
@@ -5736,7 +5736,7 @@
public void testJoinGroupReplaceStaticMember() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5747,7 +5747,7 @@
.build();
// Send join group as static member.
- JoinResult oldMemberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult oldMemberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(oldMemberJoinResult.records.isEmpty());
assertFalse(oldMemberJoinResult.joinFuture.isDone());
@@ -5755,7 +5755,7 @@
assertEquals(1, group.size());
// Replace static member with new member id. Old member id should be fenced.
- JoinResult newMemberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult newMemberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(newMemberJoinResult.records.isEmpty());
assertFalse(newMemberJoinResult.joinFuture.isDone());
@@ -5765,7 +5765,7 @@
assertEquals(1, group.size());
// Complete join for new member.
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
assertTrue(newMemberJoinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), newMemberJoinResult.joinFuture.get().errorCode());
assertEquals(0, group.numAwaitingJoinResponse());
@@ -5776,7 +5776,7 @@
public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5785,7 +5785,7 @@
.withSessionTimeoutMs(1000)
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(request, true);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5802,9 +5802,9 @@
public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
// Set initial rebalance delay to simulate a long running rebalance.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+ .withClassicGroupInitialRebalanceDelayMs(10 * 60 * 1000)
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5812,7 +5812,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
@@ -5821,11 +5821,11 @@
String memberId = group.leaderOrNull();
// Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
// A group that transitions to Empty after completing join phase will generate records.
- List<ExpiredTimeout<Void, Record>> timeouts = context.sleep(context.genericGroupNewMemberJoinTimeoutMs);
+ List<ExpiredTimeout<Void, Record>> timeouts = context.sleep(context.classicGroupNewMemberJoinTimeoutMs);
assertEquals(1, timeouts.size());
timeouts.forEach(timeout -> {
- assertEquals(genericGroupHeartbeatKey("group-id", memberId), timeout.key);
+ assertEquals(classicGroupHeartbeatKey("group-id", memberId), timeout.key);
assertEquals(Collections.singletonList(
newGroupMetadataRecordWithCurrentState(group, MetadataVersion.latest())),
timeout.result.records());
@@ -5840,7 +5840,7 @@
public void testExistingMemberJoinDeadGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5848,7 +5848,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData response = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), response.errorCode());
String memberId = response.memberId();
@@ -5856,7 +5856,7 @@
group.transitionTo(DEAD);
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -5867,7 +5867,7 @@
public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5875,7 +5875,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(request, true);
assertTrue(joinResult.records.isEmpty());
assertEquals(Errors.MEMBER_ID_REQUIRED.code(), joinResult.joinFuture.get().errorCode());
@@ -5883,7 +5883,7 @@
String memberId = joinResult.joinFuture.get().memberId();
assertThrows(IllegalStateException.class,
- () -> context.sendGenericGroupJoin(
+ () -> context.sendClassicGroupJoin(
request
.setMemberId(memberId)
.setGroupInstanceId("group-instance-id")
@@ -5895,7 +5895,7 @@
public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestProtocolCollection protocols = toProtocols("range");
@@ -5906,11 +5906,11 @@
.withProtocols(protocols)
.build();
- JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData response = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), response.errorCode());
String memberId = response.memberId();
- GenericGroupMember member = group.member(memberId);
+ ClassicGroupMember member = group.member(memberId);
assertEquals(protocols, member.supportedProtocols());
assertTrue(group.isInState(COMPLETING_REBALANCE));
@@ -5919,7 +5919,7 @@
protocols = toProtocols("range", "roundrobin");
// Send updated member metadata. This should trigger a rebalance and complete the join phase.
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request
.setMemberId(memberId)
.setProtocols(protocols)
@@ -5939,7 +5939,7 @@
public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -5947,7 +5947,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData response = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), response.errorCode());
String memberId = response.memberId();
@@ -5958,7 +5958,7 @@
group.transitionTo(STABLE);
// Sending join group as leader should trigger a rebalance.
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request.setMemberId(memberId)
);
@@ -5973,7 +5973,7 @@
public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -5983,19 +5983,19 @@
.withProtocols(toProtocols("range"))
.build();
- JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData leaderResponse = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
String leaderId = leaderResponse.leader();
assertEquals(1, group.generationId());
// Member joins.
- JoinResult memberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult memberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(memberJoinResult.records.isEmpty());
assertFalse(memberJoinResult.joinFuture.isDone());
// Leader also rejoins. Completes join group phase.
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(request.setMemberId(leaderId));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(leaderJoinResult.joinFuture.isDone());
@@ -6013,7 +6013,7 @@
JoinGroupRequestProtocolCollection protocols = toProtocols("range", "roundrobin");
- memberJoinResult = context.sendGenericGroupJoin(
+ memberJoinResult = context.sendClassicGroupJoin(
request
.setMemberId(memberId)
.setProtocols(protocols)
@@ -6023,7 +6023,7 @@
assertFalse(memberJoinResult.joinFuture.isDone());
// Leader rejoins. This completes the join group phase.
- leaderJoinResult = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+ leaderJoinResult = context.sendClassicGroupJoin(request.setMemberId(leaderId));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(leaderJoinResult.joinFuture.isDone());
@@ -6038,7 +6038,7 @@
public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6046,19 +6046,19 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData leaderResponse = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
String leaderId = leaderResponse.leader();
assertEquals(1, group.generationId());
// Member joins.
- JoinResult memberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult memberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(memberJoinResult.records.isEmpty());
assertFalse(memberJoinResult.joinFuture.isDone());
// Leader also rejoins. Completes join group phase.
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(request.setMemberId(leaderId));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(leaderJoinResult.joinFuture.isDone());
@@ -6072,7 +6072,7 @@
group.transitionTo(STABLE);
// Member rejoins with no metadata changes. This does not trigger a rebalance.
- memberJoinResult = context.sendGenericGroupJoin(request.setMemberId(memberId));
+ memberJoinResult = context.sendClassicGroupJoin(request.setMemberId(memberId));
assertTrue(memberJoinResult.records.isEmpty());
assertTrue(memberJoinResult.joinFuture.isDone());
@@ -6086,7 +6086,7 @@
// Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6094,7 +6094,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData response = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), response.errorCode());
String memberId = response.memberId();
@@ -6104,7 +6104,7 @@
group.transitionTo(PREPARING_REBALANCE);
group.transitionTo(EMPTY);
- JoinResult joinResult = context.sendGenericGroupJoin(request.setMemberId(memberId));
+ JoinResult joinResult = context.sendClassicGroupJoin(request.setMemberId(memberId));
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -6116,7 +6116,7 @@
public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6126,12 +6126,12 @@
.withRebalanceTimeoutMs(1000)
.build();
- JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+ JoinGroupResponseData leaderResponse = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(request);
assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
assertEquals(1, group.generationId());
// Add new member. This triggers a rebalance.
- JoinResult memberJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult memberJoinResult = context.sendClassicGroupJoin(request);
assertTrue(memberJoinResult.records.isEmpty());
assertFalse(memberJoinResult.joinFuture.isDone());
@@ -6152,7 +6152,7 @@
public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6162,7 +6162,7 @@
.withRebalanceTimeoutMs(1000)
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
@@ -6172,7 +6172,7 @@
group.transitionTo(DEAD);
// Advance clock by initial rebalance delay to complete join phase.
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
assertEquals(0, group.generationId());
}
@@ -6180,7 +6180,7 @@
public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6192,12 +6192,12 @@
.build();
// First member joins group and completes join phase.
- JoinGroupResponseData firstMemberResponse = context.joinGenericGroupAndCompleteJoin(request, true, true);
+ JoinGroupResponseData firstMemberResponse = context.joinClassicGroupAndCompleteJoin(request, true, true);
assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
String firstMemberId = firstMemberResponse.memberId();
// Second member joins and group goes into rebalancing state.
- JoinResult secondMemberJoinResult = context.sendGenericGroupJoin(
+ JoinResult secondMemberJoinResult = context.sendClassicGroupJoin(
request.setGroupInstanceId("second-instance-id")
);
@@ -6205,7 +6205,7 @@
assertFalse(secondMemberJoinResult.joinFuture.isDone());
// First static member rejoins and completes join phase.
- JoinResult firstMemberJoinResult = context.sendGenericGroupJoin(
+ JoinResult firstMemberJoinResult = context.sendClassicGroupJoin(
request.setMemberId(firstMemberId).setGroupInstanceId("first-instance-id"));
assertTrue(firstMemberJoinResult.records.isEmpty());
@@ -6237,7 +6237,7 @@
assertEquals(2, group.generationId());
// Let first and second member rejoin. This should complete the join phase.
- firstMemberJoinResult = context.sendGenericGroupJoin(
+ firstMemberJoinResult = context.sendClassicGroupJoin(
request
.setMemberId(firstMemberId)
.setGroupInstanceId("first-instance-id")
@@ -6249,7 +6249,7 @@
assertEquals(2, group.size());
assertEquals(2, group.generationId());
- secondMemberJoinResult = context.sendGenericGroupJoin(
+ secondMemberJoinResult = context.sendClassicGroupJoin(
request
.setMemberId(secondMemberId)
.setGroupInstanceId("second-instance-id")
@@ -6272,7 +6272,7 @@
) throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6282,7 +6282,7 @@
.withProtocols(toProtocols("range"))
.build();
- JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, true, supportSkippingAssignment);
+ JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, true, supportSkippingAssignment);
assertEquals(Errors.NONE.code(), response.errorCode());
String oldMemberId = response.memberId();
@@ -6296,7 +6296,7 @@
// Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
JoinGroupRequestProtocolCollection protocols = toProtocols("range", "roundrobin");
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request
.setProtocols(protocols)
.setRebalanceTimeoutMs(7000)
@@ -6338,7 +6338,7 @@
.setLeader(newMemberId);
}
- GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+ ClassicGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
assertEquals(expectedResponse, joinResult.joinFuture.get());
assertEquals(newMemberId, updatedMember.memberId());
@@ -6356,7 +6356,7 @@
public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6366,7 +6366,7 @@
.withProtocols(toProtocols("range", "roundrobin"))
.build();
- JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, true, true);
+ JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, true, true);
assertEquals(Errors.NONE.code(), response.errorCode());
assertEquals(1, group.size());
assertEquals(1, group.generationId());
@@ -6376,7 +6376,7 @@
group.transitionTo(STABLE);
// Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance.
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request.setProtocols(toProtocols("roundrobin"))
);
@@ -6393,7 +6393,7 @@
// If the append future fails, confirm that the member is not updated.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestProtocolCollection protocols = toProtocols("range");
@@ -6407,7 +6407,7 @@
.withSessionTimeoutMs(3000)
.build();
- JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, false, false);
+ JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, false, false);
assertEquals(Errors.NONE.code(), response.errorCode());
assertEquals(1, group.size());
assertEquals(1, group.generationId());
@@ -6423,7 +6423,7 @@
.setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
Collections.singletonList("bar"))).array()));
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request
.setProtocols(protocols)
.setRebalanceTimeoutMs(7000)
@@ -6454,7 +6454,7 @@
assertEquals(expectedResponse, joinResult.joinFuture.get());
- GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
+ ClassicGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
assertEquals(oldMemberId, revertedMember.memberId());
assertEquals(Optional.of("group-instance-id"), revertedMember.groupInstanceId());
@@ -6474,7 +6474,7 @@
// If the append future succeeds, the soft state is updated with the new member.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6484,7 +6484,7 @@
.withProtocols(toProtocols("range"))
.build();
- JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(
+ JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(
request,
true,
supportSkippingAssignment
@@ -6500,7 +6500,7 @@
group.transitionTo(STABLE);
// Static member rejoins with UNKNOWN_MEMBER_ID and the append succeeds.
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request
.setProtocols(toProtocols("range", "roundrobin"))
.setRebalanceTimeoutMs(7000)
@@ -6530,7 +6530,7 @@
assertEquals(expectedResponse, joinResult.joinFuture.get());
- GenericGroupMember newMember = group.member(group.staticMemberId("group-instance-id"));
+ ClassicGroupMember newMember = group.member(group.staticMemberId("group-instance-id"));
assertNotEquals(oldMemberId, newMember.memberId());
assertEquals(Optional.of("group-instance-id"), newMember.groupInstanceId());
@@ -6546,7 +6546,7 @@
public void testReplaceStaticMemberInCompletingRebalanceStateTriggersRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6555,7 +6555,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, true, true);
+ JoinGroupResponseData response = context.joinClassicGroupAndCompleteJoin(request, true, true);
assertEquals(Errors.NONE.code(), response.errorCode());
assertEquals(1, group.size());
@@ -6563,7 +6563,7 @@
assertTrue(group.isInState(COMPLETING_REBALANCE));
// Static member rejoins with UNKNOWN_MEMBER_ID and triggers a rebalance.
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -6593,14 +6593,14 @@
public void testNewMemberTimeoutCompletion() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
.withDefaultProtocolTypeAndProtocols()
- .withSessionTimeoutMs(context.genericGroupNewMemberJoinTimeoutMs + 5000)
+ .withSessionTimeoutMs(context.classicGroupNewMemberJoinTimeoutMs + 5000)
.build()
);
@@ -6608,16 +6608,16 @@
assertFalse(joinResult.joinFuture.isDone());
// Advance clock by initial rebalance delay to complete join phase.
- assertNoOrEmptyResult(context.sleep(context.genericGroupInitialRebalanceDelayMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupInitialRebalanceDelayMs));
assertTrue(joinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), joinResult.joinFuture.get().errorCode());
assertEquals(1, group.generationId());
assertTrue(group.isInState(COMPLETING_REBALANCE));
- assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+ assertEquals(0, group.allMembers().stream().filter(ClassicGroupMember::isNew).count());
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(joinResult.joinFuture.get().memberId())
@@ -6633,7 +6633,7 @@
assertEquals(1, group.size());
// Make sure the NewMemberTimeout is not still in effect, and the member is not kicked
- assertNoOrEmptyResult(context.sleep(context.genericGroupNewMemberJoinTimeoutMs));
+ assertNoOrEmptyResult(context.sleep(context.classicGroupNewMemberJoinTimeoutMs));
assertEquals(1, group.size());
// Member should be removed as heartbeat expires. The group is now empty.
@@ -6653,7 +6653,7 @@
assertEquals(1, timeouts.size());
String memberId = joinResult.joinFuture.get().memberId();
timeouts.forEach(timeout -> {
- assertEquals(genericGroupHeartbeatKey("group-id", memberId), timeout.key);
+ assertEquals(classicGroupHeartbeatKey("group-id", memberId), timeout.key);
assertEquals(expectedRecords, timeout.result.records());
});
@@ -6669,7 +6669,7 @@
// normal heartbeat expiration logic.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -6679,14 +6679,14 @@
.withRebalanceTimeoutMs(10000)
.build();
- JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, false, false);
+ JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(joinRequest, false, false);
assertEquals(Errors.NONE.code(), joinResponse.errorCode());
String memberId = joinResponse.memberId();
assertEquals(memberId, joinResponse.leader());
assertEquals(1, joinResponse.generationId());
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(memberId)
@@ -6703,8 +6703,8 @@
assertTrue(group.isInState(STABLE));
assertEquals(1, group.generationId());
- JoinResult otherJoinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(UNKNOWN_MEMBER_ID));
- JoinResult joinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(memberId));
+ JoinResult otherJoinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(UNKNOWN_MEMBER_ID));
+ JoinResult joinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(memberId));
assertTrue(otherJoinResult.records.isEmpty());
assertTrue(joinResult.records.isEmpty());
@@ -6726,7 +6726,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// A third member joins. Trigger a rebalance.
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -6735,12 +6735,12 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- context.sendGenericGroupJoin(request);
+ context.sendClassicGroupJoin(request);
assertTrue(group.isInState(PREPARING_REBALANCE));
// Old follower rejoins group will be matching current member.id.
- JoinResult oldFollowerJoinResult = context.sendGenericGroupJoin(
+ JoinResult oldFollowerJoinResult = context.sendClassicGroupJoin(
request
.setMemberId(rebalanceResult.followerId)
.setGroupInstanceId("follower-instance-id")
@@ -6750,7 +6750,7 @@
assertFalse(oldFollowerJoinResult.joinFuture.isDone());
// Duplicate follower joins group with unknown member id will trigger member id replacement.
- context.sendGenericGroupJoin(
+ context.sendClassicGroupJoin(
request
.setMemberId(UNKNOWN_MEMBER_ID)
.setGroupInstanceId("follower-instance-id")
@@ -6787,7 +6787,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Known leader rejoins will trigger rebalance.
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -6798,14 +6798,14 @@
.withRebalanceTimeoutMs(10000)
.build();
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(request);
assertTrue(leaderJoinResult.records.isEmpty());
assertFalse(leaderJoinResult.joinFuture.isDone());
assertTrue(group.isInState(PREPARING_REBALANCE));
// Old follower rejoins group will match current member.id.
- JoinResult oldFollowerJoinResult = context.sendGenericGroupJoin(
+ JoinResult oldFollowerJoinResult = context.sendClassicGroupJoin(
request
.setMemberId(rebalanceResult.followerId)
.setGroupInstanceId("follower-instance-id")
@@ -6861,7 +6861,7 @@
// Duplicate follower joins group with unknown member id will trigger member.id replacement,
// and will also trigger a rebalance under CompletingRebalance state; the old follower sync callback
// will return fenced exception while broker replaces the member identity with the duplicate follower joins.
- SyncResult oldFollowerSyncResult = context.sendGenericGroupSync(
+ SyncResult oldFollowerSyncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withGroupInstanceId("follower-instance-id")
@@ -6873,7 +6873,7 @@
assertTrue(oldFollowerSyncResult.records.isEmpty());
assertFalse(oldFollowerSyncResult.syncFuture.isDone());
- JoinResult duplicateFollowerJoinResult = context.sendGenericGroupJoin(
+ JoinResult duplicateFollowerJoinResult = context.sendClassicGroupJoin(
request
.setMemberId(UNKNOWN_MEMBER_ID)
.setGroupInstanceId("follower-instance-id")
@@ -6909,7 +6909,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Known leader rejoins will trigger rebalance.
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -6920,14 +6920,14 @@
.withRebalanceTimeoutMs(10000)
.build();
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(request);
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(request);
assertTrue(leaderJoinResult.records.isEmpty());
assertFalse(leaderJoinResult.joinFuture.isDone());
assertTrue(group.isInState(PREPARING_REBALANCE));
// Duplicate follower joins group will trigger member id replacement.
- JoinResult duplicateFollowerJoinResult = context.sendGenericGroupJoin(
+ JoinResult duplicateFollowerJoinResult = context.sendClassicGroupJoin(
request
.setMemberId(UNKNOWN_MEMBER_ID)
.setGroupInstanceId("follower-instance-id")
@@ -6937,7 +6937,7 @@
assertTrue(duplicateFollowerJoinResult.joinFuture.isDone());
// Old follower rejoins group will fail because member id is already updated.
- JoinResult oldFollowerJoinResult = context.sendGenericGroupJoin(
+ JoinResult oldFollowerJoinResult = context.sendClassicGroupJoin(
request.setMemberId(rebalanceResult.followerId)
);
@@ -7005,7 +7005,7 @@
public void testStaticMemberRejoinWithKnownMemberId() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData request = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -7014,12 +7014,12 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(request, false, false);
+ JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(request, false, false);
assertEquals(Errors.NONE.code(), joinResponse.errorCode());
String memberId = joinResponse.memberId();
- JoinResult rejoinResult = context.sendGenericGroupJoin(
+ JoinResult rejoinResult = context.sendClassicGroupJoin(
request.setMemberId(memberId)
);
@@ -7029,7 +7029,7 @@
assertTrue(rejoinResult.joinFuture.isDone());
assertEquals(Errors.NONE.code(), rejoinResult.joinFuture.get().errorCode());
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(memberId)
@@ -7059,7 +7059,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// A static leader rejoin with unknown id will not trigger rebalance, and no assignment will be returned.
// As the group was in Stable state and the member id was updated, this will generate records.
@@ -7070,7 +7070,7 @@
.withProtocolSuperset()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
joinRequest,
true,
supportSkippingAssignment
@@ -7108,7 +7108,7 @@
supportSkippingAssignment ? mkSet("leader-instance-id", "follower-instance-id") : Collections.emptySet()
);
- JoinResult oldLeaderJoinResult = context.sendGenericGroupJoin(
+ JoinResult oldLeaderJoinResult = context.sendClassicGroupJoin(
joinRequest.setMemberId(rebalanceResult.leaderId),
true,
supportSkippingAssignment
@@ -7126,7 +7126,7 @@
.withMemberId(rebalanceResult.leaderId)
.build();
- SyncResult oldLeaderSyncResult = context.sendGenericGroupSync(oldLeaderSyncRequest);
+ SyncResult oldLeaderSyncResult = context.sendClassicGroupSync(oldLeaderSyncRequest);
assertTrue(oldLeaderSyncResult.records.isEmpty());
assertTrue(oldLeaderSyncResult.syncFuture.isDone());
@@ -7134,7 +7134,7 @@
// Calling sync on old leader id will fail because that leader id is no longer valid and replaced.
SyncGroupRequestData newLeaderSyncRequest = oldLeaderSyncRequest.setGroupInstanceId(null);
- SyncResult newLeaderSyncResult = context.sendGenericGroupSync(newLeaderSyncRequest);
+ SyncResult newLeaderSyncResult = context.sendClassicGroupSync(newLeaderSyncRequest);
assertTrue(newLeaderSyncResult.records.isEmpty());
assertTrue(newLeaderSyncResult.syncFuture.isDone());
@@ -7151,7 +7151,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Known static leader rejoin will trigger rebalance.
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -7162,7 +7162,7 @@
.withRebalanceTimeoutMs(10000)
.build();
- JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(request, true, true, 10000);
+ JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(request, true, true, 10000);
// Follower's heartbeat expires as the leader rejoins.
assertFalse(group.hasMemberId(rebalanceResult.followerId));
@@ -7195,7 +7195,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
group.transitionTo(DEAD);
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -7205,7 +7205,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request, true, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(request, true, true);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -7222,7 +7222,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
group.transitionTo(PREPARING_REBALANCE);
group.transitionTo(EMPTY);
@@ -7234,7 +7234,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request, true, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(request, true, true);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -7255,7 +7255,7 @@
rebalanceTimeoutMs,
sessionTimeoutMs
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// A static follower rejoin with changed protocol will trigger rebalance.
JoinGroupRequestProtocolCollection protocols = toProtocols("roundrobin");
@@ -7268,7 +7268,7 @@
.withSessionTimeoutMs(sessionTimeoutMs)
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
@@ -7314,7 +7314,7 @@
rebalanceTimeoutMs,
sessionTimeoutMs
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
assertNotEquals("roundrobin", group.selectProtocol());
@@ -7330,7 +7330,7 @@
.withSessionTimeoutMs(sessionTimeoutMs)
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertFalse(joinResult.joinFuture.isDone());
assertEquals("roundrobin", group.selectProtocol());
@@ -7372,7 +7372,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
JoinGroupRequestProtocolCollection protocols = toProtocols(group.selectProtocol());
@@ -7383,7 +7383,7 @@
.withProtocols(protocols)
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7417,12 +7417,12 @@
// Join with old member id will not fail because the member id is not updated because of persistence failure
assertNotEquals(rebalanceResult.followerId, followerJoinResult.joinFuture.get().memberId());
- followerJoinResult = context.sendGenericGroupJoin(request.setMemberId(rebalanceResult.followerId));
+ followerJoinResult = context.sendClassicGroupJoin(request.setMemberId(rebalanceResult.followerId));
assertTrue(followerJoinResult.records.isEmpty());
// Join with leader and complete join phase.
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(
request.setGroupInstanceId("leader-instance-id")
.setMemberId(rebalanceResult.leaderId)
);
@@ -7441,7 +7441,7 @@
.withGenerationId(rebalanceResult.generationId + 1)
.build();
- SyncResult leaderSyncResult = context.sendGenericGroupSync(syncRequest);
+ SyncResult leaderSyncResult = context.sendClassicGroupSync(syncRequest);
// Simulate a successful write to the log. This will update the group with the new (empty) assignment.
leaderSyncResult.appendFuture.complete(null);
@@ -7456,7 +7456,7 @@
assertEquals(Errors.NONE.code(), leaderSyncResult.syncFuture.get().errorCode());
// Sync with old member id will also not fail as the member id is not updated due to persistence failure
- SyncResult oldMemberSyncResult = context.sendGenericGroupSync(
+ SyncResult oldMemberSyncResult = context.sendClassicGroupSync(
syncRequest
.setGroupInstanceId("follower-instance-id")
.setMemberId(rebalanceResult.followerId)
@@ -7477,7 +7477,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// A static follower rejoin with protocol changing to leader protocol subset won't trigger rebalance if updated
// group's selectProtocol remain unchanged.
@@ -7490,7 +7490,7 @@
.withProtocols(protocols)
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7526,7 +7526,7 @@
// Join with old member id will fail because the member id is updated
String newFollowerId = followerJoinResult.joinFuture.get().memberId();
assertNotEquals(rebalanceResult.followerId, newFollowerId);
- followerJoinResult = context.sendGenericGroupJoin(request.setMemberId(rebalanceResult.followerId));
+ followerJoinResult = context.sendClassicGroupJoin(request.setMemberId(rebalanceResult.followerId));
assertTrue(followerJoinResult.records.isEmpty());
assertEquals(Errors.FENCED_INSTANCE_ID.code(), followerJoinResult.joinFuture.get().errorCode());
@@ -7540,14 +7540,14 @@
.withAssignment(Collections.emptyList())
.build();
- SyncResult syncResult = context.sendGenericGroupSync(syncRequest);
+ SyncResult syncResult = context.sendClassicGroupSync(syncRequest);
assertTrue(syncResult.records.isEmpty());
assertTrue(syncResult.syncFuture.isDone());
assertEquals(Errors.FENCED_INSTANCE_ID.code(), syncResult.syncFuture.get().errorCode());
// Sync with new member id succeeds
- syncResult = context.sendGenericGroupSync(syncRequest.setMemberId(newFollowerId));
+ syncResult = context.sendClassicGroupSync(syncRequest.setMemberId(newFollowerId));
assertTrue(syncResult.records.isEmpty());
assertTrue(syncResult.syncFuture.isDone());
@@ -7567,7 +7567,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// A static leader rejoin with known member id will trigger rebalance.
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -7579,7 +7579,7 @@
.withSessionTimeoutMs(5000)
.build();
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7589,7 +7589,7 @@
assertFalse(leaderJoinResult.joinFuture.isDone());
// Rebalance completes immediately after follower rejoins.
- JoinResult followerJoinResult = context.sendGenericGroupJoin(
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(
request.setGroupInstanceId("follower-instance-id")
.setMemberId(rebalanceResult.followerId),
true,
@@ -7642,7 +7642,7 @@
// The follower protocol changed from protocolSuperset to general protocols.
JoinGroupRequestProtocolCollection protocols = toProtocols("range");
- followerJoinResult = context.sendGenericGroupJoin(
+ followerJoinResult = context.sendClassicGroupJoin(
request.setGroupInstanceId("follower-instance-id")
.setMemberId(rebalanceResult.followerId)
.setProtocols(protocols),
@@ -7692,7 +7692,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// A static follower rejoin with no protocol change will not trigger rebalance.
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -7702,7 +7702,7 @@
.withProtocolSuperset()
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7739,7 +7739,7 @@
);
assertNotEquals(rebalanceResult.followerId, followerJoinResult.joinFuture.get().memberId());
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withGroupInstanceId("follower-instance-id")
@@ -7764,7 +7764,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// A static follower rejoin with no protocol change will not trigger rebalance.
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -7774,7 +7774,7 @@
.withProtocolSuperset()
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7824,7 +7824,7 @@
.withProtocolSuperset()
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7853,7 +7853,7 @@
.withProtocolSuperset()
.build();
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7881,7 +7881,7 @@
.withMemberId("invalid-member-id")
.build();
- SyncResult leaderSyncResult = context.sendGenericGroupSync(request);
+ SyncResult leaderSyncResult = context.sendClassicGroupSync(request);
assertTrue(leaderSyncResult.records.isEmpty());
assertTrue(leaderSyncResult.syncFuture.isDone());
@@ -7898,7 +7898,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
String lastMemberId = rebalanceResult.leaderId;
for (int i = 0; i < 5; i++) {
@@ -7909,7 +7909,7 @@
.withProtocolSuperset()
.build();
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7948,7 +7948,7 @@
.withProtocolSuperset()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -7969,7 +7969,7 @@
"leader-instance-id",
"follower-instance-id"
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
group.transitionTo(PREPARING_REBALANCE);
group.transitionTo(EMPTY);
@@ -7982,7 +7982,7 @@
.build();
// Illegal state exception shall trigger since follower id resides in pending member bucket.
- IllegalStateException exception = assertThrows(IllegalStateException.class, () -> context.sendGenericGroupJoin(
+ IllegalStateException exception = assertThrows(IllegalStateException.class, () -> context.sendClassicGroupJoin(
request,
true,
true
@@ -8006,7 +8006,7 @@
10000,
15000
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
String newMemberInstanceId = "new-member-instance-id";
String leaderId = rebalanceResult.leaderId;
@@ -8018,7 +8018,7 @@
.withProtocolSuperset()
.build();
- JoinResult newMemberJoinResult = context.sendGenericGroupJoin(
+ JoinResult newMemberJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -8028,7 +8028,7 @@
assertFalse(newMemberJoinResult.joinFuture.isDone());
assertTrue(group.isInState(PREPARING_REBALANCE));
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(
request
.setGroupInstanceId("leader-instance-id")
.setMemberId(leaderId),
@@ -8095,7 +8095,7 @@
10000,
15000
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
String newMemberInstanceId = "new-member-instance-id";
JoinGroupRequestData request = new JoinGroupRequestBuilder()
@@ -8105,7 +8105,7 @@
.withProtocolSuperset()
.build();
- JoinResult newMemberJoinResult = context.sendGenericGroupJoin(
+ JoinResult newMemberJoinResult = context.sendClassicGroupJoin(
request,
true,
true
@@ -8115,7 +8115,7 @@
assertFalse(newMemberJoinResult.joinFuture.isDone());
assertTrue(group.isInState(PREPARING_REBALANCE));
- JoinResult oldFollowerJoinResult = context.sendGenericGroupJoin(
+ JoinResult oldFollowerJoinResult = context.sendClassicGroupJoin(
request
.setGroupInstanceId("follower-instance-id")
.setMemberId(rebalanceResult.followerId),
@@ -8231,7 +8231,7 @@
) throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
// JoinGroup(leader) with the Protocol Type of the group
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
@@ -8241,13 +8241,13 @@
.withProtocolSuperset()
.build();
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(joinRequest);
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(joinRequest);
assertTrue(leaderJoinResult.records.isEmpty());
assertFalse(leaderJoinResult.joinFuture.isDone());
// JoinGroup(follower) with the Protocol Type of the group
- JoinResult followerJoinResult = context.sendGenericGroupJoin(joinRequest.setGroupInstanceId("follower-instance-id"));
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(joinRequest.setGroupInstanceId("follower-instance-id"));
assertTrue(followerJoinResult.records.isEmpty());
assertFalse(followerJoinResult.joinFuture.isDone());
@@ -8275,7 +8275,7 @@
.withAssignment(assignment)
.build();
- SyncResult leaderSyncResult = context.sendGenericGroupSync(syncRequest);
+ SyncResult leaderSyncResult = context.sendClassicGroupSync(syncRequest);
// Simulate a successful write to the log.
leaderSyncResult.appendFuture.complete(null);
@@ -8284,7 +8284,7 @@
assertEquals(expectedProtocolType.orElse(null), leaderSyncResult.syncFuture.get().protocolType());
assertEquals(expectedProtocolName.orElse(null), leaderSyncResult.syncFuture.get().protocolName());
- SyncResult followerSyncResult = context.sendGenericGroupSync(syncRequest.setMemberId(followerId));
+ SyncResult followerSyncResult = context.sendClassicGroupSync(syncRequest.setMemberId(followerId));
assertTrue(followerSyncResult.records.isEmpty());
assertTrue(followerSyncResult.syncFuture.isDone());
@@ -8299,7 +8299,7 @@
.build();
// SyncGroup with the provided Protocol Type and Name
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId("member-id")
@@ -8316,9 +8316,9 @@
public void testSyncGroupFromUnknownMember() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
- JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(
+ JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withGroupInstanceId("leader-instance-id")
@@ -8341,7 +8341,7 @@
.withAssignment(assignment)
.build();
- SyncResult syncResult = context.sendGenericGroupSync(syncRequest);
+ SyncResult syncResult = context.sendClassicGroupSync(syncRequest);
// Simulate a successful write to log.
syncResult.appendFuture.complete(null);
@@ -8351,19 +8351,19 @@
assertEquals(assignment.get(0).assignment(), syncResult.syncFuture.get().assignment());
// Sync with unknown member.
- syncResult = context.sendGenericGroupSync(syncRequest.setMemberId("unknown-member-id"));
+ syncResult = context.sendClassicGroupSync(syncRequest.setMemberId("unknown-member-id"));
assertTrue(syncResult.records.isEmpty());
assertTrue(syncResult.syncFuture.isDone());
assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), syncResult.syncFuture.get().errorCode());
- assertEquals(GenericGroupMember.EMPTY_ASSIGNMENT, syncResult.syncFuture.get().assignment());
+ assertEquals(ClassicGroupMember.EMPTY_ASSIGNMENT, syncResult.syncFuture.get().assignment());
}
@Test
public void testSyncGroupFromIllegalGeneration() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -8372,13 +8372,13 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, true, true);
+ JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(joinRequest, true, true);
String memberId = joinResponse.memberId();
int generationId = joinResponse.generationId();
// Send the sync group with an invalid generation
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(memberId)
@@ -8395,7 +8395,7 @@
public void testSyncGroupAsLeaderAppendFailureTransformsError() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -8404,10 +8404,10 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinGroupResponseData joinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, true, true);
+ JoinGroupResponseData joinResponse = context.joinClassicGroupAndCompleteJoin(joinRequest, true, true);
// Send the sync group with an invalid generation
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(joinResponse.memberId())
@@ -8430,7 +8430,7 @@
// 2. join and sync with the first member and a new member
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -8438,12 +8438,12 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(joinRequest);
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(joinRequest);
assertTrue(followerJoinResult.records.isEmpty());
assertFalse(followerJoinResult.joinFuture.isDone());
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(leaderJoinResult.joinFuture.isDone());
@@ -8458,7 +8458,7 @@
String followerId = followerJoinResult.joinFuture.get().memberId();
// This shouldn't cause a rebalance since protocol information hasn't changed
- followerJoinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(followerId));
+ followerJoinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(followerId));
assertTrue(followerJoinResult.records.isEmpty());
assertTrue(followerJoinResult.joinFuture.isDone());
@@ -8474,8 +8474,8 @@
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -8485,12 +8485,12 @@
.withSessionTimeoutMs(5000)
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(joinRequest);
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(joinRequest);
assertTrue(followerJoinResult.records.isEmpty());
assertFalse(followerJoinResult.joinFuture.isDone());
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(leaderJoinResult.joinFuture.isDone());
@@ -8507,7 +8507,7 @@
// With no leader SyncGroup, the follower's sync request should fail with an error indicating
// that it should rejoin
- SyncResult followerSyncResult = context.sendGenericGroupSync(new SyncGroupRequestBuilder()
+ SyncResult followerSyncResult = context.sendClassicGroupSync(new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(followerId)
.withGenerationId(nextGenerationId)
@@ -8537,8 +8537,8 @@
// 2. join and sync with the first member and a new member
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -8548,12 +8548,12 @@
.withSessionTimeoutMs(5000)
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(UNKNOWN_MEMBER_ID));
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(UNKNOWN_MEMBER_ID));
assertTrue(followerJoinResult.records.isEmpty());
assertFalse(followerJoinResult.joinFuture.isDone());
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(leaderJoinResult.joinFuture.isDone());
@@ -8588,7 +8588,7 @@
.withAssignment(assignment)
.build();
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
syncRequest.setGenerationId(nextGenerationId)
);
@@ -8600,7 +8600,7 @@
assertEquals(leaderAssignment, syncResult.syncFuture.get().assignment());
// Sync group with follower to get new assignment.
- SyncResult followerSyncResult = context.sendGenericGroupSync(
+ SyncResult followerSyncResult = context.sendClassicGroupSync(
syncRequest
.setMemberId(followerId)
.setGenerationId(nextGenerationId)
@@ -8621,8 +8621,8 @@
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -8632,12 +8632,12 @@
.withSessionTimeoutMs(5000)
.build();
- JoinResult followerJoinResult = context.sendGenericGroupJoin(joinRequest);
+ JoinResult followerJoinResult = context.sendClassicGroupJoin(joinRequest);
assertTrue(followerJoinResult.records.isEmpty());
assertFalse(followerJoinResult.joinFuture.isDone());
- JoinResult leaderJoinResult = context.sendGenericGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
+ JoinResult leaderJoinResult = context.sendClassicGroupJoin(joinRequest.setMemberId(leaderJoinResponse.memberId()));
assertTrue(leaderJoinResult.records.isEmpty());
assertTrue(leaderJoinResult.joinFuture.isDone());
@@ -8661,7 +8661,7 @@
.withGenerationId(leaderJoinResponse.generationId())
.build();
- SyncResult followerSyncResult = context.sendGenericGroupSync(
+ SyncResult followerSyncResult = context.sendClassicGroupSync(
syncRequest
.setMemberId(followerId)
.setGenerationId(nextGenerationId)
@@ -8681,7 +8681,7 @@
.setAssignment(followerAssignment)
);
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
syncRequest
.setMemberId(leaderJoinResponse.memberId())
.setGenerationId(nextGenerationId)
@@ -8715,11 +8715,11 @@
public void testJoinGroupFromUnchangedLeaderShouldRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
// Join group from the leader should force the group to rebalance, which allows the
// leader to push new assignment when local metadata changes
- JoinResult leaderRejoinResult = context.sendGenericGroupJoin(
+ JoinResult leaderRejoinResult = context.sendClassicGroupJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(leaderJoinResponse.memberId())
@@ -8737,7 +8737,7 @@
public void testJoinGroupCompletionWhenPendingMemberJoins() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
// Set up a group in with a pending member. The test checks if the pending member joining
// completes the rebalancing operation
@@ -8750,7 +8750,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(request);
+ JoinResult joinResult = context.sendClassicGroupJoin(request);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -8764,7 +8764,7 @@
public void testJoinGroupCompletionWhenPendingMemberTimesOut() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
// Set up a group in with a pending member. The test checks if the timeout of the pending member will
// cause the group to return to a CompletingRebalance state.
@@ -8783,9 +8783,9 @@
public void testGenerationIdIncrementsOnRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(leaderJoinResponse.memberId())
@@ -8810,7 +8810,7 @@
"follower-instance-id"
);
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withGroupInstanceId("leader-instance-id")
@@ -8828,10 +8828,10 @@
.setMemberId(rebalanceResult.leaderId)
.setGenerationId(rebalanceResult.generationId);
- HeartbeatResponseData validHeartbeatResponse = context.sendGenericGroupHeartbeat(heartbeatRequest);
+ HeartbeatResponseData validHeartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest);
assertEquals(Errors.NONE.code(), validHeartbeatResponse.errorCode());
- assertThrows(FencedInstanceIdException.class, () -> context.sendGenericGroupHeartbeat(
+ assertThrows(FencedInstanceIdException.class, () -> context.sendClassicGroupHeartbeat(
heartbeatRequest
.setGroupInstanceId("leader-instance-id")
.setMemberId("invalid-member-id")
@@ -8848,14 +8848,14 @@
.setMemberId("member-id")
.setGenerationId(-1);
- assertThrows(UnknownMemberIdException.class, () -> context.sendGenericGroupHeartbeat(heartbeatRequest));
+ assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat(heartbeatRequest));
}
@Test
public void testHeartbeatDeadGroup() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
group.transitionTo(DEAD);
@@ -8864,16 +8864,16 @@
.setMemberId("member-id")
.setGenerationId(-1);
- assertThrows(CoordinatorNotAvailableException.class, () -> context.sendGenericGroupHeartbeat(heartbeatRequest));
+ assertThrows(CoordinatorNotAvailableException.class, () -> context.sendClassicGroupHeartbeat(heartbeatRequest));
}
@Test
public void testHeartbeatEmptyGroup() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
- group.add(new GenericGroupMember(
+ group.add(new ClassicGroupMember(
"member-id",
Optional.empty(),
"client-id",
@@ -8889,7 +8889,7 @@
.setMemberId("member-id")
.setGenerationId(0);
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(heartbeatRequest);
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest);
assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), heartbeatResponse.errorCode());
}
@@ -8897,9 +8897,9 @@
public void testHeartbeatUnknownMemberExistingGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
- assertThrows(UnknownMemberIdException.class, () -> context.sendGenericGroupHeartbeat(
+ assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat(
new HeartbeatRequestData()
.setGroupId("group-id")
.setMemberId("unknown-member-id")
@@ -8911,7 +8911,7 @@
public void testHeartbeatDuringPreparingRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
.withGroupId("group-id")
@@ -8919,7 +8919,7 @@
.withDefaultProtocolTypeAndProtocols()
.build();
- JoinResult joinResult = context.sendGenericGroupJoin(joinRequest, true);
+ JoinResult joinResult = context.sendClassicGroupJoin(joinRequest, true);
assertTrue(joinResult.records.isEmpty());
assertTrue(joinResult.joinFuture.isDone());
@@ -8927,11 +8927,11 @@
String memberId = joinResult.joinFuture.get().memberId();
- context.sendGenericGroupJoin(joinRequest.setMemberId(memberId));
+ context.sendClassicGroupJoin(joinRequest.setMemberId(memberId));
assertTrue(group.isInState(PREPARING_REBALANCE));
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(
new HeartbeatRequestData()
.setGroupId("group-id")
.setMemberId(memberId)
@@ -8945,10 +8945,10 @@
public void testHeartbeatDuringCompletingRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupResponseData leaderJoinResponse =
- context.joinGenericGroupAsDynamicMemberAndCompleteJoin(new JoinGroupRequestBuilder()
+ context.joinClassicGroupAsDynamicMemberAndCompleteJoin(new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
.withDefaultProtocolTypeAndProtocols()
@@ -8957,7 +8957,7 @@
assertEquals(1, leaderJoinResponse.generationId());
assertTrue(group.isInState(COMPLETING_REBALANCE));
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(
new HeartbeatRequestData()
.setGroupId("group-id")
.setMemberId(leaderJoinResponse.memberId())
@@ -8971,10 +8971,10 @@
public void testHeartbeatIllegalGeneration() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
assertThrows(IllegalGenerationException.class, () -> {
- context.sendGenericGroupHeartbeat(
+ context.sendClassicGroupHeartbeat(
new HeartbeatRequestData()
.setGroupId("group-id")
.setMemberId(leaderJoinResponse.memberId())
@@ -8987,9 +8987,9 @@
public void testValidHeartbeat() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(
new HeartbeatRequestData()
.setGroupId("group-id")
.setMemberId(leaderJoinResponse.memberId())
@@ -9000,16 +9000,16 @@
}
@Test
- public void testGenericGroupMemberSessionTimeout() throws Exception {
+ public void testClassicGroupMemberSessionTimeout() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Advance clock by session timeout to kick member out.
context.verifySessionExpiration(group, 5000);
- assertThrows(UnknownMemberIdException.class, () -> context.sendGenericGroupHeartbeat(
+ assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat(
new HeartbeatRequestData()
.setGroupId("group-id")
.setMemberId(leaderJoinResponse.memberId())
@@ -9018,10 +9018,10 @@
}
@Test
- public void testGenericGroupMemberHeartbeatMaintainsSession() throws Exception {
+ public void testClassicGroupMemberHeartbeatMaintainsSession() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
// Advance clock by 1/2 of session timeout.
assertNoOrEmptyResult(context.sleep(2500));
@@ -9031,25 +9031,25 @@
.setMemberId(leaderJoinResponse.memberId())
.setGenerationId(leaderJoinResponse.generationId());
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(heartbeatRequest);
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest);
assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode());
assertNoOrEmptyResult(context.sleep(2500));
- heartbeatResponse = context.sendGenericGroupHeartbeat(heartbeatRequest);
+ heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest);
assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode());
}
@Test
- public void testGenericGroupMemberSessionTimeoutDuringRebalance() throws Exception {
+ public void testClassicGroupMemberSessionTimeoutDuringRebalance() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Add a new member. This should trigger a rebalance. The new member has the
- // 'genericGroupNewMemberJoinTimeoutMs` session timeout, so it has a longer expiration than the existing member.
- JoinResult otherJoinResult = context.sendGenericGroupJoin(
+ // 'classicGroupNewMemberJoinTimeoutMs` session timeout, so it has a longer expiration than the existing member.
+ JoinResult otherJoinResult = context.sendClassicGroupJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
@@ -9071,13 +9071,13 @@
.setMemberId(leaderJoinResponse.memberId())
.setGenerationId(leaderJoinResponse.generationId());
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(heartbeatRequest);
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest);
assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode());
// Advance clock by first member's session timeout.
assertNoOrEmptyResult(context.sleep(5000));
- assertThrows(UnknownMemberIdException.class, () -> context.sendGenericGroupHeartbeat(heartbeatRequest));
+ assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupHeartbeat(heartbeatRequest));
// Advance clock by remaining rebalance timeout to complete join phase.
assertNoOrEmptyResult(context.sleep(2500));
@@ -9093,7 +9093,7 @@
public void testRebalanceCompletesBeforeMemberJoins() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
// Create a group with a single member
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
@@ -9105,7 +9105,7 @@
.withSessionTimeoutMs(5000)
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, true, true);
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAndCompleteJoin(joinRequest, true, true);
String firstMemberId = leaderJoinResponse.memberId();
int firstGenerationId = leaderJoinResponse.generationId();
@@ -9119,7 +9119,7 @@
.withGenerationId(firstGenerationId)
.build();
- SyncResult syncResult = context.sendGenericGroupSync(syncRequest);
+ SyncResult syncResult = context.sendClassicGroupSync(syncRequest);
// Simulate a successful write to the log.
syncResult.appendFuture.complete(null);
@@ -9129,8 +9129,8 @@
assertTrue(group.isInState(STABLE));
// Add a new dynamic member. This should trigger a rebalance. The new member has the
- // 'genericGroupNewMemberJoinTimeoutMs` session timeout, so it has a longer expiration than the existing member.
- JoinResult secondMemberJoinResult = context.sendGenericGroupJoin(
+ // 'classicGroupNewMemberJoinTimeoutMs` session timeout, so it has a longer expiration than the existing member.
+ JoinResult secondMemberJoinResult = context.sendClassicGroupJoin(
joinRequest
.setMemberId(UNKNOWN_MEMBER_ID)
.setGroupInstanceId(null)
@@ -9149,7 +9149,7 @@
for (int i = 0; i < 2; i++) {
assertNoOrEmptyResult(context.sleep(2500));
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(firstMemberHeartbeatRequest);
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(firstMemberHeartbeatRequest);
assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode());
}
@@ -9166,7 +9166,7 @@
String otherMemberId = secondMemberJoinResult.joinFuture.get().memberId();
- syncResult = context.sendGenericGroupSync(
+ syncResult = context.sendClassicGroupSync(
syncRequest
.setGroupInstanceId(null)
.setMemberId(otherMemberId)
@@ -9181,13 +9181,13 @@
assertTrue(group.isInState(STABLE));
// The unjoined (first) static member should be remained in the group before session timeout.
- assertThrows(IllegalGenerationException.class, () -> context.sendGenericGroupHeartbeat(firstMemberHeartbeatRequest));
+ assertThrows(IllegalGenerationException.class, () -> context.sendClassicGroupHeartbeat(firstMemberHeartbeatRequest));
// Now session timeout the unjoined (first) member. Still keeping the new member.
List<Errors> expectedErrors = Arrays.asList(Errors.NONE, Errors.NONE, Errors.REBALANCE_IN_PROGRESS);
for (Errors expectedError : expectedErrors) {
assertNoOrEmptyResult(context.sleep(2000));
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(
firstMemberHeartbeatRequest
.setMemberId(otherMemberId)
.setGenerationId(2)
@@ -9198,7 +9198,7 @@
assertEquals(1, group.size());
assertTrue(group.isInState(PREPARING_REBALANCE));
- JoinResult otherMemberRejoinResult = context.sendGenericGroupJoin(
+ JoinResult otherMemberRejoinResult = context.sendClassicGroupJoin(
joinRequest
.setMemberId(otherMemberId)
.setGroupInstanceId(null)
@@ -9211,7 +9211,7 @@
assertEquals(3, otherMemberRejoinResult.joinFuture.get().generationId());
assertTrue(group.isInState(COMPLETING_REBALANCE));
- SyncResult otherMemberResyncResult = context.sendGenericGroupSync(
+ SyncResult otherMemberResyncResult = context.sendClassicGroupSync(
syncRequest
.setGroupInstanceId(null)
.setMemberId(otherMemberId)
@@ -9229,7 +9229,7 @@
// heartbeating for a while to verify that no new rebalance is triggered unexpectedly.
for (int i = 0; i < 20; i++) {
assertNoOrEmptyResult(context.sleep(2000));
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(
firstMemberHeartbeatRequest
.setMemberId(otherMemberId)
.setGenerationId(3)
@@ -9243,9 +9243,9 @@
public void testSyncGroupEmptyAssignment() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(
new HeartbeatRequestData()
.setGroupId("group-id")
.setMemberId(leaderJoinResponse.memberId())
@@ -9263,7 +9263,7 @@
// which should remain stable throughout this test.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
// Create a group with a single member
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
@@ -9275,7 +9275,7 @@
.withSessionTimeoutMs(5000)
.build();
- JoinGroupResponseData leaderJoinResponse = context.joinGenericGroupAndCompleteJoin(joinRequest, true, true);
+ JoinGroupResponseData leaderJoinResponse = context.joinClassicGroupAndCompleteJoin(joinRequest, true, true);
String firstMemberId = leaderJoinResponse.memberId();
int firstGenerationId = leaderJoinResponse.generationId();
@@ -9283,7 +9283,7 @@
assertEquals(1, firstGenerationId);
assertTrue(group.isInState(COMPLETING_REBALANCE));
- SyncResult syncResult = context.sendGenericGroupSync(new SyncGroupRequestBuilder()
+ SyncResult syncResult = context.sendClassicGroupSync(new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(firstMemberId)
.withGenerationId(firstGenerationId)
@@ -9297,7 +9297,7 @@
assertTrue(group.isInState(STABLE));
// Add a new dynamic pending member.
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
joinRequest
.setMemberId(UNKNOWN_MEMBER_ID)
.setGroupInstanceId(null)
@@ -9320,7 +9320,7 @@
for (int i = 0; i < 2; i++) {
assertNoOrEmptyResult(context.sleep(2500));
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(heartbeatRequest);
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest);
assertEquals(Errors.NONE.code(), heartbeatResponse.errorCode());
}
@@ -9342,7 +9342,7 @@
int rebalanceTimeoutMs = 5000;
int sessionTimeoutMs = 5000;
List<JoinGroupResponseData> joinResponses = context.joinWithNMembers("group-id", 3, rebalanceTimeoutMs, sessionTimeoutMs);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Advance clock by 1/2 rebalance timeout.
assertNoOrEmptyResult(context.sleep(rebalanceTimeoutMs / 2));
@@ -9356,7 +9356,7 @@
List<ExpiredTimeout<Void, Record>> timeouts = context.sleep(rebalanceTimeoutMs / 2);
assertEquals(1, timeouts.size());
ExpiredTimeout<Void, Record> timeout = timeouts.get(0);
- assertEquals(genericGroupSyncKey("group-id"), timeout.key);
+ assertEquals(classicGroupSyncKey("group-id"), timeout.key);
assertEquals(
Collections.singletonList(newGroupMetadataRecordWithCurrentState(group, MetadataVersion.latest())),
timeout.result.records()
@@ -9380,7 +9380,7 @@
int rebalanceTimeoutMs = 5000;
int sessionTimeoutMs = 5000;
List<JoinGroupResponseData> joinResponses = context.joinWithNMembers("group-id", 3, rebalanceTimeoutMs, sessionTimeoutMs);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Advance clock by 1/2 rebalance timeout.
assertNoOrEmptyResult(context.sleep(rebalanceTimeoutMs / 2));
@@ -9390,7 +9390,7 @@
joinResponses.forEach(response -> context.verifyHeartbeat(group.groupId(), response, Errors.NONE));
// Leader sends a sync group request.
- SyncResult syncResult = context.sendGenericGroupSync(new SyncGroupRequestBuilder()
+ SyncResult syncResult = context.sendClassicGroupSync(new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withGenerationId(1)
.withMemberId(joinResponses.get(0).memberId())
@@ -9410,7 +9410,7 @@
List<ExpiredTimeout<Void, Record>> timeouts = context.sleep(rebalanceTimeoutMs / 2);
assertEquals(1, timeouts.size());
ExpiredTimeout<Void, Record> timeout = timeouts.get(0);
- assertEquals(genericGroupSyncKey("group-id"), timeout.key);
+ assertEquals(classicGroupSyncKey("group-id"), timeout.key);
assertTrue(timeout.result.records().isEmpty());
// Leader should be able to heartbeat
@@ -9432,7 +9432,7 @@
int rebalanceTimeoutMs = 5000;
int sessionTimeoutMs = 5000;
List<JoinGroupResponseData> joinResponses = context.joinWithNMembers("group-id", 3, rebalanceTimeoutMs, sessionTimeoutMs);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// Advance clock by 1/2 rebalance timeout.
assertNoOrEmptyResult(context.sleep(rebalanceTimeoutMs / 2));
@@ -9444,7 +9444,7 @@
// Followers send sync group requests.
List<CompletableFuture<SyncGroupResponseData>> followerSyncFutures = joinResponses.subList(1, 3).stream()
.map(response -> {
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withGenerationId(1)
@@ -9461,7 +9461,7 @@
List<ExpiredTimeout<Void, Record>> timeouts = context.sleep(rebalanceTimeoutMs / 2);
assertEquals(1, timeouts.size());
ExpiredTimeout<Void, Record> timeout = timeouts.get(0);
- assertEquals(genericGroupSyncKey("group-id"), timeout.key);
+ assertEquals(classicGroupSyncKey("group-id"), timeout.key);
assertTrue(timeout.result.records().isEmpty());
// Follower sync responses should fail.
@@ -9492,7 +9492,7 @@
int rebalanceTimeoutMs = 5000;
int sessionTimeoutMs = 5000;
List<JoinGroupResponseData> joinResponses = context.joinWithNMembers("group-id", 3, rebalanceTimeoutMs, sessionTimeoutMs);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
String leaderId = joinResponses.get(0).memberId();
// Advance clock by 1/2 rebalance timeout.
@@ -9504,7 +9504,7 @@
// All members send sync group requests.
List<CompletableFuture<SyncGroupResponseData>> syncFutures = joinResponses.stream().map(response -> {
- SyncResult syncResult = context.sendGenericGroupSync(
+ SyncResult syncResult = context.sendClassicGroupSync(
new SyncGroupRequestBuilder()
.withGroupId("group-id")
.withGenerationId(1)
@@ -9550,7 +9550,7 @@
public void testHeartbeatDuringRebalanceCausesRebalanceInProgress() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
// First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts)
JoinGroupRequestData joinRequest = new JoinGroupRequestBuilder()
@@ -9562,13 +9562,13 @@
.build();
JoinGroupResponseData leaderJoinResponse =
- context.joinGenericGroupAsDynamicMemberAndCompleteJoin(joinRequest);
+ context.joinClassicGroupAsDynamicMemberAndCompleteJoin(joinRequest);
assertEquals(1, leaderJoinResponse.generationId());
assertTrue(group.isInState(COMPLETING_REBALANCE));
// Then join with a new consumer to trigger a rebalance
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
joinRequest.setMemberId(UNKNOWN_MEMBER_ID)
);
@@ -9581,16 +9581,16 @@
.setMemberId(leaderJoinResponse.memberId())
.setGenerationId(leaderJoinResponse.generationId());
- HeartbeatResponseData heartbeatResponse = context.sendGenericGroupHeartbeat(heartbeatRequest);
+ HeartbeatResponseData heartbeatResponse = context.sendClassicGroupHeartbeat(heartbeatRequest);
assertEquals(Errors.REBALANCE_IN_PROGRESS.code(), heartbeatResponse.errorCode());
}
@Test
public void testListGroups() {
String consumerGroupId = "consumer-group-id";
- String genericGroupId = "generic-group-id";
+ String classicGroupId = "generic-group-id";
String memberId1 = Uuid.randomUuid().toString();
- String genericGroupType = "generic";
+ String classicGroupType = "generic";
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
@@ -9600,17 +9600,17 @@
.withConsumerGroup(new ConsumerGroupBuilder(consumerGroupId, 10))
.build();
context.replay(newGroupMetadataRecord(
- genericGroupId,
+ classicGroupId,
new GroupMetadataValue()
.setMembers(Collections.emptyList())
.setGeneration(2)
.setLeader(null)
- .setProtocolType(genericGroupType)
+ .setProtocolType(classicGroupType)
.setProtocol("range")
.setCurrentStateTimestamp(context.time.milliseconds()),
MetadataVersion.latest()));
context.commit();
- GenericGroup genericGroup = context.groupMetadataManager.getOrMaybeCreateGenericGroup(genericGroupId, false);
+ ClassicGroup classicGroup = context.groupMetadataManager.getOrMaybeCreateClassicGroup(classicGroupId, false);
context.replay(RecordHelpers.newMemberSubscriptionRecord(consumerGroupId, new ConsumerGroupMember.Builder(memberId1)
.setSubscribedTopicNames(Collections.singletonList(fooTopicName))
.build()));
@@ -9622,8 +9622,8 @@
Map<String, ListGroupsResponseData.ListedGroup> expectAllGroupMap =
Stream.of(
new ListGroupsResponseData.ListedGroup()
- .setGroupId(genericGroup.groupId())
- .setProtocolType(genericGroupType)
+ .setGroupId(classicGroup.groupId())
+ .setProtocolType(classicGroupType)
.setGroupState(EMPTY.toString()),
new ListGroupsResponseData.ListedGroup()
.setGroupId(consumerGroupId)
@@ -9639,8 +9639,8 @@
expectAllGroupMap =
Stream.of(
new ListGroupsResponseData.ListedGroup()
- .setGroupId(genericGroup.groupId())
- .setProtocolType(genericGroupType)
+ .setGroupId(classicGroup.groupId())
+ .setProtocolType(classicGroupType)
.setGroupState(EMPTY.toString()),
new ListGroupsResponseData.ListedGroup()
.setGroupId(consumerGroupId)
@@ -9654,8 +9654,8 @@
.collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity()));
expectAllGroupMap = Stream.of(
new ListGroupsResponseData.ListedGroup()
- .setGroupId(genericGroup.groupId())
- .setProtocolType(genericGroupType)
+ .setGroupId(classicGroup.groupId())
+ .setProtocolType(classicGroupType)
.setGroupState(EMPTY.toString())
).collect(Collectors.toMap(ListGroupsResponseData.ListedGroup::groupId, Function.identity()));
@@ -9846,7 +9846,7 @@
groupMetadataValue,
MetadataVersion.latest()
));
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
context.verifyDescribeGroupsReturnsDeadGroup("group-id");
@@ -10017,7 +10017,7 @@
return assignmentMap;
}
- private static List<JoinGroupResponseMember> toJoinResponseMembers(GenericGroup group) {
+ private static List<JoinGroupResponseMember> toJoinResponseMembers(ClassicGroup group) {
List<JoinGroupResponseMember> members = new ArrayList<>();
String protocolName = group.protocolName().get();
group.allMembers().forEach(member -> {
@@ -10035,8 +10035,8 @@
private static void checkJoinGroupResponse(
JoinGroupResponseData expectedResponse,
JoinGroupResponseData actualResponse,
- GenericGroup group,
- GenericGroupState expectedState,
+ ClassicGroup group,
+ ClassicGroupState expectedState,
Set<String> expectedGroupInstanceIds
) {
assertEquals(expectedResponse, actualResponse);
@@ -10063,10 +10063,10 @@
rebalanceTimeoutMs,
longSessionTimeoutMs
);
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("group-id", false);
// New member joins
- JoinResult joinResult = context.sendGenericGroupJoin(
+ JoinResult joinResult = context.sendClassicGroupJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
@@ -10098,7 +10098,7 @@
);
// Send a special leave group request from static follower, moving group towards PreparingRebalance
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(Collections.singletonList(
@@ -10130,10 +10130,10 @@
public void testPendingMembersLeaveGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupResponseData pendingJoinResponse = context.setupGroupWithPendingMember(group).pendingMemberResponse;
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(Collections.singletonList(
@@ -10161,9 +10161,9 @@
public void testLeaveGroupInvalidGroup() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
- assertThrows(UnknownMemberIdException.class, () -> context.sendGenericGroupLeave(
+ assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("invalid-group-id")
));
@@ -10174,7 +10174,7 @@
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- assertThrows(UnknownMemberIdException.class, () -> context.sendGenericGroupLeave(
+ assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("unknown-group-id")
.setMembers(Collections.singletonList(
@@ -10188,9 +10188,9 @@
public void testLeaveGroupUnknownMemberIdExistingGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
- context.joinGenericGroupAsDynamicMemberAndCompleteJoin(
+ context.joinClassicGroupAsDynamicMemberAndCompleteJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
@@ -10198,7 +10198,7 @@
.build()
);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(Collections.singletonList(
@@ -10222,10 +10222,10 @@
public void testLeaveDeadGroup() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
group.transitionTo(DEAD);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(Collections.singletonList(
@@ -10245,9 +10245,9 @@
public void testValidLeaveGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
- JoinGroupResponseData joinResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(
+ JoinGroupResponseData joinResponse = context.joinClassicGroupAsDynamicMemberAndCompleteJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
@@ -10256,7 +10256,7 @@
);
// Dynamic member leaves. The group becomes empty.
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(Collections.singletonList(
@@ -10286,9 +10286,9 @@
public void testLeaveGroupWithFencedInstanceId() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
- context.joinGenericGroupAndCompleteJoin(
+ context.joinClassicGroupAndCompleteJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withGroupInstanceId("group-instance-id")
@@ -10299,7 +10299,7 @@
true
);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(Collections.singletonList(
@@ -10324,9 +10324,9 @@
public void testLeaveGroupStaticMemberWithUnknownMemberId() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
- context.joinGenericGroupAndCompleteJoin(
+ context.joinClassicGroupAndCompleteJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withGroupInstanceId("group-instance-id")
@@ -10338,7 +10338,7 @@
);
// Having unknown member id will not affect the request processing due to valid group instance id.
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(Collections.singletonList(
@@ -10366,7 +10366,7 @@
"follower-instance-id"
);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(
@@ -10399,7 +10399,7 @@
"follower-instance-id"
);
- assertThrows(UnknownMemberIdException.class, () -> context.sendGenericGroupLeave(
+ assertThrows(UnknownMemberIdException.class, () -> context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("invalid-group-id") // Invalid group id
.setMembers(
@@ -10423,7 +10423,7 @@
"follower-instance-id"
);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(
@@ -10459,7 +10459,7 @@
"follower-instance-id"
);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(
@@ -10488,10 +10488,10 @@
public void testPendingMemberBatchLeaveGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
JoinGroupResponseData pendingJoinResponse = context.setupGroupWithPendingMember(group).pendingMemberResponse;
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(
@@ -10520,10 +10520,10 @@
public void testJoinedMemberPendingMemberBatchLeaveGroup() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
PendingMemberGroupResult pendingMemberGroupResult = context.setupGroupWithPendingMember(group);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(
@@ -10557,10 +10557,10 @@
public void testJoinedMemberPendingMemberBatchLeaveGroupWithUnknownMember() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
PendingMemberGroupResult pendingMemberGroupResult = context.setupGroupWithPendingMember(group);
- CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendGenericGroupLeave(
+ CoordinatorResult<LeaveGroupResponseData, Record> leaveResult = context.sendClassicGroupLeave(
new LeaveGroupRequestData()
.setGroupId("group-id")
.setMembers(
@@ -10597,10 +10597,10 @@
}
@Test
- public void testGenericGroupDelete() {
+ public void testClassicGroupDelete() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
List<Record> expectedRecords = Collections.singletonList(RecordHelpers.newGroupMetadataTombstoneRecord("group-id"));
List<Record> records = new ArrayList<>();
@@ -10609,10 +10609,10 @@
}
@Test
- public void testGenericGroupMaybeDelete() {
+ public void testClassicGroupMaybeDelete() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = context.createGenericGroup("group-id");
+ ClassicGroup group = context.createClassicGroup("group-id");
List<Record> expectedRecords = Collections.singletonList(RecordHelpers.newGroupMetadataTombstoneRecord("group-id"));
List<Record> records = new ArrayList<>();
@@ -10672,27 +10672,27 @@
}
@Test
- public void testGenericGroupCompletedRebalanceSensor() throws Exception {
+ public void testClassicGroupCompletedRebalanceSensor() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.joinGenericGroupAsDynamicMemberAndCompleteRebalance("group-id");
- verify(context.metrics).record(GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
+ context.joinClassicGroupAsDynamicMemberAndCompleteRebalance("group-id");
+ verify(context.metrics).record(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
}
@Test
- public void testGenericGroupRebalanceSensor() throws Exception {
+ public void testClassicGroupRebalanceSensor() throws Exception {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
- context.joinGenericGroupAsDynamicMemberAndCompleteJoin(
+ context.joinClassicGroupAsDynamicMemberAndCompleteJoin(
new JoinGroupRequestBuilder()
.withGroupId("group-id")
.withMemberId(UNKNOWN_MEMBER_ID)
.withDefaultProtocolTypeAndProtocols()
.build()
);
- verify(context.metrics).record(GENERIC_GROUP_REBALANCES_SENSOR_NAME);
+ verify(context.metrics).record(CLASSIC_GROUP_REBALANCES_SENSOR_NAME);
}
@Test
@@ -10737,35 +10737,35 @@
}
@Test
- public void testOnGenericGroupStateTransition() {
+ public void testOnClassicGroupStateTransition() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- // Creating a generic group should increment metric.
- GenericGroup group = context.createGenericGroup("group-id");
- verify(context.metrics, times(1)).onGenericGroupStateTransition(null, EMPTY);
+ // Creating a classic group should increment metric.
+ ClassicGroup group = context.createClassicGroup("group-id");
+ verify(context.metrics, times(1)).onClassicGroupStateTransition(null, EMPTY);
// Replaying a new group should not increment metric as the group was already created.
context.replay(RecordHelpers.newGroupMetadataRecord(group, Collections.emptyMap(), MetadataVersion.LATEST_PRODUCTION));
- verify(context.metrics, times(1)).onGenericGroupStateTransition(null, EMPTY);
+ verify(context.metrics, times(1)).onClassicGroupStateTransition(null, EMPTY);
// Loading a tombstone should remove group and decrement metric.
- context.createGenericGroup("group-id");
+ context.createClassicGroup("group-id");
context.replay(RecordHelpers.newGroupMetadataTombstoneRecord("group-id"));
- verify(context.metrics, times(1)).onGenericGroupStateTransition(EMPTY, null);
+ verify(context.metrics, times(1)).onClassicGroupStateTransition(EMPTY, null);
assertThrows(GroupIdNotFoundException.class, () -> context.groupMetadataManager.group("group-id"));
// Replaying a tombstone for a group that has already been deleted should not decrement metric.
context.replay(RecordHelpers.newGroupMetadataTombstoneRecord("group-id"));
- verify(context.metrics, times(1)).onGenericGroupStateTransition(EMPTY, null);
+ verify(context.metrics, times(1)).onClassicGroupStateTransition(EMPTY, null);
}
@Test
- public void testOnGenericGroupStateTransitionOnLoading() {
+ public void testOnClassicGroupStateTransitionOnLoading() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.build();
- GenericGroup group = new GenericGroup(
+ ClassicGroup group = new ClassicGroup(
new LogContext(),
"group-id",
EMPTY,
@@ -10782,8 +10782,8 @@
context.replay(RecordHelpers.newGroupMetadataTombstoneRecord("group-id"));
});
- verify(context.metrics, times(1)).onGenericGroupStateTransition(null, EMPTY);
- verify(context.metrics, times(1)).onGenericGroupStateTransition(EMPTY, null);
+ verify(context.metrics, times(1)).onClassicGroupStateTransition(null, EMPTY);
+ verify(context.metrics, times(1)).onClassicGroupStateTransition(EMPTY, null);
}
@Test
@@ -10838,7 +10838,7 @@
timeouts.forEach(timeout -> assertEquals(EMPTY_RESULT, timeout.result));
}
- private static List<String> verifyGenericGroupJoinResponses(
+ private static List<String> verifyClassicGroupJoinResponses(
List<JoinResult> joinResults,
int expectedSuccessCount,
Errors expectedFailure
@@ -10898,7 +10898,7 @@
}
private static Record newGroupMetadataRecordWithCurrentState(
- GenericGroup group,
+ ClassicGroup group,
MetadataVersion metadataVersion
) {
return RecordHelpers.newGroupMetadataRecord(group, group.groupAssignment(), metadataVersion);
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java
index b86ddc7..8731cca 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java
@@ -52,9 +52,9 @@
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
-import org.apache.kafka.coordinator.group.generic.GenericGroup;
-import org.apache.kafka.coordinator.group.generic.GenericGroupMember;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroup;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
import org.apache.kafka.image.MetadataImage;
@@ -547,11 +547,11 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create a dead group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
- group.transitionTo(GenericGroupState.DEAD);
+ group.transitionTo(ClassicGroupState.DEAD);
// Verify that the request is rejected with the correct exception.
assertThrows(CoordinatorNotAvailableException.class, () -> context.commitOffset(
@@ -577,7 +577,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -606,7 +606,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -615,7 +615,7 @@
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
@@ -643,7 +643,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -676,7 +676,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -709,7 +709,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -718,7 +718,7 @@
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
@@ -746,7 +746,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -755,7 +755,7 @@
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
@@ -781,7 +781,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -790,10 +790,10 @@
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
- group.transitionTo(GenericGroupState.STABLE);
+ group.transitionTo(ClassicGroupState.STABLE);
CoordinatorResult<OffsetCommitResponseData, Record> result = context.commitOffset(
new OffsetCommitRequestData()
@@ -849,24 +849,24 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create a group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
// Add member.
- GenericGroupMember member = mkGenericMember("member", Optional.empty());
+ ClassicGroupMember member = mkGenericMember("member", Optional.empty());
group.add(member);
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
- group.transitionTo(GenericGroupState.STABLE);
+ group.transitionTo(ClassicGroupState.STABLE);
// Schedule session timeout. This would be normally done when
// the group transitions to stable.
- context.groupMetadataManager.rescheduleGenericGroupMemberHeartbeat(group, member);
+ context.groupMetadataManager.rescheduleClassicGroupMemberHeartbeat(group, member);
// Advance time by half of the session timeout. No timeouts are
// expired.
@@ -952,7 +952,7 @@
);
// A generic should have been created.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
false
);
@@ -1492,20 +1492,20 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create a group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
// Add member.
- GenericGroupMember member = mkGenericMember("member", Optional.empty());
+ ClassicGroupMember member = mkGenericMember("member", Optional.empty());
group.add(member);
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
- group.transitionTo(GenericGroupState.STABLE);
+ group.transitionTo(ClassicGroupState.STABLE);
CoordinatorResult<TxnOffsetCommitResponseData, Record> result = context.commitTransactionalOffset(
new TxnOffsetCommitRequestData()
@@ -1585,7 +1585,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -1614,20 +1614,20 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create a group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
// Add member.
- GenericGroupMember member = mkGenericMember("member", Optional.empty());
+ ClassicGroupMember member = mkGenericMember("member", Optional.empty());
group.add(member);
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
- group.transitionTo(GenericGroupState.STABLE);
+ group.transitionTo(ClassicGroupState.STABLE);
assertThrows(IllegalGenerationException.class, () -> context.commitTransactionalOffset(
new TxnOffsetCommitRequestData()
@@ -1653,11 +1653,11 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create a dead group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"group",
true
);
- group.transitionTo(GenericGroupState.DEAD);
+ group.transitionTo(ClassicGroupState.DEAD);
List<OffsetFetchRequestData.OffsetFetchRequestTopics> request = Arrays.asList(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
@@ -1861,11 +1861,11 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create a dead group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"group",
true
);
- group.transitionTo(GenericGroupState.DEAD);
+ group.transitionTo(ClassicGroupState.DEAD);
assertEquals(Collections.emptyList(), context.fetchAllOffsets("group", Long.MAX_VALUE));
}
@@ -2083,7 +2083,7 @@
@Test
public void testGenericGroupOffsetDelete() {
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -2095,7 +2095,7 @@
@Test
public void testGenericGroupOffsetDeleteWithErrors() {
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -2155,8 +2155,8 @@
public void testDeleteGroupAllOffsets(Group.GroupType groupType) {
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
switch (groupType) {
- case GENERIC:
- context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ case CLASSIC:
+ context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -2496,7 +2496,7 @@
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
// Create an empty group.
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
"foo",
true
);
@@ -2505,10 +2505,10 @@
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
// Transition to next generation.
- group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
+ group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
group.initNextGeneration();
assertEquals(1, group.generationId());
- group.transitionTo(GenericGroupState.STABLE);
+ group.transitionTo(ClassicGroupState.STABLE);
CoordinatorResult<OffsetCommitResponseData, Record> result = context.commitOffset(
new OffsetCommitRequestData()
@@ -2588,7 +2588,7 @@
@Test
public void testOffsetDeletionsSensor() {
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
- GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("foo", true);
+ ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup("foo", true);
context.commitOffset("foo", "bar", 0, 100L, 0);
context.commitOffset("foo", "bar", 1, 150L, 0);
@@ -2659,11 +2659,11 @@
));
}
- private GenericGroupMember mkGenericMember(
+ private ClassicGroupMember mkGenericMember(
String memberId,
Optional<String> groupInstanceId
) {
- return new GenericGroupMember(
+ return new ClassicGroupMember(
memberId,
groupInstanceId,
"client-id",
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java
index 4c8690f..cb6c583 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/RecordHelpersTest.java
@@ -42,9 +42,9 @@
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
-import org.apache.kafka.coordinator.group.generic.GenericGroup;
-import org.apache.kafka.coordinator.group.generic.GenericGroupMember;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroup;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.common.MetadataVersion;
@@ -525,10 +525,10 @@
.setMembers(expectedMembers),
expectedGroupMetadataValueVersion));
- GenericGroup group = new GenericGroup(
+ ClassicGroup group = new ClassicGroup(
new LogContext(),
"group-id",
- GenericGroupState.PREPARING_REBALANCE,
+ ClassicGroupState.PREPARING_REBALANCE,
time,
mock(GroupCoordinatorMetricsShard.class)
);
@@ -541,7 +541,7 @@
.setName("range")
.setMetadata(member.subscription()));
- group.add(new GenericGroupMember(
+ group.add(new ClassicGroupMember(
member.memberId(),
Optional.of(member.groupInstanceId()),
member.clientId(),
@@ -550,7 +550,7 @@
member.sessionTimeout(),
"consumer",
protocols,
- GenericGroupMember.EMPTY_ASSIGNMENT
+ ClassicGroupMember.EMPTY_ASSIGNMENT
));
assignment.put(member.memberId(), member.assignment());
@@ -596,10 +596,10 @@
.setAssignment(new byte[]{1, 2})
);
- GenericGroup group = new GenericGroup(
+ ClassicGroup group = new ClassicGroup(
new LogContext(),
"group-id",
- GenericGroupState.PREPARING_REBALANCE,
+ ClassicGroupState.PREPARING_REBALANCE,
time,
mock(GroupCoordinatorMetricsShard.class)
);
@@ -610,7 +610,7 @@
.setName("range")
.setMetadata(null));
- group.add(new GenericGroupMember(
+ group.add(new ClassicGroupMember(
member.memberId(),
Optional.of(member.groupInstanceId()),
member.clientId(),
@@ -648,10 +648,10 @@
.setAssignment(null)
);
- GenericGroup group = new GenericGroup(
+ ClassicGroup group = new ClassicGroup(
new LogContext(),
"group-id",
- GenericGroupState.PREPARING_REBALANCE,
+ ClassicGroupState.PREPARING_REBALANCE,
time,
mock(GroupCoordinatorMetricsShard.class)
);
@@ -662,7 +662,7 @@
.setName("range")
.setMetadata(member.subscription()));
- group.add(new GenericGroupMember(
+ group.add(new ClassicGroupMember(
member.memberId(),
Optional.of(member.groupInstanceId()),
member.clientId(),
@@ -708,10 +708,10 @@
.setMembers(expectedMembers),
expectedGroupMetadataValueVersion));
- GenericGroup group = new GenericGroup(
+ ClassicGroup group = new ClassicGroup(
new LogContext(),
"group-id",
- GenericGroupState.PREPARING_REBALANCE,
+ ClassicGroupState.PREPARING_REBALANCE,
time,
mock(GroupCoordinatorMetricsShard.class)
);
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMemberTest.java
similarity index 92%
rename from group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupMemberTest.java
rename to group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMemberTest.java
index 075f418..9eb9156 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupMemberTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupMemberTest.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.kafka.coordinator.group.generic;
+package org.apache.kafka.coordinator.group.classic;
import org.apache.kafka.common.message.DescribeGroupsResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol;
@@ -28,14 +28,14 @@
import java.util.Set;
import java.util.concurrent.CompletableFuture;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupMember.EMPTY_ASSIGNMENT;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupMember.EMPTY_ASSIGNMENT;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
-public class GenericGroupMemberTest {
+public class ClassicGroupMemberTest {
@Test
public void testMatchesSupportedProtocols() {
@@ -44,7 +44,7 @@
.setName("range")
.setMetadata(new byte[]{0}));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -101,7 +101,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -135,7 +135,7 @@
.setName("roundrobin")
.setMetadata(new byte[]{1}));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -153,7 +153,7 @@
@Test
public void testMetadataRaisesOnUnsupportedProtocol() {
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -180,7 +180,7 @@
.setName("roundrobin")
.setMetadata(new byte[]{1}));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -199,7 +199,7 @@
@Test
public void testHasValidGroupInstanceId() {
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -229,12 +229,12 @@
expectedProtocolNames.add("range");
expectedProtocolNames.add("roundrobin");
- assertEquals(expectedProtocolNames, GenericGroupMember.plainProtocolSet(protocolCollection));
+ assertEquals(expectedProtocolNames, ClassicGroupMember.plainProtocolSet(protocolCollection));
}
@Test
public void testHasHeartbeatSatisfied() {
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -262,7 +262,7 @@
@Test
public void testDescribeNoMetadata() {
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
@@ -295,7 +295,7 @@
.setMetadata(new byte[]{0})
).iterator());
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
"member",
Optional.of("group-instance-id"),
"client-id",
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java
similarity index 92%
rename from group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java
rename to group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java
index 612b21f..f523d85 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/generic/GenericGroupTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.kafka.coordinator.group.generic;
+package org.apache.kafka.coordinator.group.classic;
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
@@ -51,11 +51,11 @@
import java.util.concurrent.CompletableFuture;
import static org.apache.kafka.common.utils.Utils.mkSet;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+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.EMPTY;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPARING_REBALANCE;
+import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.STABLE;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -67,7 +67,7 @@
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
-public class GenericGroupTest {
+public class ClassicGroupTest {
private final String protocolType = "consumer";
private final String groupInstanceId = "groupInstanceId";
private final String memberId = "memberId";
@@ -82,11 +82,11 @@
new TopicPartition("__consumer_offsets", 0)
);
- private GenericGroup group = null;
+ private ClassicGroup group = null;
@BeforeEach
public void initialize() {
- group = new GenericGroup(logContext, "groupId", EMPTY, Time.SYSTEM, metrics);
+ group = new ClassicGroup(logContext, "groupId", EMPTY, Time.SYSTEM, metrics);
}
@Test
@@ -242,7 +242,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member1 = new GenericGroupMember(
+ ClassicGroupMember member1 = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -263,7 +263,7 @@
.setName("range")
.setMetadata(new byte[0]));
- GenericGroupMember member2 = new GenericGroupMember(
+ ClassicGroupMember member2 = new ClassicGroupMember(
"member2",
Optional.empty(),
clientId,
@@ -279,7 +279,7 @@
assertTrue(group.selectProtocol().equals("range") ||
group.selectProtocol().equals("roundrobin"));
- GenericGroupMember member3 = new GenericGroupMember(
+ ClassicGroupMember member3 = new ClassicGroupMember(
"member3",
Optional.empty(),
clientId,
@@ -311,7 +311,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member1 = new GenericGroupMember(
+ ClassicGroupMember member1 = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -333,7 +333,7 @@
.setMetadata(new byte[0]));
- GenericGroupMember member2 = new GenericGroupMember(
+ ClassicGroupMember member2 = new ClassicGroupMember(
"member2",
Optional.empty(),
clientId,
@@ -359,7 +359,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member1 = new GenericGroupMember(
+ ClassicGroupMember member1 = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -394,7 +394,7 @@
Collections.singletonList("foo")
)).array()));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -425,7 +425,7 @@
.setName("range")
.setMetadata(new byte[0]));
- GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
+ ClassicGroupMember memberWithFaultyProtocol = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -454,7 +454,7 @@
.setName("range")
.setMetadata(new byte[0]));
- GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+ ClassicGroupMember memberWithNonConsumerProtocol = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -480,7 +480,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -523,7 +523,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -568,7 +568,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.of(groupInstanceId),
clientId,
@@ -606,7 +606,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -640,7 +640,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -670,7 +670,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -701,7 +701,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -728,7 +728,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -754,7 +754,7 @@
assertFalse(group.hasMemberId(memberId));
assertTrue(group.isPendingMember(memberId));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -788,7 +788,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.of(groupInstanceId),
clientId,
@@ -826,7 +826,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -851,7 +851,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.of(groupInstanceId),
clientId,
@@ -876,7 +876,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -902,7 +902,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember leader = new GenericGroupMember(
+ ClassicGroupMember leader = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -917,7 +917,7 @@
assertTrue(group.isLeader(memberId));
assertFalse(leader.isAwaitingJoin());
- GenericGroupMember newLeader = new GenericGroupMember(
+ ClassicGroupMember newLeader = new ClassicGroupMember(
"new-leader",
Optional.empty(),
clientId,
@@ -929,7 +929,7 @@
);
group.add(newLeader, new CompletableFuture<>());
- GenericGroupMember newMember = new GenericGroupMember(
+ ClassicGroupMember newMember = new ClassicGroupMember(
"new-member",
Optional.empty(),
clientId,
@@ -952,7 +952,7 @@
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember leader = new GenericGroupMember(
+ ClassicGroupMember leader = new ClassicGroupMember(
memberId,
Optional.empty(),
clientId,
@@ -967,7 +967,7 @@
assertTrue(group.isLeader(memberId));
assertTrue(leader.isAwaitingJoin());
- GenericGroupMember newMember = new GenericGroupMember(
+ ClassicGroupMember newMember = new ClassicGroupMember(
"new-member",
Optional.empty(),
clientId,
@@ -989,7 +989,7 @@
group.validateOffsetCommit("", "", -1, false);
// Add a member.
- group.add(new GenericGroupMember(
+ group.add(new ClassicGroupMember(
"member-id",
Optional.of("instance-id"),
"",
@@ -1066,7 +1066,7 @@
protocols.add(new JoinGroupRequestProtocol()
.setName("roundrobin")
.setMetadata(new byte[0]));
- GenericGroupMember member = new GenericGroupMember(
+ ClassicGroupMember member = new ClassicGroupMember(
memberId,
Optional.of(groupInstanceId),
clientId,
@@ -1116,7 +1116,7 @@
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty());
MockTime time = new MockTime();
long currentStateTimestamp = time.milliseconds();
- GenericGroup group = new GenericGroup(new LogContext(), "groupId", EMPTY, time, mock(GroupCoordinatorMetricsShard.class));
+ ClassicGroup group = new ClassicGroup(new LogContext(), "groupId", EMPTY, time, mock(GroupCoordinatorMetricsShard.class));
// 1. Test no protocol type. Simple consumer case, Base timestamp based off of last commit timestamp.
Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
@@ -1133,7 +1133,7 @@
.setMetadata(ConsumerProtocol.serializeSubscription(
new ConsumerPartitionAssignor.Subscription(Collections.singletonList("topic"))).array()));
- GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+ ClassicGroupMember memberWithNonConsumerProtocol = new ClassicGroupMember(
"memberWithNonConsumerProtocol",
Optional.empty(),
clientId,
@@ -1161,7 +1161,7 @@
// 4. Test consumer protocol type + subscribed topics + Stable state. Base timestamp based off of last commit timestamp.
group.remove("memberWithNonConsumerProtocol");
- GenericGroupMember memberWithConsumerProtocol = new GenericGroupMember(
+ ClassicGroupMember memberWithConsumerProtocol = new ClassicGroupMember(
"memberWithConsumerProtocol",
Optional.empty(),
clientId,
@@ -1191,7 +1191,7 @@
@Test
public void testIsSubscribedToTopic() {
- GenericGroup group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM, mock(GroupCoordinatorMetricsShard.class));
+ ClassicGroup group = new ClassicGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM, mock(GroupCoordinatorMetricsShard.class));
// 1. group has no protocol type => not subscribed
assertFalse(group.isSubscribedToTopic("topic"));
@@ -1203,7 +1203,7 @@
.setMetadata(ConsumerProtocol.serializeSubscription(
new ConsumerPartitionAssignor.Subscription(Collections.singletonList("topic"))).array()));
- GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
+ ClassicGroupMember memberWithNonConsumerProtocol = new ClassicGroupMember(
"memberWithNonConsumerProtocol",
Optional.empty(),
clientId,
@@ -1223,7 +1223,7 @@
// 3. group uses consumer group protocol type but empty members => not subscribed
group.remove("memberWithNonConsumerProtocol");
- GenericGroupMember memberWithConsumerProtocol = new GenericGroupMember(
+ ClassicGroupMember memberWithConsumerProtocol = new ClassicGroupMember(
"memberWithConsumerProtocol",
Optional.empty(),
clientId,
@@ -1258,24 +1258,24 @@
// Confirm metrics is not updated when a new GenericGroup is created but only when the group transitions
// its state.
GroupCoordinatorMetricsShard metrics = mock(GroupCoordinatorMetricsShard.class);
- GenericGroup group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM, metrics);
- verify(metrics, times(0)).onGenericGroupStateTransition(any(), any());
+ ClassicGroup group = new ClassicGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM, metrics);
+ verify(metrics, times(0)).onClassicGroupStateTransition(any(), any());
group.transitionTo(PREPARING_REBALANCE);
- verify(metrics, times(1)).onGenericGroupStateTransition(EMPTY, PREPARING_REBALANCE);
+ verify(metrics, times(1)).onClassicGroupStateTransition(EMPTY, PREPARING_REBALANCE);
group.transitionTo(COMPLETING_REBALANCE);
- verify(metrics, times(1)).onGenericGroupStateTransition(PREPARING_REBALANCE, COMPLETING_REBALANCE);
+ verify(metrics, times(1)).onClassicGroupStateTransition(PREPARING_REBALANCE, COMPLETING_REBALANCE);
group.transitionTo(STABLE);
- verify(metrics, times(1)).onGenericGroupStateTransition(COMPLETING_REBALANCE, STABLE);
+ verify(metrics, times(1)).onClassicGroupStateTransition(COMPLETING_REBALANCE, STABLE);
group.transitionTo(DEAD);
- verify(metrics, times(1)).onGenericGroupStateTransition(STABLE, DEAD);
+ verify(metrics, times(1)).onClassicGroupStateTransition(STABLE, DEAD);
}
- private void assertState(GenericGroup group, GenericGroupState targetState) {
- Set<GenericGroupState> otherStates = new HashSet<>();
+ private void assertState(ClassicGroup group, ClassicGroupState targetState) {
+ Set<ClassicGroupState> otherStates = new HashSet<>();
otherStates.add(STABLE);
otherStates.add(PREPARING_REBALANCE);
otherStates.add(COMPLETING_REBALANCE);
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java
index 46bcf6b..f11ba4d 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsShardTest.java
@@ -24,19 +24,19 @@
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
-import org.apache.kafka.coordinator.group.generic.GenericGroup;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroup;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.stream.IntStream;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
-import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
+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.EMPTY;
+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.metrics.MetricsTestUtils.assertGaugeValue;
import static org.apache.kafka.coordinator.group.metrics.MetricsTestUtils.metricName;
import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -106,39 +106,39 @@
coordinatorMetrics.activateMetricsShard(shard);
LogContext logContext = new LogContext();
- GenericGroup group0 = new GenericGroup(logContext, "groupId0", EMPTY, Time.SYSTEM, shard);
- GenericGroup group1 = new GenericGroup(logContext, "groupId1", EMPTY, Time.SYSTEM, shard);
- GenericGroup group2 = new GenericGroup(logContext, "groupId2", EMPTY, Time.SYSTEM, shard);
- GenericGroup group3 = new GenericGroup(logContext, "groupId3", EMPTY, Time.SYSTEM, shard);
+ ClassicGroup group0 = new ClassicGroup(logContext, "groupId0", EMPTY, Time.SYSTEM, shard);
+ ClassicGroup group1 = new ClassicGroup(logContext, "groupId1", EMPTY, Time.SYSTEM, shard);
+ ClassicGroup group2 = new ClassicGroup(logContext, "groupId2", EMPTY, Time.SYSTEM, shard);
+ ClassicGroup group3 = new ClassicGroup(logContext, "groupId3", EMPTY, Time.SYSTEM, shard);
- IntStream.range(0, 4).forEach(__ -> shard.incrementNumGenericGroups(EMPTY));
+ IntStream.range(0, 4).forEach(__ -> shard.incrementNumClassicGroups(EMPTY));
- assertEquals(4, shard.numGenericGroups());
+ assertEquals(4, shard.numClassicGroups());
group0.transitionTo(PREPARING_REBALANCE);
group0.transitionTo(COMPLETING_REBALANCE);
group1.transitionTo(PREPARING_REBALANCE);
group2.transitionTo(DEAD);
- assertEquals(1, shard.numGenericGroups(GenericGroupState.EMPTY));
- assertEquals(1, shard.numGenericGroups(GenericGroupState.PREPARING_REBALANCE));
- assertEquals(1, shard.numGenericGroups(GenericGroupState.COMPLETING_REBALANCE));
- assertEquals(1, shard.numGenericGroups(GenericGroupState.DEAD));
- assertEquals(0, shard.numGenericGroups(GenericGroupState.STABLE));
+ assertEquals(1, shard.numClassicGroups(ClassicGroupState.EMPTY));
+ assertEquals(1, shard.numClassicGroups(ClassicGroupState.PREPARING_REBALANCE));
+ assertEquals(1, shard.numClassicGroups(ClassicGroupState.COMPLETING_REBALANCE));
+ assertEquals(1, shard.numClassicGroups(ClassicGroupState.DEAD));
+ assertEquals(0, shard.numClassicGroups(ClassicGroupState.STABLE));
group0.transitionTo(STABLE);
group1.transitionTo(COMPLETING_REBALANCE);
group3.transitionTo(DEAD);
- assertEquals(0, shard.numGenericGroups(GenericGroupState.EMPTY));
- assertEquals(0, shard.numGenericGroups(GenericGroupState.PREPARING_REBALANCE));
- assertEquals(1, shard.numGenericGroups(GenericGroupState.COMPLETING_REBALANCE));
- assertEquals(2, shard.numGenericGroups(GenericGroupState.DEAD));
- assertEquals(1, shard.numGenericGroups(GenericGroupState.STABLE));
+ assertEquals(0, shard.numClassicGroups(ClassicGroupState.EMPTY));
+ assertEquals(0, shard.numClassicGroups(ClassicGroupState.PREPARING_REBALANCE));
+ assertEquals(1, shard.numClassicGroups(ClassicGroupState.COMPLETING_REBALANCE));
+ assertEquals(2, shard.numClassicGroups(ClassicGroupState.DEAD));
+ assertEquals(1, shard.numClassicGroups(ClassicGroupState.STABLE));
assertGaugeValue(
metrics,
- metrics.metricName("group-count", "group-coordinator-metrics", Collections.singletonMap("protocol", "generic")),
+ metrics.metricName("group-count", "group-coordinator-metrics", Collections.singletonMap("protocol", "classic")),
4
);
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 4);
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java
index 8e203b5..8aa6521 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/metrics/GroupCoordinatorMetricsTest.java
@@ -24,7 +24,7 @@
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState;
-import org.apache.kafka.coordinator.group.generic.GenericGroupState;
+import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
@@ -33,9 +33,9 @@
import java.util.HashSet;
import java.util.stream.IntStream;
-import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.GENERIC_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.GENERIC_GROUP_REBALANCES_SENSOR_NAME;
+import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.CLASSIC_GROUP_REBALANCES_SENSOR_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.METRICS_GROUP;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.OFFSET_COMMITS_SENSOR_NAME;
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.OFFSET_EXPIRED_SENSOR_NAME;
@@ -69,7 +69,7 @@
metrics.metricName(
"group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
- Collections.singletonMap("protocol", "generic")),
+ Collections.singletonMap("protocol", "classic")),
metrics.metricName(
"group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
@@ -132,11 +132,11 @@
coordinatorMetrics.activateMetricsShard(shard0);
coordinatorMetrics.activateMetricsShard(shard1);
- IntStream.range(0, 5).forEach(__ -> shard0.incrementNumGenericGroups(GenericGroupState.PREPARING_REBALANCE));
- IntStream.range(0, 1).forEach(__ -> shard0.decrementNumGenericGroups(GenericGroupState.COMPLETING_REBALANCE));
- IntStream.range(0, 5).forEach(__ -> shard1.incrementNumGenericGroups(GenericGroupState.STABLE));
- IntStream.range(0, 4).forEach(__ -> shard1.incrementNumGenericGroups(GenericGroupState.DEAD));
- IntStream.range(0, 4).forEach(__ -> shard1.decrementNumGenericGroups(GenericGroupState.EMPTY));
+ IntStream.range(0, 5).forEach(__ -> shard0.incrementNumClassicGroups(ClassicGroupState.PREPARING_REBALANCE));
+ IntStream.range(0, 1).forEach(__ -> shard0.decrementNumClassicGroups(ClassicGroupState.COMPLETING_REBALANCE));
+ IntStream.range(0, 5).forEach(__ -> shard1.incrementNumClassicGroups(ClassicGroupState.STABLE));
+ IntStream.range(0, 4).forEach(__ -> shard1.incrementNumClassicGroups(ClassicGroupState.DEAD));
+ IntStream.range(0, 4).forEach(__ -> shard1.decrementNumClassicGroups(ClassicGroupState.EMPTY));
IntStream.range(0, 5).forEach(__ -> shard0.incrementNumConsumerGroups(ConsumerGroupState.ASSIGNING));
IntStream.range(0, 5).forEach(__ -> shard1.incrementNumConsumerGroups(ConsumerGroupState.RECONCILING));
@@ -146,12 +146,12 @@
IntStream.range(0, 2).forEach(__ -> shard1.incrementNumOffsets());
IntStream.range(0, 1).forEach(__ -> shard1.decrementNumOffsets());
- assertEquals(4, shard0.numGenericGroups());
- assertEquals(5, shard1.numGenericGroups());
+ assertEquals(4, shard0.numClassicGroups());
+ assertEquals(5, shard1.numClassicGroups());
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 9);
assertGaugeValue(
metrics,
- metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "generic")),
+ metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "classic")),
9
);
@@ -182,7 +182,7 @@
new SnapshotRegistry(new LogContext()), new TopicPartition("__consumer_offsets", 0)
);
- shard.record(GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME, 10);
+ shard.record(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME, 10);
assertMetricValue(metrics, metrics.metricName("group-completed-rebalance-rate", GroupCoordinatorMetrics.METRICS_GROUP), 1.0 / 3.0);
assertMetricValue(metrics, metrics.metricName("group-completed-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), 10);
@@ -194,7 +194,7 @@
assertMetricValue(metrics, metrics.metricName("offset-expiration-rate", GroupCoordinatorMetrics.METRICS_GROUP), 1.0);
assertMetricValue(metrics, metrics.metricName("offset-expiration-count", GroupCoordinatorMetrics.METRICS_GROUP), 30);
- shard.record(GENERIC_GROUP_REBALANCES_SENSOR_NAME, 40);
+ shard.record(CLASSIC_GROUP_REBALANCES_SENSOR_NAME, 40);
assertMetricValue(metrics, metrics.metricName("group-rebalance-rate", GroupCoordinatorMetrics.METRICS_GROUP), 4.0 / 3.0);
assertMetricValue(metrics, metrics.metricName("group-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), 40);