mirror of https://github.com/apache/kafka.git
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>
This commit is contained in:
parent
79757b3081
commit
98aca56ee5
|
@ -109,10 +109,10 @@ public class ConsumerConfig extends AbstractConfig {
|
||||||
* <code>group.protocol</code>
|
* <code>group.protocol</code>
|
||||||
*/
|
*/
|
||||||
public static final String GROUP_PROTOCOL_CONFIG = "group.protocol";
|
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 " +
|
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 " +
|
"support \"classic\" or \"consumer\". If \"consumer\" is specified, then the consumer group protocol will be " +
|
||||||
"used. Otherwise, the generic group protocol will be used.";
|
"used. Otherwise, the classic group protocol will be used.";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <code>group.remote.assignor</code>
|
* <code>group.remote.assignor</code>
|
||||||
|
|
|
@ -19,8 +19,8 @@ package org.apache.kafka.clients.consumer;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
|
|
||||||
public enum GroupProtocol {
|
public enum GroupProtocol {
|
||||||
/** Generic group protocol. */
|
/** Classic group protocol. */
|
||||||
GENERIC("GENERIC"),
|
CLASSIC("CLASSIC"),
|
||||||
|
|
||||||
/** Consumer group protocol */
|
/** Consumer group protocol */
|
||||||
CONSUMER("CONSUMER");
|
CONSUMER("CONSUMER");
|
||||||
|
|
|
@ -96,13 +96,13 @@ import static org.apache.kafka.common.utils.Utils.join;
|
||||||
import static org.apache.kafka.common.utils.Utils.swallow;
|
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.
|
* In this implementation, all network I/O happens in the thread of the application making the call.
|
||||||
*
|
*
|
||||||
* <p/>
|
* <p/>
|
||||||
*
|
*
|
||||||
* <em>Note:</em> per its name, this implementation is left for backward compatibility purposes. The updated consumer
|
* <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.
|
* 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> {
|
public class LegacyKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
|
||||||
|
|
|
@ -41,7 +41,7 @@
|
||||||
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
|
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
|
||||||
"about": "The unique group identifier." },
|
"about": "The unique group identifier." },
|
||||||
{ "name": "GenerationIdOrMemberEpoch", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true,
|
{ "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,
|
{ "name": "MemberId", "type": "string", "versions": "1+", "ignorable": true,
|
||||||
"about": "The member ID assigned by the group coordinator." },
|
"about": "The member ID assigned by the group coordinator." },
|
||||||
{ "name": "GroupInstanceId", "type": "string", "versions": "7+",
|
{ "name": "GroupInstanceId", "type": "string", "versions": "7+",
|
||||||
|
|
|
@ -173,7 +173,7 @@ public class ConsumerConfigTest {
|
||||||
configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
|
configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
|
||||||
configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass);
|
configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass);
|
||||||
final ConsumerConfig consumerConfig = new ConsumerConfig(configs);
|
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));
|
assertNull(consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -192,7 +192,7 @@ public class ConsumerConfigTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@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) {
|
public void testProtocolConfigValidation(String protocol, boolean isValid) {
|
||||||
final Map<String, Object> configs = new HashMap<>();
|
final Map<String, Object> configs = new HashMap<>();
|
||||||
configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
|
configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass);
|
||||||
|
|
|
@ -279,7 +279,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testPollReturnsRecords(GroupProtocol groupProtocol) {
|
public void testPollReturnsRecords(GroupProtocol groupProtocol) {
|
||||||
consumer = setUpConsumerWithRecordsToPoll(groupProtocol, tp0, 5);
|
consumer = setUpConsumerWithRecordsToPoll(groupProtocol, tp0, 5);
|
||||||
|
@ -294,7 +294,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testSecondPollWithDeserializationErrorThrowsRecordDeserializationException(GroupProtocol groupProtocol) {
|
public void testSecondPollWithDeserializationErrorThrowsRecordDeserializationException(GroupProtocol groupProtocol) {
|
||||||
int invalidRecordNumber = 4;
|
int invalidRecordNumber = 4;
|
||||||
|
@ -674,7 +674,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void verifyHeartbeatSent(GroupProtocol groupProtocol) throws Exception {
|
public void verifyHeartbeatSent(GroupProtocol groupProtocol) throws Exception {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -707,7 +707,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void verifyHeartbeatSentWhenFetchedDataReady(GroupProtocol groupProtocol) throws Exception {
|
public void verifyHeartbeatSentWhenFetchedDataReady(GroupProtocol groupProtocol) throws Exception {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -738,7 +738,7 @@ public class KafkaConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void verifyPollTimesOutDuringMetadataUpdate(GroupProtocol groupProtocol) {
|
public void verifyPollTimesOutDuringMetadataUpdate(GroupProtocol groupProtocol) {
|
||||||
final ConsumerMetadata metadata = createMetadata(subscription);
|
final ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
final MockClient client = new MockClient(time, metadata);
|
final MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -764,7 +764,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public void verifyDeprecatedPollDoesNotTimeOutDuringMetadataUpdate(GroupProtocol groupProtocol) {
|
public void verifyDeprecatedPollDoesNotTimeOutDuringMetadataUpdate(GroupProtocol groupProtocol) {
|
||||||
final ConsumerMetadata metadata = createMetadata(subscription);
|
final ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -812,7 +812,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void verifyNoCoordinatorLookupForManualAssignmentWithOffsetCommit(GroupProtocol groupProtocol) {
|
public void verifyNoCoordinatorLookupForManualAssignmentWithOffsetCommit(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -851,7 +851,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupProtocol) {
|
public void testFetchProgressWithMissingPartitionPosition(GroupProtocol groupProtocol) {
|
||||||
// Verifies that we can make progress on one partition while we are awaiting
|
// Verifies that we can make progress on one partition while we are awaiting
|
||||||
// a reset on another partition.
|
// a reset on another partition.
|
||||||
|
@ -911,7 +911,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) {
|
public void testMissingOffsetNoResetPolicy(GroupProtocol groupProtocol) {
|
||||||
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
|
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -935,7 +935,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testResetToCommittedOffset(GroupProtocol groupProtocol) {
|
public void testResetToCommittedOffset(GroupProtocol groupProtocol) {
|
||||||
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
|
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.NONE);
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -960,7 +960,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testResetUsingAutoResetPolicy(GroupProtocol groupProtocol) {
|
public void testResetUsingAutoResetPolicy(GroupProtocol groupProtocol) {
|
||||||
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.LATEST);
|
SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.LATEST);
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -1004,7 +1004,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCommitsFetchedDuringAssign(GroupProtocol groupProtocol) {
|
public void testCommitsFetchedDuringAssign(GroupProtocol groupProtocol) {
|
||||||
long offset1 = 10000;
|
long offset1 = 10000;
|
||||||
long offset2 = 20000;
|
long offset2 = 20000;
|
||||||
|
@ -1043,7 +1043,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testFetchStableOffsetThrowInCommitted(GroupProtocol groupProtocol) {
|
public void testFetchStableOffsetThrowInCommitted(GroupProtocol groupProtocol) {
|
||||||
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).committed(Collections.singleton(tp0)));
|
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).committed(Collections.singleton(tp0)));
|
||||||
}
|
}
|
||||||
|
@ -1051,7 +1051,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testFetchStableOffsetThrowInPoll(GroupProtocol groupProtocol) {
|
public void testFetchStableOffsetThrowInPoll(GroupProtocol groupProtocol) {
|
||||||
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).poll(Duration.ZERO));
|
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).poll(Duration.ZERO));
|
||||||
}
|
}
|
||||||
|
@ -1059,7 +1059,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testFetchStableOffsetThrowInPosition(GroupProtocol groupProtocol) {
|
public void testFetchStableOffsetThrowInPosition(GroupProtocol groupProtocol) {
|
||||||
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).position(tp0));
|
assertThrows(UnsupportedVersionException.class, () -> setupThrowableConsumer(groupProtocol).position(tp0));
|
||||||
}
|
}
|
||||||
|
@ -1090,7 +1090,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testNoCommittedOffsets(GroupProtocol groupProtocol) {
|
public void testNoCommittedOffsets(GroupProtocol groupProtocol) {
|
||||||
long offset1 = 10000;
|
long offset1 = 10000;
|
||||||
|
|
||||||
|
@ -1118,7 +1118,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testAutoCommitSentBeforePositionUpdate(GroupProtocol groupProtocol) {
|
public void testAutoCommitSentBeforePositionUpdate(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1152,7 +1152,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testRegexSubscription(GroupProtocol groupProtocol) {
|
public void testRegexSubscription(GroupProtocol groupProtocol) {
|
||||||
String unmatchedTopic = "unmatched";
|
String unmatchedTopic = "unmatched";
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -1181,7 +1181,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testChangingRegexSubscription(GroupProtocol groupProtocol) {
|
public void testChangingRegexSubscription(GroupProtocol groupProtocol) {
|
||||||
String otherTopic = "other";
|
String otherTopic = "other";
|
||||||
TopicPartition otherTopicPartition = new TopicPartition(otherTopic, 0);
|
TopicPartition otherTopicPartition = new TopicPartition(otherTopic, 0);
|
||||||
|
@ -1218,7 +1218,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testWakeupWithFetchDataAvailable(GroupProtocol groupProtocol) throws Exception {
|
public void testWakeupWithFetchDataAvailable(GroupProtocol groupProtocol) throws Exception {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1259,7 +1259,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testPollThrowsInterruptExceptionIfInterrupted(GroupProtocol groupProtocol) {
|
public void testPollThrowsInterruptExceptionIfInterrupted(GroupProtocol groupProtocol) {
|
||||||
final ConsumerMetadata metadata = createMetadata(subscription);
|
final ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
final MockClient client = new MockClient(time, metadata);
|
final MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1285,7 +1285,7 @@ public class KafkaConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void fetchResponseWithUnexpectedPartitionIsIgnored(GroupProtocol groupProtocol) {
|
public void fetchResponseWithUnexpectedPartitionIsIgnored(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1320,7 +1320,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testSubscriptionChangesWithAutoCommitEnabled(GroupProtocol groupProtocol) {
|
public void testSubscriptionChangesWithAutoCommitEnabled(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -1436,7 +1436,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testSubscriptionChangesWithAutoCommitDisabled(GroupProtocol groupProtocol) {
|
public void testSubscriptionChangesWithAutoCommitDisabled(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1493,7 +1493,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testUnsubscribeShouldTriggerPartitionsRevokedWithValidGeneration(GroupProtocol groupProtocol) {
|
public void testUnsubscribeShouldTriggerPartitionsRevokedWithValidGeneration(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1519,7 +1519,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testUnsubscribeShouldTriggerPartitionsLostWithNoGeneration(GroupProtocol groupProtocol) throws Exception {
|
public void testUnsubscribeShouldTriggerPartitionsLostWithNoGeneration(GroupProtocol groupProtocol) throws Exception {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1557,7 +1557,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupProtocol) {
|
public void testManualAssignmentChangeWithAutoCommitEnabled(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -1615,7 +1615,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol groupProtocol) {
|
public void testManualAssignmentChangeWithAutoCommitDisabled(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1673,7 +1673,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testOffsetOfPausedPartitions(GroupProtocol groupProtocol) {
|
public void testOffsetOfPausedPartitions(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -1749,7 +1749,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test references RPCs to be sent that are not part of the CONSUMER group protocol.
|
// 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.
|
// We are deferring any attempts at generalizing this test for both group protocols to the future.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testGracefulClose(GroupProtocol groupProtocol) throws Exception {
|
public void testGracefulClose(GroupProtocol groupProtocol) throws Exception {
|
||||||
Map<TopicPartition, Errors> response = new HashMap<>();
|
Map<TopicPartition, Errors> response = new HashMap<>();
|
||||||
response.put(tp0, Errors.NONE);
|
response.put(tp0, Errors.NONE);
|
||||||
|
@ -1762,7 +1762,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test references RPCs to be sent that are not part of the CONSUMER group protocol.
|
// 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.
|
// We are deferring any attempts at generalizing this test for both group protocols to the future.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCloseTimeoutDueToNoResponseForCloseFetchRequest(GroupProtocol groupProtocol) throws Exception {
|
public void testCloseTimeoutDueToNoResponseForCloseFetchRequest(GroupProtocol groupProtocol) throws Exception {
|
||||||
Map<TopicPartition, Errors> response = new HashMap<>();
|
Map<TopicPartition, Errors> response = new HashMap<>();
|
||||||
response.put(tp0, Errors.NONE);
|
response.put(tp0, Errors.NONE);
|
||||||
|
@ -1781,7 +1781,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCloseTimeout(GroupProtocol groupProtocol) throws Exception {
|
public void testCloseTimeout(GroupProtocol groupProtocol) throws Exception {
|
||||||
consumerCloseTest(groupProtocol, 5000, Collections.emptyList(), 5000, false);
|
consumerCloseTest(groupProtocol, 5000, Collections.emptyList(), 5000, false);
|
||||||
}
|
}
|
||||||
|
@ -1789,7 +1789,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testLeaveGroupTimeout(GroupProtocol groupProtocol) throws Exception {
|
public void testLeaveGroupTimeout(GroupProtocol groupProtocol) throws Exception {
|
||||||
Map<TopicPartition, Errors> response = new HashMap<>();
|
Map<TopicPartition, Errors> response = new HashMap<>();
|
||||||
response.put(tp0, Errors.NONE);
|
response.put(tp0, Errors.NONE);
|
||||||
|
@ -1800,7 +1800,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCloseNoWait(GroupProtocol groupProtocol) throws Exception {
|
public void testCloseNoWait(GroupProtocol groupProtocol) throws Exception {
|
||||||
consumerCloseTest(groupProtocol, 0, Collections.emptyList(), 0, false);
|
consumerCloseTest(groupProtocol, 0, Collections.emptyList(), 0, false);
|
||||||
}
|
}
|
||||||
|
@ -1808,7 +1808,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCloseInterrupt(GroupProtocol groupProtocol) throws Exception {
|
public void testCloseInterrupt(GroupProtocol groupProtocol) throws Exception {
|
||||||
consumerCloseTest(groupProtocol, Long.MAX_VALUE, Collections.emptyList(), 0, true);
|
consumerCloseTest(groupProtocol, Long.MAX_VALUE, Collections.emptyList(), 0, true);
|
||||||
}
|
}
|
||||||
|
@ -1816,7 +1816,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCloseShouldBeIdempotent(GroupProtocol groupProtocol) {
|
public void testCloseShouldBeIdempotent(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = spy(new MockClient(time, metadata));
|
MockClient client = spy(new MockClient(time, metadata));
|
||||||
|
@ -1918,7 +1918,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test references RPCs to be sent that are not part of the CONSUMER group protocol.
|
// 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.
|
// We are deferring any attempts at generalizing this test for both group protocols to the future.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testShouldAttemptToRejoinGroupAfterSyncGroupFailed(GroupProtocol groupProtocol) throws Exception {
|
public void testShouldAttemptToRejoinGroupAfterSyncGroupFailed(GroupProtocol groupProtocol) throws Exception {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -2085,7 +2085,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires topic metadata logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testPartitionsForNonExistingTopic(GroupProtocol groupProtocol) {
|
public void testPartitionsForNonExistingTopic(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -2106,7 +2106,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires topic metadata logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testPartitionsForAuthenticationFailure(GroupProtocol groupProtocol) {
|
public void testPartitionsForAuthenticationFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
||||||
assertThrows(AuthenticationException.class, () -> consumer.partitionsFor("some other topic"));
|
assertThrows(AuthenticationException.class, () -> consumer.partitionsFor("some other topic"));
|
||||||
|
@ -2115,7 +2115,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testBeginningOffsetsAuthenticationFailure(GroupProtocol groupProtocol) {
|
public void testBeginningOffsetsAuthenticationFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
||||||
assertThrows(AuthenticationException.class, () -> consumer.beginningOffsets(Collections.singleton(tp0)));
|
assertThrows(AuthenticationException.class, () -> consumer.beginningOffsets(Collections.singleton(tp0)));
|
||||||
|
@ -2124,7 +2124,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testEndOffsetsAuthenticationFailure(GroupProtocol groupProtocol) {
|
public void testEndOffsetsAuthenticationFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
||||||
assertThrows(AuthenticationException.class, () -> consumer.endOffsets(Collections.singleton(tp0)));
|
assertThrows(AuthenticationException.class, () -> consumer.endOffsets(Collections.singleton(tp0)));
|
||||||
|
@ -2133,7 +2133,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testPollAuthenticationFailure(GroupProtocol groupProtocol) {
|
public void testPollAuthenticationFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
||||||
consumer.subscribe(singleton(topic));
|
consumer.subscribe(singleton(topic));
|
||||||
|
@ -2143,7 +2143,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testOffsetsForTimesAuthenticationFailure(GroupProtocol groupProtocol) {
|
public void testOffsetsForTimesAuthenticationFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
||||||
assertThrows(AuthenticationException.class, () -> consumer.offsetsForTimes(singletonMap(tp0, 0L)));
|
assertThrows(AuthenticationException.class, () -> consumer.offsetsForTimes(singletonMap(tp0, 0L)));
|
||||||
|
@ -2152,7 +2152,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCommitSyncAuthenticationFailure(GroupProtocol groupProtocol) {
|
public void testCommitSyncAuthenticationFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
||||||
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
|
||||||
|
@ -2163,7 +2163,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testCommittedAuthenticationFailure(GroupProtocol groupProtocol) {
|
public void testCommittedAuthenticationFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerWithPendingAuthenticationError(groupProtocol);
|
||||||
assertThrows(AuthenticationException.class, () -> consumer.committed(Collections.singleton(tp0)).get(tp0));
|
assertThrows(AuthenticationException.class, () -> consumer.committed(Collections.singleton(tp0)).get(tp0));
|
||||||
|
@ -2172,7 +2172,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testMeasureCommitSyncDurationOnFailure(GroupProtocol groupProtocol) {
|
public void testMeasureCommitSyncDurationOnFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer
|
final KafkaConsumer<String, String> consumer
|
||||||
= consumerWithPendingError(groupProtocol, new MockTime(Duration.ofSeconds(1).toMillis()));
|
= consumerWithPendingError(groupProtocol, new MockTime(Duration.ofSeconds(1).toMillis()));
|
||||||
|
@ -2190,7 +2190,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testMeasureCommitSyncDuration(GroupProtocol groupProtocol) {
|
public void testMeasureCommitSyncDuration(GroupProtocol groupProtocol) {
|
||||||
Time time = new MockTime(Duration.ofSeconds(1).toMillis());
|
Time time = new MockTime(Duration.ofSeconds(1).toMillis());
|
||||||
SubscriptionState subscription = new SubscriptionState(new LogContext(),
|
SubscriptionState subscription = new SubscriptionState(new LogContext(),
|
||||||
|
@ -2221,7 +2221,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testMeasureCommittedDurationOnFailure(GroupProtocol groupProtocol) {
|
public void testMeasureCommittedDurationOnFailure(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer
|
final KafkaConsumer<String, String> consumer
|
||||||
= consumerWithPendingError(groupProtocol, new MockTime(Duration.ofSeconds(1).toMillis()));
|
= consumerWithPendingError(groupProtocol, new MockTime(Duration.ofSeconds(1).toMillis()));
|
||||||
|
@ -2239,7 +2239,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testMeasureCommittedDuration(GroupProtocol groupProtocol) {
|
public void testMeasureCommittedDuration(GroupProtocol groupProtocol) {
|
||||||
long offset1 = 10000;
|
long offset1 = 10000;
|
||||||
Time time = new MockTime(Duration.ofSeconds(1).toMillis());
|
Time time = new MockTime(Duration.ofSeconds(1).toMillis());
|
||||||
|
@ -2272,7 +2272,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testRebalanceException(GroupProtocol groupProtocol) {
|
public void testRebalanceException(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -2317,7 +2317,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testReturnRecordsDuringRebalance(GroupProtocol groupProtocol) throws InterruptedException {
|
public void testReturnRecordsDuringRebalance(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
Time time = new MockTime(1L);
|
Time time = new MockTime(1L);
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -2445,7 +2445,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testGetGroupMetadata(GroupProtocol groupProtocol) {
|
public void testGetGroupMetadata(GroupProtocol groupProtocol) {
|
||||||
final ConsumerMetadata metadata = createMetadata(subscription);
|
final ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
final MockClient client = new MockClient(time, metadata);
|
final MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -2478,7 +2478,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testInvalidGroupMetadata(GroupProtocol groupProtocol) throws InterruptedException {
|
public void testInvalidGroupMetadata(GroupProtocol groupProtocol) throws InterruptedException {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -2508,7 +2508,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testCurrentLag(GroupProtocol groupProtocol) {
|
public void testCurrentLag(GroupProtocol groupProtocol) {
|
||||||
final ConsumerMetadata metadata = createMetadata(subscription);
|
final ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -2564,7 +2564,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testListOffsetShouldUpdateSubscriptions(GroupProtocol groupProtocol) {
|
public void testListOffsetShouldUpdateSubscriptions(GroupProtocol groupProtocol) {
|
||||||
final ConsumerMetadata metadata = createMetadata(subscription);
|
final ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
final MockClient client = new MockClient(time, metadata);
|
final MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -2998,7 +2998,7 @@ public class KafkaConsumerTest {
|
||||||
// TODO: this test requires rebalance logic which is not yet implemented in the CONSUMER group protocol.
|
// 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.
|
// Once it is implemented, this should use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) {
|
public void testSubscriptionOnInvalidTopic(GroupProtocol groupProtocol) {
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
MockClient client = new MockClient(time, metadata);
|
MockClient client = new MockClient(time, metadata);
|
||||||
|
@ -3137,7 +3137,7 @@ public void testClosingConsumerUnregistersConsumerMetrics(GroupProtocol groupPro
|
||||||
|
|
||||||
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
|
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testEnforceRebalanceWithManualAssignment(GroupProtocol groupProtocol) {
|
public void testEnforceRebalanceWithManualAssignment(GroupProtocol groupProtocol) {
|
||||||
consumer = newConsumer(groupProtocol, null);
|
consumer = newConsumer(groupProtocol, null);
|
||||||
consumer.assign(singleton(new TopicPartition("topic", 0)));
|
consumer.assign(singleton(new TopicPartition("topic", 0)));
|
||||||
|
@ -3146,7 +3146,7 @@ public void testClosingConsumerUnregistersConsumerMetrics(GroupProtocol groupPro
|
||||||
|
|
||||||
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
|
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testEnforceRebalanceTriggersRebalanceOnNextPoll(GroupProtocol groupProtocol) {
|
public void testEnforceRebalanceTriggersRebalanceOnNextPoll(GroupProtocol groupProtocol) {
|
||||||
Time time = new MockTime(1L);
|
Time time = new MockTime(1L);
|
||||||
ConsumerMetadata metadata = createMetadata(subscription);
|
ConsumerMetadata metadata = createMetadata(subscription);
|
||||||
|
@ -3177,7 +3177,7 @@ public void testClosingConsumerUnregistersConsumerMetrics(GroupProtocol groupPro
|
||||||
|
|
||||||
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
|
// NOTE: this test uses the enforceRebalance API which is not implemented in the CONSUMER group protocol.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testEnforceRebalanceReason(GroupProtocol groupProtocol) {
|
public void testEnforceRebalanceReason(GroupProtocol groupProtocol) {
|
||||||
Time time = new MockTime(1L);
|
Time time = new MockTime(1L);
|
||||||
|
|
||||||
|
@ -3241,7 +3241,7 @@ public void testClosingConsumerUnregistersConsumerMetrics(GroupProtocol groupPro
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void configurableObjectsShouldSeeGeneratedClientId(GroupProtocol groupProtocol) {
|
public void configurableObjectsShouldSeeGeneratedClientId(GroupProtocol groupProtocol) {
|
||||||
Properties props = new Properties();
|
Properties props = new Properties();
|
||||||
props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name());
|
props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name());
|
||||||
|
@ -3288,7 +3288,7 @@ public void testClosingConsumerUnregistersConsumerMetrics(GroupProtocol groupPro
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testOffsetsForTimesTimeout(GroupProtocol groupProtocol) {
|
public void testOffsetsForTimesTimeout(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
|
||||||
assertEquals(
|
assertEquals(
|
||||||
|
@ -3300,7 +3300,7 @@ public void testClosingConsumerUnregistersConsumerMetrics(GroupProtocol groupPro
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testBeginningOffsetsTimeout(GroupProtocol groupProtocol) {
|
public void testBeginningOffsetsTimeout(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
|
||||||
assertEquals(
|
assertEquals(
|
||||||
|
@ -3312,7 +3312,7 @@ public void testClosingConsumerUnregistersConsumerMetrics(GroupProtocol groupPro
|
||||||
// TODO: this test triggers a bug with the CONSUMER group protocol implementation.
|
// 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.
|
// The bug will be investigated and fixed so this test can use both group protocols.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@EnumSource(value = GroupProtocol.class, names = "GENERIC")
|
@EnumSource(value = GroupProtocol.class, names = "CLASSIC")
|
||||||
public void testEndOffsetsTimeout(GroupProtocol groupProtocol) {
|
public void testEndOffsetsTimeout(GroupProtocol groupProtocol) {
|
||||||
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
|
final KafkaConsumer<String, String> consumer = consumerForCheckingTimeoutException(groupProtocol);
|
||||||
assertEquals(
|
assertEquals(
|
||||||
|
|
|
@ -1129,7 +1129,7 @@ public class AsyncKafkaConsumerTest {
|
||||||
public void testGroupRemoteAssignorUnusedInGenericProtocol() {
|
public void testGroupRemoteAssignorUnusedInGenericProtocol() {
|
||||||
final Properties props = requiredConsumerProperties();
|
final Properties props = requiredConsumerProperties();
|
||||||
props.put(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroupA");
|
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");
|
props.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "someAssignor");
|
||||||
final ConsumerConfig config = new ConsumerConfig(props);
|
final ConsumerConfig config = new ConsumerConfig(props);
|
||||||
consumer = newConsumer(config);
|
consumer = newConsumer(config);
|
||||||
|
|
|
@ -561,7 +561,7 @@ class BrokerServer(
|
||||||
config.offsetMetadataMaxSize,
|
config.offsetMetadataMaxSize,
|
||||||
config.groupMaxSize,
|
config.groupMaxSize,
|
||||||
config.groupInitialRebalanceDelay,
|
config.groupInitialRebalanceDelay,
|
||||||
GroupCoordinatorConfig.GENERIC_GROUP_NEW_MEMBER_JOIN_TIMEOUT_MS,
|
GroupCoordinatorConfig.CLASSIC_GROUP_NEW_MEMBER_JOIN_TIMEOUT_MS,
|
||||||
config.groupMinSessionTimeoutMs,
|
config.groupMinSessionTimeoutMs,
|
||||||
config.groupMaxSessionTimeoutMs,
|
config.groupMaxSessionTimeoutMs,
|
||||||
config.offsetsRetentionCheckIntervalMs,
|
config.offsetsRetentionCheckIntervalMs,
|
||||||
|
|
|
@ -167,7 +167,7 @@ object Defaults {
|
||||||
|
|
||||||
/** New group coordinator configs */
|
/** New group coordinator configs */
|
||||||
val NewGroupCoordinatorEnable = false
|
val NewGroupCoordinatorEnable = false
|
||||||
val GroupCoordinatorRebalanceProtocols = List(GroupType.GENERIC.toString).asJava
|
val GroupCoordinatorRebalanceProtocols = List(GroupType.CLASSIC.toString).asJava
|
||||||
val GroupCoordinatorNumThreads = 1
|
val GroupCoordinatorNumThreads = 1
|
||||||
|
|
||||||
/** Consumer group configs */
|
/** Consumer group configs */
|
||||||
|
@ -2013,8 +2013,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
|
||||||
val groupCoordinatorRebalanceProtocols = {
|
val groupCoordinatorRebalanceProtocols = {
|
||||||
val protocols = getList(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp)
|
val protocols = getList(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp)
|
||||||
.asScala.map(_.toUpperCase).map(GroupType.valueOf).toSet
|
.asScala.map(_.toUpperCase).map(GroupType.valueOf).toSet
|
||||||
if (!protocols.contains(GroupType.GENERIC)) {
|
if (!protocols.contains(GroupType.CLASSIC)) {
|
||||||
throw new ConfigException(s"Disabling the '${GroupType.GENERIC}' protocol is not supported.")
|
throw new ConfigException(s"Disabling the '${GroupType.CLASSIC}' protocol is not supported.")
|
||||||
}
|
}
|
||||||
if (protocols.contains(GroupType.CONSUMER)) {
|
if (protocols.contains(GroupType.CONSUMER)) {
|
||||||
warn(s"The new '${GroupType.CONSUMER}' rebalance protocol is enabled along with the new group coordinator. " +
|
warn(s"The new '${GroupType.CONSUMER}' rebalance protocol is enabled along with the new group coordinator. " +
|
||||||
|
|
|
@ -112,15 +112,15 @@ abstract class BaseConsumerTest extends AbstractConsumerTest {
|
||||||
|
|
||||||
object BaseConsumerTest {
|
object BaseConsumerTest {
|
||||||
// We want to test the following combinations:
|
// We want to test the following combinations:
|
||||||
// * ZooKeeper and the generic group protocol
|
// * ZooKeeper and the classic group protocol
|
||||||
// * KRaft and the generic group protocol
|
// * KRaft and the classic group protocol
|
||||||
// * KRaft with the new group coordinator enabled and the generic 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
|
// * KRaft with the new group coordinator enabled and the consumer group protocol
|
||||||
def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = {
|
def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = {
|
||||||
java.util.stream.Stream.of(
|
java.util.stream.Stream.of(
|
||||||
Arguments.of("zk", "generic"),
|
Arguments.of("zk", "classic"),
|
||||||
Arguments.of("kraft", "generic"),
|
Arguments.of("kraft", "classic"),
|
||||||
Arguments.of("kraft+kip848", "generic"),
|
Arguments.of("kraft+kip848", "classic"),
|
||||||
Arguments.of("kraft+kip848", "consumer"))
|
Arguments.of("kraft+kip848", "consumer"))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -130,18 +130,18 @@ object BaseConsumerTest {
|
||||||
// single combination are written using @CsvSource rather than the more elegant @MethodSource.
|
// single combination are written using @CsvSource rather than the more elegant @MethodSource.
|
||||||
// def getTestQuorumAndGroupProtocolParametersZkOnly() : java.util.stream.Stream[Arguments] = {
|
// def getTestQuorumAndGroupProtocolParametersZkOnly() : java.util.stream.Stream[Arguments] = {
|
||||||
// java.util.stream.Stream.of(
|
// 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:
|
// For tests that only work with the classic group protocol, we want to test the following combinations:
|
||||||
// * ZooKeeper and the generic group protocol
|
// * ZooKeeper and the classic group protocol
|
||||||
// * KRaft and the generic group protocol
|
// * KRaft and the classic group protocol
|
||||||
// * KRaft with the new group coordinator enabled and the generic group protocol
|
// * KRaft with the new group coordinator enabled and the classic group protocol
|
||||||
def getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly() : java.util.stream.Stream[Arguments] = {
|
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() : java.util.stream.Stream[Arguments] = {
|
||||||
java.util.stream.Stream.of(
|
java.util.stream.Stream.of(
|
||||||
Arguments.of("zk", "generic"),
|
Arguments.of("zk", "classic"),
|
||||||
Arguments.of("kraft", "generic"),
|
Arguments.of("kraft", "classic"),
|
||||||
Arguments.of("kraft+kip848", "generic"))
|
Arguments.of("kraft+kip848", "classic"))
|
||||||
}
|
}
|
||||||
|
|
||||||
val updateProducerCount = new AtomicInteger()
|
val updateProducerCount = new AtomicInteger()
|
||||||
|
|
|
@ -134,7 +134,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
// Deprecated poll(timeout) not supported for consumer group protocol
|
// Deprecated poll(timeout) not supported for consumer group protocol
|
||||||
@deprecated("poll(Duration) is the replacement", since = "2.0")
|
@deprecated("poll(Duration) is the replacement", since = "2.0")
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testDeprecatedPollBlocksForAssignment(quorum: String, groupProtocol: String): Unit = {
|
def testDeprecatedPollBlocksForAssignment(quorum: String, groupProtocol: String): Unit = {
|
||||||
val consumer = createConsumer()
|
val consumer = createConsumer()
|
||||||
consumer.subscribe(Set(topic).asJava)
|
consumer.subscribe(Set(topic).asJava)
|
||||||
|
@ -171,7 +171,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when KAFKA-16008 has been fixed.
|
// TODO: enable this test for the consumer group protocol when KAFKA-16008 has been fixed.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMaxPollIntervalMs(quorum: String, groupProtocol: String): Unit = {
|
def testMaxPollIntervalMs(quorum: String, groupProtocol: String): Unit = {
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 1000.toString)
|
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 1000.toString)
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString)
|
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString)
|
||||||
|
@ -198,7 +198,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when KAFKA-16009 has been fixed.
|
// TODO: enable this test for the consumer group protocol when KAFKA-16009 has been fixed.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMaxPollIntervalMsDelayInRevocation(quorum: String, groupProtocol: String): Unit = {
|
def testMaxPollIntervalMsDelayInRevocation(quorum: String, groupProtocol: String): Unit = {
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString)
|
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, 5000.toString)
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString)
|
this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 500.toString)
|
||||||
|
@ -290,7 +290,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when support for committing offsets on close is implemented.
|
// TODO: enable this test for the consumer group protocol when support for committing offsets on close is implemented.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testAutoCommitOnCloseAfterWakeup(quorum: String, groupProtocol: String): Unit = {
|
def testAutoCommitOnCloseAfterWakeup(quorum: String, groupProtocol: String): Unit = {
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
|
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
|
||||||
val consumer = createConsumer()
|
val consumer = createConsumer()
|
||||||
|
@ -352,7 +352,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
*/
|
*/
|
||||||
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
|
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testPatternSubscription(quorum: String, groupProtocol: String): Unit = {
|
def testPatternSubscription(quorum: String, groupProtocol: String): Unit = {
|
||||||
val numRecords = 10000
|
val numRecords = 10000
|
||||||
val producer = createProducer()
|
val producer = createProducer()
|
||||||
|
@ -411,7 +411,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
*/
|
*/
|
||||||
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
|
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testSubsequentPatternSubscription(quorum: String, groupProtocol: String): Unit = {
|
def testSubsequentPatternSubscription(quorum: String, groupProtocol: String): Unit = {
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "30000")
|
this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "30000")
|
||||||
val consumer = createConsumer()
|
val consumer = createConsumer()
|
||||||
|
@ -464,7 +464,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
*/
|
*/
|
||||||
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
|
// TODO: enable this test for the consumer group protocol when support for pattern subscriptions is implemented.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testPatternUnsubscription(quorum: String, groupProtocol: String): Unit = {
|
def testPatternUnsubscription(quorum: String, groupProtocol: String): Unit = {
|
||||||
val numRecords = 10000
|
val numRecords = 10000
|
||||||
val producer = createProducer()
|
val producer = createProducer()
|
||||||
|
@ -533,7 +533,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testExpandingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = {
|
def testExpandingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = {
|
||||||
val otherTopic = "other"
|
val otherTopic = "other"
|
||||||
val initialAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1))
|
val initialAssignment = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1))
|
||||||
|
@ -548,7 +548,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testShrinkingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = {
|
def testShrinkingTopicSubscriptions(quorum: String, groupProtocol: String): Unit = {
|
||||||
val otherTopic = "other"
|
val otherTopic = "other"
|
||||||
createTopic(otherTopic, 2, brokerCount)
|
createTopic(otherTopic, 2, brokerCount)
|
||||||
|
@ -975,9 +975,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
assertEquals(0, consumer.assignment().size)
|
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)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testRoundRobinAssignment(quorum: String, groupProtocol: String): Unit = {
|
def testRoundRobinAssignment(quorum: String, groupProtocol: String): Unit = {
|
||||||
// 1 consumer using round-robin assignment
|
// 1 consumer using round-robin assignment
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
|
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
|
||||||
|
@ -1013,9 +1013,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
assertEquals(0, consumer.assignment().size)
|
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)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMultiConsumerRoundRobinAssignor(quorum: String, groupProtocol: String): Unit = {
|
def testMultiConsumerRoundRobinAssignor(quorum: String, groupProtocol: String): Unit = {
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
|
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName)
|
this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName)
|
||||||
|
@ -1052,9 +1052,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
* - (#par / 10) partition per consumer, where one partition from each of the early (#par mod 9) consumers
|
* - (#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
|
* 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)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMultiConsumerStickyAssignor(quorum: String, groupProtocol: String): Unit = {
|
def testMultiConsumerStickyAssignor(quorum: String, groupProtocol: String): Unit = {
|
||||||
|
|
||||||
def reverse(m: Map[Long, Set[TopicPartition]]) =
|
def reverse(m: Map[Long, Set[TopicPartition]]) =
|
||||||
|
@ -1100,9 +1100,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
* This test re-uses BaseConsumerTest's consumers.
|
* This test re-uses BaseConsumerTest's consumers.
|
||||||
* As a result, it is testing the default assignment strategy set by BaseConsumerTest
|
* 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)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMultiConsumerDefaultAssignor(quorum: String, groupProtocol: String): Unit = {
|
def testMultiConsumerDefaultAssignor(quorum: String, groupProtocol: String): Unit = {
|
||||||
// use consumers and topics defined in this class + one more topic
|
// use consumers and topics defined in this class + one more topic
|
||||||
val producer = createProducer()
|
val producer = createProducer()
|
||||||
|
@ -1137,7 +1137,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Only the generic group protocol supports client-side assignors
|
// Only the classic group protocol supports client-side assignors
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@CsvSource(Array(
|
@CsvSource(Array(
|
||||||
"org.apache.kafka.clients.consumer.CooperativeStickyAssignor, zk",
|
"org.apache.kafka.clients.consumer.CooperativeStickyAssignor, zk",
|
||||||
|
@ -1147,7 +1147,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
))
|
))
|
||||||
def testRebalanceAndRejoin(assignmentStrategy: String, quorum: String): Unit = {
|
def testRebalanceAndRejoin(assignmentStrategy: String, quorum: String): Unit = {
|
||||||
// create 2 consumers
|
// 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.GROUP_ID_CONFIG, "rebalance-and-rejoin-group")
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, assignmentStrategy)
|
this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, assignmentStrategy)
|
||||||
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
|
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
|
||||||
|
@ -1230,9 +1230,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
* As a result, it is testing the default assignment strategy set by BaseConsumerTest
|
* 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)
|
* 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)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMultiConsumerDefaultAssignorAndVerifyAssignment(quorum: String, groupProtocol: String): Unit = {
|
def testMultiConsumerDefaultAssignorAndVerifyAssignment(quorum: String, groupProtocol: String): Unit = {
|
||||||
// create two new topics, each having 3 partitions
|
// create two new topics, each having 3 partitions
|
||||||
val topic1 = "topic1"
|
val topic1 = "topic1"
|
||||||
|
@ -1266,21 +1266,21 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when KAFKA-16010 has been fixed.
|
// TODO: enable this test for the consumer group protocol when KAFKA-16010 has been fixed.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMultiConsumerSessionTimeoutOnStopPolling(quorum: String, groupProtocol: String): Unit = {
|
def testMultiConsumerSessionTimeoutOnStopPolling(quorum: String, groupProtocol: String): Unit = {
|
||||||
runMultiConsumerSessionTimeoutTest(false)
|
runMultiConsumerSessionTimeoutTest(false)
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when KAFKA-16011 has been fixed.
|
// TODO: enable this test for the consumer group protocol when KAFKA-16011 has been fixed.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testMultiConsumerSessionTimeoutOnClose(quorum: String, groupProtocol: String): Unit = {
|
def testMultiConsumerSessionTimeoutOnClose(quorum: String, groupProtocol: String): Unit = {
|
||||||
runMultiConsumerSessionTimeoutTest(true)
|
runMultiConsumerSessionTimeoutTest(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
|
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testInterceptors(quorum: String, groupProtocol: String): Unit = {
|
def testInterceptors(quorum: String, groupProtocol: String): Unit = {
|
||||||
val appendStr = "mock"
|
val appendStr = "mock"
|
||||||
MockConsumerInterceptor.resetCounters()
|
MockConsumerInterceptor.resetCounters()
|
||||||
|
@ -1341,7 +1341,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
|
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testAutoCommitIntercept(quorum: String, groupProtocol: String): Unit = {
|
def testAutoCommitIntercept(quorum: String, groupProtocol: String): Unit = {
|
||||||
val topic2 = "topic2"
|
val topic2 = "topic2"
|
||||||
createTopic(topic2, 2, brokerCount)
|
createTopic(topic2, 2, brokerCount)
|
||||||
|
@ -1393,7 +1393,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
|
// TODO: enable this test for the consumer group protocol when consumer interceptors are supported
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testInterceptorsWithWrongKeyValue(quorum: String, groupProtocol: String): Unit = {
|
def testInterceptorsWithWrongKeyValue(quorum: String, groupProtocol: String): Unit = {
|
||||||
val appendStr = "mock"
|
val appendStr = "mock"
|
||||||
// create producer with interceptor that has different key and value types from the producer
|
// create producer with interceptor that has different key and value types from the producer
|
||||||
|
@ -1556,7 +1556,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when auto-commit support is implemented.
|
// TODO: enable this test for the consumer group protocol when auto-commit support is implemented.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testAutoCommitOnRebalance(quorum: String, groupProtocol: String): Unit = {
|
def testAutoCommitOnRebalance(quorum: String, groupProtocol: String): Unit = {
|
||||||
val topic2 = "topic2"
|
val topic2 = "topic2"
|
||||||
createTopic(topic2, 2, brokerCount)
|
createTopic(topic2, 2, brokerCount)
|
||||||
|
@ -1596,7 +1596,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testPerPartitionLeadMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = {
|
def testPerPartitionLeadMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = {
|
||||||
val numMessages = 1000
|
val numMessages = 1000
|
||||||
val topic2 = "topic2"
|
val topic2 = "topic2"
|
||||||
|
@ -1636,7 +1636,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testPerPartitionLagMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = {
|
def testPerPartitionLagMetricsCleanUpWithSubscribe(quorum: String, groupProtocol: String): Unit = {
|
||||||
val numMessages = 1000
|
val numMessages = 1000
|
||||||
val topic2 = "topic2"
|
val topic2 = "topic2"
|
||||||
|
@ -2034,9 +2034,9 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
assertThrows(classOf[InvalidGroupIdException], () => consumer1.commitSync())
|
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)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testConsumingWithEmptyGroupId(quorum: String, groupProtocol: String): Unit = {
|
def testConsumingWithEmptyGroupId(quorum: String, groupProtocol: String): Unit = {
|
||||||
val topic = "test_topic"
|
val topic = "test_topic"
|
||||||
val partition = 0
|
val partition = 0
|
||||||
|
@ -2097,7 +2097,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
|
|
||||||
// TODO: enable this test for the consumer group protocol when static membership is implemented.
|
// TODO: enable this test for the consumer group protocol when static membership is implemented.
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersGenericGroupProtocolOnly"))
|
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly"))
|
||||||
def testStaticConsumerDetectsNewPartitionCreatedAfterRestart(quorum:String, groupProtocol: String): Unit = {
|
def testStaticConsumerDetectsNewPartitionCreatedAfterRestart(quorum:String, groupProtocol: String): Unit = {
|
||||||
val foo = "foo"
|
val foo = "foo"
|
||||||
val foo0 = new TopicPartition(foo, 0)
|
val foo0 = new TopicPartition(foo, 0)
|
||||||
|
@ -2231,7 +2231,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
|
||||||
// partitionsFor not implemented in consumer group protocol and this test requires ZK also
|
// partitionsFor not implemented in consumer group protocol and this test requires ZK also
|
||||||
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
|
||||||
@CsvSource(Array(
|
@CsvSource(Array(
|
||||||
"zk, generic"
|
"zk, classic"
|
||||||
))
|
))
|
||||||
def testAssignAndConsumeWithLeaderChangeValidatingPositions(quorum:String, groupProtocol: String): Unit = {
|
def testAssignAndConsumeWithLeaderChangeValidatingPositions(quorum:String, groupProtocol: String): Unit = {
|
||||||
val numRecords = 10
|
val numRecords = 10
|
||||||
|
|
|
@ -61,8 +61,8 @@ object TestInfoUtils {
|
||||||
}
|
}
|
||||||
|
|
||||||
def maybeGroupProtocolSpecified(testInfo: TestInfo): Option[GroupProtocol] = {
|
def maybeGroupProtocolSpecified(testInfo: TestInfo): Option[GroupProtocol] = {
|
||||||
if (testInfo.getDisplayName().contains("groupProtocol=generic"))
|
if (testInfo.getDisplayName().contains("groupProtocol=classic"))
|
||||||
Some(GroupProtocol.GENERIC)
|
Some(GroupProtocol.CLASSIC)
|
||||||
else if (testInfo.getDisplayName().contains("groupProtocol=consumer"))
|
else if (testInfo.getDisplayName().contains("groupProtocol=consumer"))
|
||||||
Some(GroupProtocol.CONSUMER)
|
Some(GroupProtocol.CONSUMER)
|
||||||
else
|
else
|
||||||
|
|
|
@ -50,7 +50,7 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ClusterTest(clusterType = Type.KRAFT, serverProperties = Array(
|
@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.num.partitions", value = "1"),
|
||||||
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1")
|
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1")
|
||||||
))
|
))
|
||||||
|
@ -138,7 +138,7 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ClusterTest(clusterType = Type.KRAFT, serverProperties = Array(
|
@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.num.partitions", value = "1"),
|
||||||
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1")
|
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1")
|
||||||
))
|
))
|
||||||
|
@ -252,7 +252,7 @@ class ConsumerGroupHeartbeatRequestTest(cluster: ClusterInstance) {
|
||||||
}
|
}
|
||||||
|
|
||||||
@ClusterTest(clusterType = Type.KRAFT, serverProperties = Array(
|
@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.num.partitions", value = "1"),
|
||||||
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"),
|
new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"),
|
||||||
new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "5000"),
|
new ClusterConfigProperty(key = "group.consumer.session.timeout.ms", value = "5000"),
|
||||||
|
|
|
@ -21,7 +21,7 @@ import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDef
|
||||||
import kafka.test.junit.ClusterTestExtensions
|
import kafka.test.junit.ClusterTestExtensions
|
||||||
import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup
|
import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
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.Assertions.{assertEquals, fail}
|
||||||
import org.junit.jupiter.api.{Tag, Timeout}
|
import org.junit.jupiter.api.{Tag, Timeout}
|
||||||
import org.junit.jupiter.api.extension.ExtendWith
|
import org.junit.jupiter.api.extension.ExtendWith
|
||||||
|
@ -120,7 +120,7 @@ class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinator
|
||||||
assertEquals(
|
assertEquals(
|
||||||
List(new DescribedGroup()
|
List(new DescribedGroup()
|
||||||
.setGroupId("grp")
|
.setGroupId("grp")
|
||||||
.setGroupState(GenericGroupState.DEAD.toString)
|
.setGroupState(ClassicGroupState.DEAD.toString)
|
||||||
),
|
),
|
||||||
describeGroups(List("grp"))
|
describeGroups(List("grp"))
|
||||||
)
|
)
|
||||||
|
|
|
@ -21,7 +21,7 @@ import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDef
|
||||||
import kafka.test.junit.ClusterTestExtensions
|
import kafka.test.junit.ClusterTestExtensions
|
||||||
import org.apache.kafka.common.message.DescribeGroupsResponseData.{DescribedGroup, DescribedGroupMember}
|
import org.apache.kafka.common.message.DescribeGroupsResponseData.{DescribedGroup, DescribedGroupMember}
|
||||||
import org.apache.kafka.common.protocol.ApiKeys
|
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.Assertions.assertEquals
|
||||||
import org.junit.jupiter.api.{Tag, Timeout}
|
import org.junit.jupiter.api.{Tag, Timeout}
|
||||||
import org.junit.jupiter.api.extension.ExtendWith
|
import org.junit.jupiter.api.extension.ExtendWith
|
||||||
|
@ -80,7 +80,7 @@ class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinat
|
||||||
List(
|
List(
|
||||||
new DescribedGroup()
|
new DescribedGroup()
|
||||||
.setGroupId("grp-1")
|
.setGroupId("grp-1")
|
||||||
.setGroupState(GenericGroupState.STABLE.toString)
|
.setGroupState(ClassicGroupState.STABLE.toString)
|
||||||
.setProtocolType("consumer")
|
.setProtocolType("consumer")
|
||||||
.setProtocolData("consumer-range")
|
.setProtocolData("consumer-range")
|
||||||
.setMembers(List(
|
.setMembers(List(
|
||||||
|
@ -94,7 +94,7 @@ class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinat
|
||||||
).asJava),
|
).asJava),
|
||||||
new DescribedGroup()
|
new DescribedGroup()
|
||||||
.setGroupId("grp-2")
|
.setGroupId("grp-2")
|
||||||
.setGroupState(GenericGroupState.COMPLETING_REBALANCE.toString)
|
.setGroupState(ClassicGroupState.COMPLETING_REBALANCE.toString)
|
||||||
.setProtocolType("consumer")
|
.setProtocolType("consumer")
|
||||||
.setMembers(List(
|
.setMembers(List(
|
||||||
new DescribedGroupMember()
|
new DescribedGroupMember()
|
||||||
|
@ -107,7 +107,7 @@ class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinat
|
||||||
).asJava),
|
).asJava),
|
||||||
new DescribedGroup()
|
new DescribedGroup()
|
||||||
.setGroupId("grp-unknown")
|
.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(
|
describeGroups(
|
||||||
groupIds = List("grp-1", "grp-2", "grp-unknown"),
|
groupIds = List("grp-1", "grp-2", "grp-unknown"),
|
||||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
||||||
import org.apache.kafka.common.message.SyncGroupRequestData
|
import org.apache.kafka.common.message.SyncGroupRequestData
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
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.{Tag, Timeout}
|
||||||
import org.junit.jupiter.api.extension.ExtendWith
|
import org.junit.jupiter.api.extension.ExtendWith
|
||||||
|
|
||||||
|
@ -156,7 +156,7 @@ class HeartbeatRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
|
||||||
|
|
||||||
TestUtils.waitUntilTrue(() => {
|
TestUtils.waitUntilTrue(() => {
|
||||||
val described = describeGroups(groupIds = List("grp"))
|
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.")
|
}, msg = s"The group is not in PREPARING_REBALANCE state.")
|
||||||
|
|
||||||
// Heartbeat PREPARING_REBALANCE group.
|
// Heartbeat PREPARING_REBALANCE group.
|
||||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
||||||
import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember
|
import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember
|
||||||
import org.apache.kafka.common.message.{JoinGroupResponseData, SyncGroupRequestData}
|
import org.apache.kafka.common.message.{JoinGroupResponseData, SyncGroupRequestData}
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
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.Assertions.assertEquals
|
||||||
import org.junit.jupiter.api.{Tag, Timeout}
|
import org.junit.jupiter.api.{Tag, Timeout}
|
||||||
import org.junit.jupiter.api.extension.ExtendWith
|
import org.junit.jupiter.api.extension.ExtendWith
|
||||||
|
@ -190,7 +190,7 @@ class JoinGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
|
||||||
|
|
||||||
TestUtils.waitUntilTrue(() => {
|
TestUtils.waitUntilTrue(() => {
|
||||||
val described = describeGroups(groupIds = List("grp"))
|
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.")
|
}, msg = s"The group is not in PREPARING_REBALANCE state.")
|
||||||
|
|
||||||
// The leader rejoins.
|
// The leader rejoins.
|
||||||
|
@ -316,7 +316,7 @@ class JoinGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
|
||||||
|
|
||||||
TestUtils.waitUntilTrue(() => {
|
TestUtils.waitUntilTrue(() => {
|
||||||
val described = describeGroups(groupIds = List("grp"))
|
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.")
|
}, msg = s"The group is not in PREPARING_REBALANCE state.")
|
||||||
|
|
||||||
// A new follower with duplicated group instance id joins.
|
// A new follower with duplicated group instance id joins.
|
||||||
|
@ -329,7 +329,7 @@ class JoinGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
|
||||||
|
|
||||||
TestUtils.waitUntilTrue(() => {
|
TestUtils.waitUntilTrue(() => {
|
||||||
val described = describeGroups(groupIds = List("grp"))
|
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.")
|
}, msg = s"The group is not in COMPLETING_REBALANCE state.")
|
||||||
|
|
||||||
// The old follower rejoin request should be fenced.
|
// The old follower rejoin request should be fenced.
|
||||||
|
|
|
@ -1816,18 +1816,18 @@ class KafkaConfigTest {
|
||||||
val props = new Properties()
|
val props = new Properties()
|
||||||
props.putAll(kraftProps())
|
props.putAll(kraftProps())
|
||||||
|
|
||||||
// Only generic and consumer are supported.
|
// Only classic and consumer are supported.
|
||||||
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "foo")
|
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "foo")
|
||||||
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
|
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
|
||||||
|
|
||||||
// generic cannot be disabled.
|
// classic cannot be disabled.
|
||||||
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "consumer")
|
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "consumer")
|
||||||
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
|
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
|
||||||
|
|
||||||
// This is OK.
|
// This is OK.
|
||||||
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "generic,consumer")
|
props.put(KafkaConfig.GroupCoordinatorRebalanceProtocolsProp, "classic,consumer")
|
||||||
val config = KafkaConfig.fromProps(props)
|
val config = KafkaConfig.fromProps(props)
|
||||||
assertEquals(Set(GroupType.GENERIC, GroupType.CONSUMER), config.groupCoordinatorRebalanceProtocols)
|
assertEquals(Set(GroupType.CLASSIC, GroupType.CONSUMER), config.groupCoordinatorRebalanceProtocols)
|
||||||
assertTrue(config.isNewGroupCoordinatorEnabled)
|
assertTrue(config.isNewGroupCoordinatorEnabled)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,7 @@ import kafka.test.annotation.{ClusterConfigProperty, ClusterTest, ClusterTestDef
|
||||||
import kafka.test.junit.ClusterTestExtensions
|
import kafka.test.junit.ClusterTestExtensions
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||||
import org.apache.kafka.common.requests.JoinGroupRequest
|
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.Assertions.assertEquals
|
||||||
import org.junit.jupiter.api.{Tag, Timeout}
|
import org.junit.jupiter.api.{Tag, Timeout}
|
||||||
import org.junit.jupiter.api.extension.ExtendWith
|
import org.junit.jupiter.api.extension.ExtendWith
|
||||||
|
@ -115,7 +115,7 @@ class LeaveGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBa
|
||||||
|
|
||||||
// grp-1 is empty.
|
// grp-1 is empty.
|
||||||
assertEquals(
|
assertEquals(
|
||||||
GenericGroupState.EMPTY.toString,
|
ClassicGroupState.EMPTY.toString,
|
||||||
describeGroups(List("grp-1")).head.groupState
|
describeGroups(List("grp-1")).head.groupState
|
||||||
)
|
)
|
||||||
|
|
||||||
|
@ -142,7 +142,7 @@ class LeaveGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBa
|
||||||
|
|
||||||
// grp-2 is empty.
|
// grp-2 is empty.
|
||||||
assertEquals(
|
assertEquals(
|
||||||
GenericGroupState.EMPTY.toString,
|
ClassicGroupState.EMPTY.toString,
|
||||||
describeGroups(List("grp-2")).head.groupState
|
describeGroups(List("grp-2")).head.groupState
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,7 @@ import kafka.test.junit.ClusterTestExtensions
|
||||||
import org.apache.kafka.common.message.ListGroupsResponseData
|
import org.apache.kafka.common.message.ListGroupsResponseData
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState
|
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.Assertions.{assertEquals, fail}
|
||||||
import org.junit.jupiter.api.{Tag, Timeout}
|
import org.junit.jupiter.api.{Tag, Timeout}
|
||||||
import org.junit.jupiter.api.extension.ExtendWith
|
import org.junit.jupiter.api.extension.ExtendWith
|
||||||
|
@ -81,14 +81,14 @@ class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBa
|
||||||
val (memberId1InGroup1, _) = joinDynamicConsumerGroupWithOldProtocol(groupId = "grp-1")
|
val (memberId1InGroup1, _) = joinDynamicConsumerGroupWithOldProtocol(groupId = "grp-1")
|
||||||
val response1 = new ListGroupsResponseData.ListedGroup()
|
val response1 = new ListGroupsResponseData.ListedGroup()
|
||||||
.setGroupId("grp-1")
|
.setGroupId("grp-1")
|
||||||
.setGroupState(if (version >= 4) GenericGroupState.STABLE.toString else "")
|
.setGroupState(if (version >= 4) ClassicGroupState.STABLE.toString else "")
|
||||||
.setProtocolType("consumer")
|
.setProtocolType("consumer")
|
||||||
|
|
||||||
// Create grp-2 in old protocol without completing rebalance. Grp-2 is in COMPLETING_REBALANCE state.
|
// 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 (memberId1InGroup2, _) = joinDynamicConsumerGroupWithOldProtocol(groupId = "grp-2", completeRebalance = false)
|
||||||
val response2 = new ListGroupsResponseData.ListedGroup()
|
val response2 = new ListGroupsResponseData.ListedGroup()
|
||||||
.setGroupId("grp-2")
|
.setGroupId("grp-2")
|
||||||
.setGroupState(if (version >= 4) GenericGroupState.COMPLETING_REBALANCE.toString else "")
|
.setGroupState(if (version >= 4) ClassicGroupState.COMPLETING_REBALANCE.toString else "")
|
||||||
.setProtocolType("consumer")
|
.setProtocolType("consumer")
|
||||||
|
|
||||||
// Create grp-3 in old protocol and complete a rebalance. Then memeber 1 leaves grp-3. Grp-3 is in EMPTY state.
|
// 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 @@ class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBa
|
||||||
leaveGroup(groupId = "grp-3", memberId = memberId1InGroup3, useNewProtocol = false, ApiKeys.LEAVE_GROUP.latestVersion(isUnstableApiEnabled))
|
leaveGroup(groupId = "grp-3", memberId = memberId1InGroup3, useNewProtocol = false, ApiKeys.LEAVE_GROUP.latestVersion(isUnstableApiEnabled))
|
||||||
val response3 = new ListGroupsResponseData.ListedGroup()
|
val response3 = new ListGroupsResponseData.ListedGroup()
|
||||||
.setGroupId("grp-3")
|
.setGroupId("grp-3")
|
||||||
.setGroupState(if (version >= 4) GenericGroupState.EMPTY.toString else "")
|
.setGroupState(if (version >= 4) ClassicGroupState.EMPTY.toString else "")
|
||||||
.setProtocolType("consumer")
|
.setProtocolType("consumer")
|
||||||
|
|
||||||
var memberId1InGroup4: String = null
|
var memberId1InGroup4: String = null
|
||||||
|
@ -157,7 +157,7 @@ class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBa
|
||||||
if (useNewProtocol) List(response2, response5).toSet else List(response2).toSet,
|
if (useNewProtocol) List(response2, response5).toSet else List(response2).toSet,
|
||||||
listGroups(
|
listGroups(
|
||||||
statesFilter = List(
|
statesFilter = List(
|
||||||
GenericGroupState.COMPLETING_REBALANCE.toString,
|
ClassicGroupState.COMPLETING_REBALANCE.toString,
|
||||||
ConsumerGroupState.RECONCILING.toString,
|
ConsumerGroupState.RECONCILING.toString,
|
||||||
),
|
),
|
||||||
version = version.toShort
|
version = version.toShort
|
||||||
|
@ -168,8 +168,8 @@ class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBa
|
||||||
if (useNewProtocol) List(response1, response3, response6).toSet else List(response1, response3).toSet,
|
if (useNewProtocol) List(response1, response3, response6).toSet else List(response1, response3).toSet,
|
||||||
listGroups(
|
listGroups(
|
||||||
statesFilter = List(
|
statesFilter = List(
|
||||||
GenericGroupState.STABLE.toString,
|
ClassicGroupState.STABLE.toString,
|
||||||
GenericGroupState.EMPTY.toString,
|
ClassicGroupState.EMPTY.toString,
|
||||||
ConsumerGroupState.EMPTY.toString
|
ConsumerGroupState.EMPTY.toString
|
||||||
),
|
),
|
||||||
version = version.toShort
|
version = version.toShort
|
||||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
||||||
import org.apache.kafka.common.message.SyncGroupRequestData
|
import org.apache.kafka.common.message.SyncGroupRequestData
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
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.{Tag, Timeout}
|
||||||
import org.junit.jupiter.api.extension.ExtendWith
|
import org.junit.jupiter.api.extension.ExtendWith
|
||||||
|
|
||||||
|
@ -182,7 +182,7 @@ class SyncGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBas
|
||||||
|
|
||||||
TestUtils.waitUntilTrue(() => {
|
TestUtils.waitUntilTrue(() => {
|
||||||
val described = describeGroups(groupIds = List("grp"))
|
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.")
|
}, msg = s"The group is not in PREPARING_REBALANCE state.")
|
||||||
|
|
||||||
// The leader rejoins.
|
// The leader rejoins.
|
||||||
|
|
|
@ -28,7 +28,7 @@ import java.util.Optional;
|
||||||
public interface Group {
|
public interface Group {
|
||||||
enum GroupType {
|
enum GroupType {
|
||||||
CONSUMER("consumer"),
|
CONSUMER("consumer"),
|
||||||
GENERIC("generic");
|
CLASSIC("classic");
|
||||||
|
|
||||||
private final String name;
|
private final String name;
|
||||||
|
|
||||||
|
|
|
@ -73,7 +73,7 @@ public interface GroupCoordinator {
|
||||||
);
|
);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Join a Generic Group.
|
* Join a Classic Group.
|
||||||
*
|
*
|
||||||
* @param context The request context.
|
* @param context The request context.
|
||||||
* @param request The JoinGroupRequest data.
|
* @param request The JoinGroupRequest data.
|
||||||
|
@ -89,7 +89,7 @@ public interface GroupCoordinator {
|
||||||
);
|
);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sync a Generic Group.
|
* Sync a Classic Group.
|
||||||
*
|
*
|
||||||
* @param context The coordinator request context.
|
* @param context The coordinator request context.
|
||||||
* @param request The SyncGroupRequest data.
|
* @param request The SyncGroupRequest data.
|
||||||
|
@ -105,7 +105,7 @@ public interface GroupCoordinator {
|
||||||
);
|
);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Heartbeat to a Generic Group.
|
* Heartbeat to a Classic Group.
|
||||||
*
|
*
|
||||||
* @param context The coordinator request context.
|
* @param context The coordinator request context.
|
||||||
* @param request The HeartbeatRequest data.
|
* @param request The HeartbeatRequest data.
|
||||||
|
@ -119,7 +119,7 @@ public interface GroupCoordinator {
|
||||||
);
|
);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Leave a Generic Group.
|
* Leave a Classic Group.
|
||||||
*
|
*
|
||||||
* @param context The coordinator request context.
|
* @param context The coordinator request context.
|
||||||
* @param request The LeaveGroupRequest data.
|
* @param request The LeaveGroupRequest data.
|
||||||
|
|
|
@ -28,7 +28,7 @@ public class GroupCoordinatorConfig {
|
||||||
/**
|
/**
|
||||||
* The timeout used to wait for a new member in milliseconds.
|
* 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.
|
* The number of threads or event loops running.
|
||||||
|
@ -67,29 +67,29 @@ public class GroupCoordinatorConfig {
|
||||||
public final int offsetMetadataMaxSize;
|
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.
|
* 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.
|
* Frequency at which to check for expired offsets.
|
||||||
|
@ -126,11 +126,11 @@ public class GroupCoordinatorConfig {
|
||||||
List<PartitionAssignor> consumerGroupAssignors,
|
List<PartitionAssignor> consumerGroupAssignors,
|
||||||
int offsetsTopicSegmentBytes,
|
int offsetsTopicSegmentBytes,
|
||||||
int offsetMetadataMaxSize,
|
int offsetMetadataMaxSize,
|
||||||
int genericGroupMaxSize,
|
int classicGroupMaxSize,
|
||||||
int genericGroupInitialRebalanceDelayMs,
|
int classicGroupInitialRebalanceDelayMs,
|
||||||
int genericGroupNewMemberJoinTimeoutMs,
|
int classicGroupNewMemberJoinTimeoutMs,
|
||||||
int genericGroupMinSessionTimeoutMs,
|
int classicGroupMinSessionTimeoutMs,
|
||||||
int genericGroupMaxSessionTimeoutMs,
|
int classicGroupMaxSessionTimeoutMs,
|
||||||
long offsetsRetentionCheckIntervalMs,
|
long offsetsRetentionCheckIntervalMs,
|
||||||
long offsetsRetentionMs,
|
long offsetsRetentionMs,
|
||||||
int offsetCommitTimeoutMs
|
int offsetCommitTimeoutMs
|
||||||
|
@ -142,11 +142,11 @@ public class GroupCoordinatorConfig {
|
||||||
this.consumerGroupAssignors = consumerGroupAssignors;
|
this.consumerGroupAssignors = consumerGroupAssignors;
|
||||||
this.offsetsTopicSegmentBytes = offsetsTopicSegmentBytes;
|
this.offsetsTopicSegmentBytes = offsetsTopicSegmentBytes;
|
||||||
this.offsetMetadataMaxSize = offsetMetadataMaxSize;
|
this.offsetMetadataMaxSize = offsetMetadataMaxSize;
|
||||||
this.genericGroupMaxSize = genericGroupMaxSize;
|
this.classicGroupMaxSize = classicGroupMaxSize;
|
||||||
this.genericGroupInitialRebalanceDelayMs = genericGroupInitialRebalanceDelayMs;
|
this.classicGroupInitialRebalanceDelayMs = classicGroupInitialRebalanceDelayMs;
|
||||||
this.genericGroupNewMemberJoinTimeoutMs = genericGroupNewMemberJoinTimeoutMs;
|
this.classicGroupNewMemberJoinTimeoutMs = classicGroupNewMemberJoinTimeoutMs;
|
||||||
this.genericGroupMinSessionTimeoutMs = genericGroupMinSessionTimeoutMs;
|
this.classicGroupMinSessionTimeoutMs = classicGroupMinSessionTimeoutMs;
|
||||||
this.genericGroupMaxSessionTimeoutMs = genericGroupMaxSessionTimeoutMs;
|
this.classicGroupMaxSessionTimeoutMs = classicGroupMaxSessionTimeoutMs;
|
||||||
this.offsetsRetentionCheckIntervalMs = offsetsRetentionCheckIntervalMs;
|
this.offsetsRetentionCheckIntervalMs = offsetsRetentionCheckIntervalMs;
|
||||||
this.offsetsRetentionMs = offsetsRetentionMs;
|
this.offsetsRetentionMs = offsetsRetentionMs;
|
||||||
this.offsetCommitTimeoutMs = offsetCommitTimeoutMs;
|
this.offsetCommitTimeoutMs = offsetCommitTimeoutMs;
|
||||||
|
|
|
@ -355,10 +355,10 @@ public class GroupCoordinatorService implements GroupCoordinator {
|
||||||
CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
|
CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
|
||||||
|
|
||||||
runtime.scheduleWriteOperation(
|
runtime.scheduleWriteOperation(
|
||||||
"generic-group-join",
|
"classic-group-join",
|
||||||
topicPartitionFor(request.groupId()),
|
topicPartitionFor(request.groupId()),
|
||||||
Duration.ofMillis(config.offsetCommitTimeoutMs),
|
Duration.ofMillis(config.offsetCommitTimeoutMs),
|
||||||
coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
|
coordinator -> coordinator.classicGroupJoin(context, request, responseFuture)
|
||||||
).exceptionally(exception -> {
|
).exceptionally(exception -> {
|
||||||
if (!(exception instanceof KafkaException)) {
|
if (!(exception instanceof KafkaException)) {
|
||||||
log.error("JoinGroup request {} hit an unexpected exception: {}",
|
log.error("JoinGroup request {} hit an unexpected exception: {}",
|
||||||
|
@ -399,10 +399,10 @@ public class GroupCoordinatorService implements GroupCoordinator {
|
||||||
CompletableFuture<SyncGroupResponseData> responseFuture = new CompletableFuture<>();
|
CompletableFuture<SyncGroupResponseData> responseFuture = new CompletableFuture<>();
|
||||||
|
|
||||||
runtime.scheduleWriteOperation(
|
runtime.scheduleWriteOperation(
|
||||||
"generic-group-sync",
|
"classic-group-sync",
|
||||||
topicPartitionFor(request.groupId()),
|
topicPartitionFor(request.groupId()),
|
||||||
Duration.ofMillis(config.offsetCommitTimeoutMs),
|
Duration.ofMillis(config.offsetCommitTimeoutMs),
|
||||||
coordinator -> coordinator.genericGroupSync(context, request, responseFuture)
|
coordinator -> coordinator.classicGroupSync(context, request, responseFuture)
|
||||||
).exceptionally(exception -> {
|
).exceptionally(exception -> {
|
||||||
if (!(exception instanceof KafkaException)) {
|
if (!(exception instanceof KafkaException)) {
|
||||||
log.error("SyncGroup request {} hit an unexpected exception: {}",
|
log.error("SyncGroup request {} hit an unexpected exception: {}",
|
||||||
|
@ -441,9 +441,9 @@ public class GroupCoordinatorService implements GroupCoordinator {
|
||||||
|
|
||||||
// Using a read operation is okay here as we ignore the last committed offset in the snapshot registry.
|
// 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.
|
// 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()),
|
topicPartitionFor(request.groupId()),
|
||||||
(coordinator, __) -> coordinator.genericGroupHeartbeat(context, request)
|
(coordinator, __) -> coordinator.classicGroupHeartbeat(context, request)
|
||||||
).exceptionally(exception -> {
|
).exceptionally(exception -> {
|
||||||
if (!(exception instanceof KafkaException)) {
|
if (!(exception instanceof KafkaException)) {
|
||||||
log.error("Heartbeat request {} hit an unexpected exception: {}",
|
log.error("Heartbeat request {} hit an unexpected exception: {}",
|
||||||
|
@ -482,10 +482,10 @@ public class GroupCoordinatorService implements GroupCoordinator {
|
||||||
}
|
}
|
||||||
|
|
||||||
return runtime.scheduleWriteOperation(
|
return runtime.scheduleWriteOperation(
|
||||||
"generic-group-leave",
|
"classic-group-leave",
|
||||||
topicPartitionFor(request.groupId()),
|
topicPartitionFor(request.groupId()),
|
||||||
Duration.ofMillis(config.offsetCommitTimeoutMs),
|
Duration.ofMillis(config.offsetCommitTimeoutMs),
|
||||||
coordinator -> coordinator.genericGroupLeave(context, request)
|
coordinator -> coordinator.classicGroupLeave(context, request)
|
||||||
).exceptionally(exception -> {
|
).exceptionally(exception -> {
|
||||||
if (!(exception instanceof KafkaException)) {
|
if (!(exception instanceof KafkaException)) {
|
||||||
log.error("LeaveGroup request {} hit an unexpected exception: {}",
|
log.error("LeaveGroup request {} hit an unexpected exception: {}",
|
||||||
|
|
|
@ -83,7 +83,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The group coordinator shard is a replicated state machine that manages the metadata of all
|
* 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:
|
* has two kinds of methods:
|
||||||
* 1) The request handlers which handle the requests and generate a response and records to
|
* 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
|
* mutate the hard state. Those records will be written by the runtime and applied to the
|
||||||
|
@ -182,11 +182,11 @@ public class GroupCoordinatorShard implements CoordinatorShard<Record> {
|
||||||
.withConsumerGroupMaxSize(config.consumerGroupMaxSize)
|
.withConsumerGroupMaxSize(config.consumerGroupMaxSize)
|
||||||
.withConsumerGroupSessionTimeout(config.consumerGroupSessionTimeoutMs)
|
.withConsumerGroupSessionTimeout(config.consumerGroupSessionTimeoutMs)
|
||||||
.withConsumerGroupHeartbeatInterval(config.consumerGroupHeartbeatIntervalMs)
|
.withConsumerGroupHeartbeatInterval(config.consumerGroupHeartbeatIntervalMs)
|
||||||
.withGenericGroupMaxSize(config.genericGroupMaxSize)
|
.withClassicGroupMaxSize(config.classicGroupMaxSize)
|
||||||
.withGenericGroupInitialRebalanceDelayMs(config.genericGroupInitialRebalanceDelayMs)
|
.withClassicGroupInitialRebalanceDelayMs(config.classicGroupInitialRebalanceDelayMs)
|
||||||
.withGenericGroupNewMemberJoinTimeoutMs(config.genericGroupNewMemberJoinTimeoutMs)
|
.withClassicGroupNewMemberJoinTimeoutMs(config.classicGroupNewMemberJoinTimeoutMs)
|
||||||
.withGenericGroupMinSessionTimeoutMs(config.genericGroupMinSessionTimeoutMs)
|
.withClassicGroupMinSessionTimeoutMs(config.classicGroupMinSessionTimeoutMs)
|
||||||
.withGenericGroupMaxSessionTimeoutMs(config.genericGroupMaxSessionTimeoutMs)
|
.withClassicGroupMaxSessionTimeoutMs(config.classicGroupMaxSessionTimeoutMs)
|
||||||
.withGroupCoordinatorMetricsShard(metricsShard)
|
.withGroupCoordinatorMetricsShard(metricsShard)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -313,12 +313,12 @@ public class GroupCoordinatorShard implements CoordinatorShard<Record> {
|
||||||
* @return A Result containing the JoinGroup response and
|
* @return A Result containing the JoinGroup response and
|
||||||
* a list of records to update the state machine.
|
* a list of records to update the state machine.
|
||||||
*/
|
*/
|
||||||
public CoordinatorResult<Void, Record> genericGroupJoin(
|
public CoordinatorResult<Void, Record> classicGroupJoin(
|
||||||
RequestContext context,
|
RequestContext context,
|
||||||
JoinGroupRequestData request,
|
JoinGroupRequestData request,
|
||||||
CompletableFuture<JoinGroupResponseData> responseFuture
|
CompletableFuture<JoinGroupResponseData> responseFuture
|
||||||
) {
|
) {
|
||||||
return groupMetadataManager.genericGroupJoin(
|
return groupMetadataManager.classicGroupJoin(
|
||||||
context,
|
context,
|
||||||
request,
|
request,
|
||||||
responseFuture
|
responseFuture
|
||||||
|
@ -334,12 +334,12 @@ public class GroupCoordinatorShard implements CoordinatorShard<Record> {
|
||||||
* @return A Result containing the SyncGroup response and
|
* @return A Result containing the SyncGroup response and
|
||||||
* a list of records to update the state machine.
|
* a list of records to update the state machine.
|
||||||
*/
|
*/
|
||||||
public CoordinatorResult<Void, Record> genericGroupSync(
|
public CoordinatorResult<Void, Record> classicGroupSync(
|
||||||
RequestContext context,
|
RequestContext context,
|
||||||
SyncGroupRequestData request,
|
SyncGroupRequestData request,
|
||||||
CompletableFuture<SyncGroupResponseData> responseFuture
|
CompletableFuture<SyncGroupResponseData> responseFuture
|
||||||
) {
|
) {
|
||||||
return groupMetadataManager.genericGroupSync(
|
return groupMetadataManager.classicGroupSync(
|
||||||
context,
|
context,
|
||||||
request,
|
request,
|
||||||
responseFuture
|
responseFuture
|
||||||
|
@ -347,18 +347,18 @@ public class GroupCoordinatorShard implements CoordinatorShard<Record> {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Handles a generic group HeartbeatRequest.
|
* Handles a classic group HeartbeatRequest.
|
||||||
*
|
*
|
||||||
* @param context The request context.
|
* @param context The request context.
|
||||||
* @param request The actual Heartbeat request.
|
* @param request The actual Heartbeat request.
|
||||||
*
|
*
|
||||||
* @return The HeartbeatResponse.
|
* @return The HeartbeatResponse.
|
||||||
*/
|
*/
|
||||||
public HeartbeatResponseData genericGroupHeartbeat(
|
public HeartbeatResponseData classicGroupHeartbeat(
|
||||||
RequestContext context,
|
RequestContext context,
|
||||||
HeartbeatRequestData request
|
HeartbeatRequestData request
|
||||||
) {
|
) {
|
||||||
return groupMetadataManager.genericGroupHeartbeat(
|
return groupMetadataManager.classicGroupHeartbeat(
|
||||||
context,
|
context,
|
||||||
request
|
request
|
||||||
);
|
);
|
||||||
|
@ -529,11 +529,11 @@ public class GroupCoordinatorShard implements CoordinatorShard<Record> {
|
||||||
* @return A Result containing the LeaveGroup response and
|
* @return A Result containing the LeaveGroup response and
|
||||||
* a list of records to update the state machine.
|
* a list of records to update the state machine.
|
||||||
*/
|
*/
|
||||||
public CoordinatorResult<LeaveGroupResponseData, Record> genericGroupLeave(
|
public CoordinatorResult<LeaveGroupResponseData, Record> classicGroupLeave(
|
||||||
RequestContext context,
|
RequestContext context,
|
||||||
LeaveGroupRequestData request
|
LeaveGroupRequestData request
|
||||||
) throws ApiException {
|
) throws ApiException {
|
||||||
return groupMetadataManager.genericGroupLeave(context, request);
|
return groupMetadataManager.classicGroupLeave(context, request);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -18,7 +18,7 @@ package org.apache.kafka.coordinator.group;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An offset is considered expired based on different factors, such as the state of the group
|
* 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.
|
* how offsets for the group should be expired.
|
||||||
*/
|
*/
|
||||||
public interface OffsetExpirationCondition {
|
public interface OffsetExpirationCondition {
|
||||||
|
|
|
@ -33,7 +33,7 @@ public class OffsetExpirationConditionImpl implements OffsetExpirationCondition
|
||||||
/**
|
/**
|
||||||
* Determine whether an offset is expired. Older versions have an expire timestamp per partition. If this
|
* 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
|
* 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.
|
* exceeded the offset retention.
|
||||||
*
|
*
|
||||||
* @param offset The offset and metadata.
|
* @param offset The offset and metadata.
|
||||||
|
|
|
@ -41,8 +41,8 @@ import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroup;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroupState;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics;
|
||||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
||||||
|
@ -300,7 +300,7 @@ public class OffsetMetadataManager {
|
||||||
// either the admin client or a consumer which does not use the group management
|
// 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
|
// facility. In this case, a so-called simple group is created and the request
|
||||||
// is accepted.
|
// is accepted.
|
||||||
group = groupMetadataManager.getOrMaybeCreateGenericGroup(request.groupId(), true);
|
group = groupMetadataManager.getOrMaybeCreateClassicGroup(request.groupId(), true);
|
||||||
} else {
|
} else {
|
||||||
if (context.header.apiVersion() >= 9) {
|
if (context.header.apiVersion() >= 9) {
|
||||||
// Starting from version 9 of the OffsetCommit API, we return GROUP_ID_NOT_FOUND
|
// Starting from version 9 of the OffsetCommit API, we return GROUP_ID_NOT_FOUND
|
||||||
|
@ -354,7 +354,7 @@ public class OffsetMetadataManager {
|
||||||
// either the admin client or a consumer which does not use the group management
|
// 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
|
// facility. In this case, a so-called simple group is created and the request
|
||||||
// is accepted.
|
// is accepted.
|
||||||
group = groupMetadataManager.getOrMaybeCreateGenericGroup(request.groupId(), true);
|
group = groupMetadataManager.getOrMaybeCreateClassicGroup(request.groupId(), true);
|
||||||
} else {
|
} else {
|
||||||
throw Errors.ILLEGAL_GENERATION.exception();
|
throw Errors.ILLEGAL_GENERATION.exception();
|
||||||
}
|
}
|
||||||
|
@ -450,12 +450,12 @@ public class OffsetMetadataManager {
|
||||||
|
|
||||||
// In the old consumer group protocol, the offset commits maintain the session if
|
// In the old consumer group protocol, the offset commits maintain the session if
|
||||||
// the group is in Stable or PreparingRebalance state.
|
// the group is in Stable or PreparingRebalance state.
|
||||||
if (group.type() == Group.GroupType.GENERIC) {
|
if (group.type() == Group.GroupType.CLASSIC) {
|
||||||
GenericGroup genericGroup = (GenericGroup) group;
|
ClassicGroup classicGroup = (ClassicGroup) group;
|
||||||
if (genericGroup.isInState(GenericGroupState.STABLE) || genericGroup.isInState(GenericGroupState.PREPARING_REBALANCE)) {
|
if (classicGroup.isInState(ClassicGroupState.STABLE) || classicGroup.isInState(ClassicGroupState.PREPARING_REBALANCE)) {
|
||||||
groupMetadataManager.rescheduleGenericGroupMemberHeartbeat(
|
groupMetadataManager.rescheduleClassicGroupMemberHeartbeat(
|
||||||
genericGroup,
|
classicGroup,
|
||||||
genericGroup.member(request.memberId())
|
classicGroup.member(request.memberId())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -850,15 +850,15 @@ public class OffsetMetadataManager {
|
||||||
final int partition = key.partition();
|
final int partition = key.partition();
|
||||||
|
|
||||||
if (value != null) {
|
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
|
// replayed. However, it won't if the consumer commits offsets but does not
|
||||||
// use the membership functionality. In this case, we automatically create
|
// 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.
|
// without a protocol type.
|
||||||
try {
|
try {
|
||||||
groupMetadataManager.group(groupId);
|
groupMetadataManager.group(groupId);
|
||||||
} catch (GroupIdNotFoundException ex) {
|
} catch (GroupIdNotFoundException ex) {
|
||||||
groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, true);
|
groupMetadataManager.getOrMaybeCreateClassicGroup(groupId, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (producerId == RecordBatch.NO_PRODUCER_ID) {
|
if (producerId == RecordBatch.NO_PRODUCER_ID) {
|
||||||
|
|
|
@ -37,7 +37,7 @@ import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroup;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||||
import org.apache.kafka.server.common.MetadataVersion;
|
import org.apache.kafka.server.common.MetadataVersion;
|
||||||
|
|
||||||
|
@ -380,13 +380,13 @@ public class RecordHelpers {
|
||||||
/**
|
/**
|
||||||
* Creates a GroupMetadata record.
|
* Creates a GroupMetadata record.
|
||||||
*
|
*
|
||||||
* @param group The generic group.
|
* @param group The classic group.
|
||||||
* @param assignment The generic group assignment.
|
* @param assignment The classic group assignment.
|
||||||
* @param metadataVersion The metadata version.
|
* @param metadataVersion The metadata version.
|
||||||
* @return The record.
|
* @return The record.
|
||||||
*/
|
*/
|
||||||
public static Record newGroupMetadataRecord(
|
public static Record newGroupMetadataRecord(
|
||||||
GenericGroup group,
|
ClassicGroup group,
|
||||||
Map<String, byte[]> assignment,
|
Map<String, byte[]> assignment,
|
||||||
MetadataVersion metadataVersion
|
MetadataVersion metadataVersion
|
||||||
) {
|
) {
|
||||||
|
@ -457,12 +457,12 @@ public class RecordHelpers {
|
||||||
/**
|
/**
|
||||||
* Creates an empty GroupMetadata record.
|
* Creates an empty GroupMetadata record.
|
||||||
*
|
*
|
||||||
* @param group The generic group.
|
* @param group The classic group.
|
||||||
* @param metadataVersion The metadata version.
|
* @param metadataVersion The metadata version.
|
||||||
* @return The record.
|
* @return The record.
|
||||||
*/
|
*/
|
||||||
public static Record newEmptyGroupMetadataRecord(
|
public static Record newEmptyGroupMetadataRecord(
|
||||||
GenericGroup group,
|
ClassicGroup group,
|
||||||
MetadataVersion metadataVersion
|
MetadataVersion metadataVersion
|
||||||
) {
|
) {
|
||||||
return new Record(
|
return new Record(
|
||||||
|
|
|
@ -14,7 +14,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.coordinator.group.generic;
|
package org.apache.kafka.coordinator.group.classic;
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
|
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
|
||||||
import org.apache.kafka.common.errors.ApiException;
|
import org.apache.kafka.common.errors.ApiException;
|
||||||
|
@ -56,20 +56,20 @@ import java.util.UUID;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.EMPTY;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPARING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
|
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.
|
* member assignment is driven solely from the client side.
|
||||||
*
|
*
|
||||||
* The APIs members use to make changes to the group membership
|
* The APIs members use to make changes to the group membership
|
||||||
* consist of JoinGroup, SyncGroup, and LeaveGroup.
|
* consist of JoinGroup, SyncGroup, and LeaveGroup.
|
||||||
*/
|
*/
|
||||||
public class GenericGroup implements Group {
|
public class ClassicGroup implements Group {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Empty generation.
|
* Empty generation.
|
||||||
|
@ -110,12 +110,12 @@ public class GenericGroup implements Group {
|
||||||
/**
|
/**
|
||||||
* The current group state.
|
* The current group state.
|
||||||
*/
|
*/
|
||||||
private GenericGroupState state;
|
private ClassicGroupState state;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The previous group state.
|
* The previous group state.
|
||||||
*/
|
*/
|
||||||
private GenericGroupState previousState;
|
private ClassicGroupState previousState;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The timestamp of when the group transitioned
|
* The timestamp of when the group transitioned
|
||||||
|
@ -146,7 +146,7 @@ public class GenericGroup implements Group {
|
||||||
/**
|
/**
|
||||||
* The members of the group.
|
* 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.
|
* The static members of the group.
|
||||||
|
@ -191,10 +191,10 @@ public class GenericGroup implements Group {
|
||||||
*/
|
*/
|
||||||
private final GroupCoordinatorMetricsShard metrics;
|
private final GroupCoordinatorMetricsShard metrics;
|
||||||
|
|
||||||
public GenericGroup(
|
public ClassicGroup(
|
||||||
LogContext logContext,
|
LogContext logContext,
|
||||||
String groupId,
|
String groupId,
|
||||||
GenericGroupState initialState,
|
ClassicGroupState initialState,
|
||||||
Time time,
|
Time time,
|
||||||
GroupCoordinatorMetricsShard metrics
|
GroupCoordinatorMetricsShard metrics
|
||||||
) {
|
) {
|
||||||
|
@ -212,10 +212,10 @@ public class GenericGroup implements Group {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public GenericGroup(
|
public ClassicGroup(
|
||||||
LogContext logContext,
|
LogContext logContext,
|
||||||
String groupId,
|
String groupId,
|
||||||
GenericGroupState initialState,
|
ClassicGroupState initialState,
|
||||||
Time time,
|
Time time,
|
||||||
GroupCoordinatorMetricsShard metrics,
|
GroupCoordinatorMetricsShard metrics,
|
||||||
int generationId,
|
int generationId,
|
||||||
|
@ -225,7 +225,7 @@ public class GenericGroup implements Group {
|
||||||
Optional<Long> currentStateTimestamp
|
Optional<Long> currentStateTimestamp
|
||||||
) {
|
) {
|
||||||
Objects.requireNonNull(logContext);
|
Objects.requireNonNull(logContext);
|
||||||
this.log = logContext.logger(GenericGroup.class);
|
this.log = logContext.logger(ClassicGroup.class);
|
||||||
this.groupId = Objects.requireNonNull(groupId);
|
this.groupId = Objects.requireNonNull(groupId);
|
||||||
this.state = Objects.requireNonNull(initialState);
|
this.state = Objects.requireNonNull(initialState);
|
||||||
this.previousState = DEAD;
|
this.previousState = DEAD;
|
||||||
|
@ -241,11 +241,11 @@ public class GenericGroup implements Group {
|
||||||
/**
|
/**
|
||||||
* The type of this group.
|
* The type of this group.
|
||||||
*
|
*
|
||||||
* @return The group type (Generic).
|
* @return The group type (Classic).
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public GroupType type() {
|
public GroupType type() {
|
||||||
return GroupType.GENERIC;
|
return GroupType.CLASSIC;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -299,11 +299,11 @@ public class GenericGroup implements Group {
|
||||||
/**
|
/**
|
||||||
* @return the current group state.
|
* @return the current group state.
|
||||||
*/
|
*/
|
||||||
public GenericGroupState currentState() {
|
public ClassicGroupState currentState() {
|
||||||
return this.state;
|
return this.state;
|
||||||
}
|
}
|
||||||
|
|
||||||
public GenericGroupState previousState() {
|
public ClassicGroupState previousState() {
|
||||||
return this.previousState;
|
return this.previousState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -320,7 +320,7 @@ public class GenericGroup implements Group {
|
||||||
* @param groupState the state to match against.
|
* @param groupState the state to match against.
|
||||||
* @return true if the state matches, false otherwise.
|
* @return true if the state matches, false otherwise.
|
||||||
*/
|
*/
|
||||||
public boolean isInState(GenericGroupState groupState) {
|
public boolean isInState(ClassicGroupState groupState) {
|
||||||
return this.state == groupState;
|
return this.state == groupState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -340,7 +340,7 @@ public class GenericGroup implements Group {
|
||||||
* @param memberId the member id.
|
* @param memberId the member id.
|
||||||
* @return the member metadata if it exists, null otherwise.
|
* @return the member metadata if it exists, null otherwise.
|
||||||
*/
|
*/
|
||||||
public GenericGroupMember member(String memberId) {
|
public ClassicGroupMember member(String memberId) {
|
||||||
return members.get(memberId);
|
return members.get(memberId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -407,7 +407,7 @@ public class GenericGroup implements Group {
|
||||||
*
|
*
|
||||||
* @param member the member to add.
|
* @param member the member to add.
|
||||||
*/
|
*/
|
||||||
public void add(GenericGroupMember member) {
|
public void add(ClassicGroupMember member) {
|
||||||
add(member, null);
|
add(member, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -417,7 +417,7 @@ public class GenericGroup implements Group {
|
||||||
* @param member the member to add.
|
* @param member the member to add.
|
||||||
* @param future the future to complete once the join group phase completes.
|
* @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 -> {
|
member.groupInstanceId().ifPresent(instanceId -> {
|
||||||
if (staticMembers.containsKey(instanceId)) {
|
if (staticMembers.containsKey(instanceId)) {
|
||||||
throw new IllegalStateException("Static member with groupInstanceId=" +
|
throw new IllegalStateException("Static member with groupInstanceId=" +
|
||||||
|
@ -460,7 +460,7 @@ public class GenericGroup implements Group {
|
||||||
* @param memberId the member id to remove.
|
* @param memberId the member id to remove.
|
||||||
*/
|
*/
|
||||||
public void remove(String memberId) {
|
public void remove(String memberId) {
|
||||||
GenericGroupMember removedMember = members.remove(memberId);
|
ClassicGroupMember removedMember = members.remove(memberId);
|
||||||
if (removedMember != null) {
|
if (removedMember != null) {
|
||||||
decrementSupportedProtocols(removedMember);
|
decrementSupportedProtocols(removedMember);
|
||||||
if (removedMember.isAwaitingJoin()) {
|
if (removedMember.isAwaitingJoin()) {
|
||||||
|
@ -493,9 +493,9 @@ public class GenericGroup implements Group {
|
||||||
*/
|
*/
|
||||||
public boolean maybeElectNewJoinedLeader() {
|
public boolean maybeElectNewJoinedLeader() {
|
||||||
if (leaderId.isPresent()) {
|
if (leaderId.isPresent()) {
|
||||||
GenericGroupMember currentLeader = member(leaderId.get());
|
ClassicGroupMember currentLeader = member(leaderId.get());
|
||||||
if (!currentLeader.isAwaitingJoin()) {
|
if (!currentLeader.isAwaitingJoin()) {
|
||||||
for (GenericGroupMember member : members.values()) {
|
for (ClassicGroupMember member : members.values()) {
|
||||||
if (member.isAwaitingJoin()) {
|
if (member.isAwaitingJoin()) {
|
||||||
leaderId = Optional.of(member.memberId());
|
leaderId = Optional.of(member.memberId());
|
||||||
log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
|
log.info("Group leader [memberId: {}, groupInstanceId: {}] " +
|
||||||
|
@ -531,12 +531,12 @@ public class GenericGroup implements Group {
|
||||||
* @param newMemberId the new member id that will replace the old member id.
|
* @param newMemberId the new member id that will replace the old member id.
|
||||||
* @return the member with the new id.
|
* @return the member with the new id.
|
||||||
*/
|
*/
|
||||||
public GenericGroupMember replaceStaticMember(
|
public ClassicGroupMember replaceStaticMember(
|
||||||
String groupInstanceId,
|
String groupInstanceId,
|
||||||
String oldMemberId,
|
String oldMemberId,
|
||||||
String newMemberId
|
String newMemberId
|
||||||
) {
|
) {
|
||||||
GenericGroupMember removedMember = members.remove(oldMemberId);
|
ClassicGroupMember removedMember = members.remove(oldMemberId);
|
||||||
if (removedMember == null) {
|
if (removedMember == null) {
|
||||||
throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
|
throw new IllegalArgumentException("Cannot replace non-existing member id " + oldMemberId);
|
||||||
}
|
}
|
||||||
|
@ -560,7 +560,7 @@ public class GenericGroup implements Group {
|
||||||
.setErrorCode(Errors.FENCED_INSTANCE_ID.code());
|
.setErrorCode(Errors.FENCED_INSTANCE_ID.code());
|
||||||
completeSyncFuture(removedMember, syncGroupResponse);
|
completeSyncFuture(removedMember, syncGroupResponse);
|
||||||
|
|
||||||
GenericGroupMember newMember = new GenericGroupMember(
|
ClassicGroupMember newMember = new ClassicGroupMember(
|
||||||
newMemberId,
|
newMemberId,
|
||||||
removedMember.groupInstanceId(),
|
removedMember.groupInstanceId(),
|
||||||
removedMember.clientId(),
|
removedMember.clientId(),
|
||||||
|
@ -693,8 +693,8 @@ public class GenericGroup implements Group {
|
||||||
* @return members who have yet to rejoin during the
|
* @return members who have yet to rejoin during the
|
||||||
* join group phase.
|
* join group phase.
|
||||||
*/
|
*/
|
||||||
public Map<String, GenericGroupMember> notYetRejoinedMembers() {
|
public Map<String, ClassicGroupMember> notYetRejoinedMembers() {
|
||||||
Map<String, GenericGroupMember> notYetRejoinedMembers = new HashMap<>();
|
Map<String, ClassicGroupMember> notYetRejoinedMembers = new HashMap<>();
|
||||||
members.values().forEach(member -> {
|
members.values().forEach(member -> {
|
||||||
if (!member.isAwaitingJoin()) {
|
if (!member.isAwaitingJoin()) {
|
||||||
notYetRejoinedMembers.put(member.memberId(), member);
|
notYetRejoinedMembers.put(member.memberId(), member);
|
||||||
|
@ -741,7 +741,7 @@ public class GenericGroup implements Group {
|
||||||
/**
|
/**
|
||||||
* @return all members.
|
* @return all members.
|
||||||
*/
|
*/
|
||||||
public Collection<GenericGroupMember> allMembers() {
|
public Collection<ClassicGroupMember> allMembers() {
|
||||||
return members.values();
|
return members.values();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -751,7 +751,7 @@ public class GenericGroup implements Group {
|
||||||
*/
|
*/
|
||||||
public int rebalanceTimeoutMs() {
|
public int rebalanceTimeoutMs() {
|
||||||
int maxRebalanceTimeoutMs = 0;
|
int maxRebalanceTimeoutMs = 0;
|
||||||
for (GenericGroupMember member : members.values()) {
|
for (ClassicGroupMember member : members.values()) {
|
||||||
maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
|
maxRebalanceTimeoutMs = Math.max(maxRebalanceTimeoutMs, member.rebalanceTimeoutMs());
|
||||||
}
|
}
|
||||||
return maxRebalanceTimeoutMs;
|
return maxRebalanceTimeoutMs;
|
||||||
|
@ -856,8 +856,8 @@ public class GenericGroup implements Group {
|
||||||
/**
|
/**
|
||||||
* Validates the OffsetFetch request.
|
* Validates the OffsetFetch request.
|
||||||
*
|
*
|
||||||
* @param memberId The member id. 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 generic 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.
|
* @param lastCommittedOffset The last committed offsets in the timeline.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
|
@ -984,12 +984,12 @@ public class GenericGroup implements Group {
|
||||||
* Transition to a group state.
|
* Transition to a group state.
|
||||||
* @param groupState the group state.
|
* @param groupState the group state.
|
||||||
*/
|
*/
|
||||||
public void transitionTo(GenericGroupState groupState) {
|
public void transitionTo(ClassicGroupState groupState) {
|
||||||
assertValidTransition(groupState);
|
assertValidTransition(groupState);
|
||||||
previousState = state;
|
previousState = state;
|
||||||
state = groupState;
|
state = groupState;
|
||||||
currentStateTimestamp = Optional.of(time.milliseconds());
|
currentStateTimestamp = Optional.of(time.milliseconds());
|
||||||
metrics.onGenericGroupStateTransition(previousState, state);
|
metrics.onClassicGroupStateTransition(previousState, state);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1028,7 +1028,7 @@ public class GenericGroup implements Group {
|
||||||
*
|
*
|
||||||
* @param member the member.
|
* @param member the member.
|
||||||
*/
|
*/
|
||||||
private void incrementSupportedProtocols(GenericGroupMember member) {
|
private void incrementSupportedProtocols(ClassicGroupMember member) {
|
||||||
member.supportedProtocols().forEach(protocol -> {
|
member.supportedProtocols().forEach(protocol -> {
|
||||||
int count = supportedProtocols.getOrDefault(protocol.name(), 0);
|
int count = supportedProtocols.getOrDefault(protocol.name(), 0);
|
||||||
supportedProtocols.put(protocol.name(), count + 1);
|
supportedProtocols.put(protocol.name(), count + 1);
|
||||||
|
@ -1041,7 +1041,7 @@ public class GenericGroup implements Group {
|
||||||
*
|
*
|
||||||
* @param member the member.
|
* @param member the member.
|
||||||
*/
|
*/
|
||||||
private void decrementSupportedProtocols(GenericGroupMember member) {
|
private void decrementSupportedProtocols(ClassicGroupMember member) {
|
||||||
member.supportedProtocols().forEach(protocol -> {
|
member.supportedProtocols().forEach(protocol -> {
|
||||||
int count = supportedProtocols.getOrDefault(protocol.name(), 0);
|
int count = supportedProtocols.getOrDefault(protocol.name(), 0);
|
||||||
supportedProtocols.put(protocol.name(), count - 1);
|
supportedProtocols.put(protocol.name(), count - 1);
|
||||||
|
@ -1069,10 +1069,10 @@ public class GenericGroup implements Group {
|
||||||
*
|
*
|
||||||
* @return a boolean based on the condition mentioned above.
|
* @return a boolean based on the condition mentioned above.
|
||||||
*/
|
*/
|
||||||
public boolean supportsProtocols(GenericGroupMember member) {
|
public boolean supportsProtocols(ClassicGroupMember member) {
|
||||||
return supportsProtocols(
|
return supportsProtocols(
|
||||||
member.protocolType(),
|
member.protocolType(),
|
||||||
GenericGroupMember.plainProtocolSet(member.supportedProtocols())
|
ClassicGroupMember.plainProtocolSet(member.supportedProtocols())
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1091,7 +1091,7 @@ public class GenericGroup implements Group {
|
||||||
) {
|
) {
|
||||||
return supportsProtocols(
|
return supportsProtocols(
|
||||||
memberProtocolType,
|
memberProtocolType,
|
||||||
GenericGroupMember.plainProtocolSet(memberProtocols)
|
ClassicGroupMember.plainProtocolSet(memberProtocols)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1122,7 +1122,7 @@ public class GenericGroup implements Group {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* 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
|
* 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
|
* consider the group not subscribed to the topic if the group is not using any protocol or not using the
|
||||||
* consumer group protocol.
|
* consumer group protocol.
|
||||||
|
@ -1191,7 +1191,7 @@ public class GenericGroup implements Group {
|
||||||
* @param future the future that is invoked once the join phase is complete.
|
* @param future the future that is invoked once the join phase is complete.
|
||||||
*/
|
*/
|
||||||
public void updateMember(
|
public void updateMember(
|
||||||
GenericGroupMember member,
|
ClassicGroupMember member,
|
||||||
JoinGroupRequestProtocolCollection protocols,
|
JoinGroupRequestProtocolCollection protocols,
|
||||||
int rebalanceTimeoutMs,
|
int rebalanceTimeoutMs,
|
||||||
int sessionTimeoutMs,
|
int sessionTimeoutMs,
|
||||||
|
@ -1219,7 +1219,7 @@ public class GenericGroup implements Group {
|
||||||
* @return true if a join future actually completes.
|
* @return true if a join future actually completes.
|
||||||
*/
|
*/
|
||||||
public boolean completeJoinFuture(
|
public boolean completeJoinFuture(
|
||||||
GenericGroupMember member,
|
ClassicGroupMember member,
|
||||||
JoinGroupResponseData response
|
JoinGroupResponseData response
|
||||||
) {
|
) {
|
||||||
if (member.isAwaitingJoin()) {
|
if (member.isAwaitingJoin()) {
|
||||||
|
@ -1239,7 +1239,7 @@ public class GenericGroup implements Group {
|
||||||
* @return true if a sync future actually completes.
|
* @return true if a sync future actually completes.
|
||||||
*/
|
*/
|
||||||
public boolean completeSyncFuture(
|
public boolean completeSyncFuture(
|
||||||
GenericGroupMember member,
|
ClassicGroupMember member,
|
||||||
SyncGroupResponseData response
|
SyncGroupResponseData response
|
||||||
) {
|
) {
|
||||||
if (member.isAwaitingSync()) {
|
if (member.isAwaitingSync()) {
|
||||||
|
@ -1272,9 +1272,9 @@ public class GenericGroup implements Group {
|
||||||
*
|
*
|
||||||
* @return the members.
|
* @return the members.
|
||||||
*/
|
*/
|
||||||
public List<JoinGroupResponseMember> currentGenericGroupMembers() {
|
public List<JoinGroupResponseMember> currentClassicGroupMembers() {
|
||||||
if (isInState(DEAD) || isInState(PREPARING_REBALANCE)) {
|
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);
|
groupId + " in state " + state);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1301,7 +1301,7 @@ public class GenericGroup implements Group {
|
||||||
*/
|
*/
|
||||||
public Map<String, byte[]> groupAssignment() {
|
public Map<String, byte[]> groupAssignment() {
|
||||||
return allMembers().stream().collect(Collectors.toMap(
|
return allMembers().stream().collect(Collectors.toMap(
|
||||||
GenericGroupMember::memberId, GenericGroupMember::assignment
|
ClassicGroupMember::memberId, ClassicGroupMember::assignment
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1310,7 +1310,7 @@ public class GenericGroup implements Group {
|
||||||
*
|
*
|
||||||
* @param targetState the target state to transition to.
|
* @param targetState the target state to transition to.
|
||||||
*/
|
*/
|
||||||
private void assertValidTransition(GenericGroupState targetState) {
|
private void assertValidTransition(ClassicGroupState targetState) {
|
||||||
if (!targetState.validPreviousStates().contains(state)) {
|
if (!targetState.validPreviousStates().contains(state)) {
|
||||||
throw new IllegalStateException("Group " + groupId + " should be in one of " +
|
throw new IllegalStateException("Group " + groupId + " should be in one of " +
|
||||||
targetState.validPreviousStates() + " states before moving to " + targetState +
|
targetState.validPreviousStates() + " states before moving to " + targetState +
|
||||||
|
@ -1320,7 +1320,7 @@ public class GenericGroup implements Group {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "GenericGroupMetadata(" +
|
return "ClassicGroupMetadata(" +
|
||||||
"groupId=" + groupId + ", " +
|
"groupId=" + groupId + ", " +
|
||||||
"generation=" + generationId + ", " +
|
"generation=" + generationId + ", " +
|
||||||
"protocolType=" + protocolType + ", " +
|
"protocolType=" + protocolType + ", " +
|
|
@ -15,7 +15,7 @@
|
||||||
* limitations under the License.
|
* 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.DescribeGroupsResponseData;
|
||||||
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol;
|
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol;
|
||||||
|
@ -31,7 +31,7 @@ import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.stream.Collectors;
|
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:
|
* Member metadata contains the following:
|
||||||
*
|
*
|
||||||
|
@ -52,7 +52,7 @@ import java.util.stream.Collectors;
|
||||||
* is kept in metadata until the leader provides the group assignment
|
* is kept in metadata until the leader provides the group assignment
|
||||||
* and the group transitions to stable
|
* and the group transitions to stable
|
||||||
*/
|
*/
|
||||||
public class GenericGroupMember {
|
public class ClassicGroupMember {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An empty assignment.
|
* An empty assignment.
|
||||||
|
@ -119,7 +119,7 @@ public class GenericGroupMember {
|
||||||
*/
|
*/
|
||||||
private boolean isNew = false;
|
private boolean isNew = false;
|
||||||
|
|
||||||
public GenericGroupMember(
|
public ClassicGroupMember(
|
||||||
String memberId,
|
String memberId,
|
||||||
Optional<String> groupInstanceId,
|
Optional<String> groupInstanceId,
|
||||||
String clientId,
|
String clientId,
|
||||||
|
@ -142,7 +142,7 @@ public class GenericGroupMember {
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public GenericGroupMember(
|
public ClassicGroupMember(
|
||||||
String memberId,
|
String memberId,
|
||||||
Optional<String> groupInstanceId,
|
Optional<String> groupInstanceId,
|
||||||
String clientId,
|
String clientId,
|
||||||
|
@ -420,7 +420,7 @@ public class GenericGroupMember {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "GenericGroupMember(" +
|
return "ClassicGroupMember(" +
|
||||||
"memberId='" + memberId + '\'' +
|
"memberId='" + memberId + '\'' +
|
||||||
", groupInstanceId='" + groupInstanceId.orElse("") + '\'' +
|
", groupInstanceId='" + groupInstanceId.orElse("") + '\'' +
|
||||||
", clientId='" + clientId + '\'' +
|
", clientId='" + clientId + '\'' +
|
|
@ -15,17 +15,17 @@
|
||||||
* limitations under the License.
|
* 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.Arrays;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
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.
|
* 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
|
* Group has no more members, but lingers until all offsets have expired. This state
|
||||||
|
@ -104,7 +104,7 @@ public enum GenericGroupState {
|
||||||
DEAD("Dead");
|
DEAD("Dead");
|
||||||
|
|
||||||
private final String name;
|
private final String name;
|
||||||
private Set<GenericGroupState> validPreviousStates;
|
private Set<ClassicGroupState> validPreviousStates;
|
||||||
|
|
||||||
static {
|
static {
|
||||||
EMPTY.addValidPreviousStates(PREPARING_REBALANCE);
|
EMPTY.addValidPreviousStates(PREPARING_REBALANCE);
|
||||||
|
@ -114,7 +114,7 @@ public enum GenericGroupState {
|
||||||
DEAD.addValidPreviousStates(STABLE, PREPARING_REBALANCE, COMPLETING_REBALANCE, EMPTY, DEAD);
|
DEAD.addValidPreviousStates(STABLE, PREPARING_REBALANCE, COMPLETING_REBALANCE, EMPTY, DEAD);
|
||||||
}
|
}
|
||||||
|
|
||||||
GenericGroupState(String name) {
|
ClassicGroupState(String name) {
|
||||||
this.name = name;
|
this.name = name;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -123,14 +123,14 @@ public enum GenericGroupState {
|
||||||
return name;
|
return name;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void addValidPreviousStates(GenericGroupState... validPreviousStates) {
|
private void addValidPreviousStates(ClassicGroupState... validPreviousStates) {
|
||||||
this.validPreviousStates = new HashSet<>(Arrays.asList(validPreviousStates));
|
this.validPreviousStates = new HashSet<>(Arrays.asList(validPreviousStates));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return valid previous states a group must be in to transition to this state.
|
* @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;
|
return this.validPreviousStates;
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -26,7 +26,7 @@ import org.apache.kafka.common.metrics.stats.Meter;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
import org.apache.kafka.coordinator.group.Group;
|
import org.apache.kafka.coordinator.group.Group;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState;
|
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.server.metrics.KafkaYammerMetrics;
|
||||||
import org.apache.kafka.timeline.SnapshotRegistry;
|
import org.apache.kafka.timeline.SnapshotRegistry;
|
||||||
|
|
||||||
|
@ -45,21 +45,21 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
public static final String METRICS_GROUP = "group-coordinator-metrics";
|
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");
|
"GroupMetadataManager", "NumGroups");
|
||||||
public final static com.yammer.metrics.core.MetricName NUM_OFFSETS = getMetricName(
|
public final static com.yammer.metrics.core.MetricName NUM_OFFSETS = getMetricName(
|
||||||
"GroupMetadataManager", "NumOffsets");
|
"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");
|
"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");
|
"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");
|
"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");
|
"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");
|
"GroupMetadataManager", "NumGroupsEmpty");
|
||||||
|
|
||||||
public final static String GROUP_COUNT_METRIC_NAME = "group-count";
|
public final static String GROUP_COUNT_METRIC_NAME = "group-count";
|
||||||
|
@ -70,11 +70,11 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
public static final String OFFSET_COMMITS_SENSOR_NAME = "OffsetCommits";
|
public static final String OFFSET_COMMITS_SENSOR_NAME = "OffsetCommits";
|
||||||
public static final String OFFSET_EXPIRED_SENSOR_NAME = "OffsetExpired";
|
public static final String OFFSET_EXPIRED_SENSOR_NAME = "OffsetExpired";
|
||||||
public static final String OFFSET_DELETIONS_SENSOR_NAME = "OffsetDeletions";
|
public static final String OFFSET_DELETIONS_SENSOR_NAME = "OffsetDeletions";
|
||||||
public static final String GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME = "CompletedRebalances";
|
public static final String CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME = "CompletedRebalances";
|
||||||
public static final String GENERIC_GROUP_REBALANCES_SENSOR_NAME = "GenericGroupRebalances";
|
public static final String CLASSIC_GROUP_REBALANCES_SENSOR_NAME = "ClassicGroupRebalances";
|
||||||
public static final String CONSUMER_GROUP_REBALANCES_SENSOR_NAME = "ConsumerGroupRebalances";
|
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 consumerGroupCountMetricName;
|
||||||
private final MetricName consumerGroupCountEmptyMetricName;
|
private final MetricName consumerGroupCountEmptyMetricName;
|
||||||
private final MetricName consumerGroupCountAssigningMetricName;
|
private final MetricName consumerGroupCountAssigningMetricName;
|
||||||
|
@ -99,11 +99,11 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
this.registry = Objects.requireNonNull(registry);
|
this.registry = Objects.requireNonNull(registry);
|
||||||
this.metrics = Objects.requireNonNull(metrics);
|
this.metrics = Objects.requireNonNull(metrics);
|
||||||
|
|
||||||
genericGroupCountMetricName = metrics.metricName(
|
classicGroupCountMetricName = metrics.metricName(
|
||||||
GROUP_COUNT_METRIC_NAME,
|
GROUP_COUNT_METRIC_NAME,
|
||||||
METRICS_GROUP,
|
METRICS_GROUP,
|
||||||
"The total number of groups using the generic rebalance protocol.",
|
"The total number of groups using the classic rebalance protocol.",
|
||||||
Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.GENERIC.toString())
|
Collections.singletonMap(GROUP_COUNT_PROTOCOL_TAG, Group.GroupType.CLASSIC.toString())
|
||||||
);
|
);
|
||||||
|
|
||||||
consumerGroupCountMetricName = metrics.metricName(
|
consumerGroupCountMetricName = metrics.metricName(
|
||||||
|
@ -177,23 +177,23 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
METRICS_GROUP,
|
METRICS_GROUP,
|
||||||
"The total number of administrative deleted offsets")));
|
"The total number of administrative deleted offsets")));
|
||||||
|
|
||||||
Sensor genericGroupCompletedRebalancesSensor = metrics.sensor(GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
|
Sensor classicGroupCompletedRebalancesSensor = metrics.sensor(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME);
|
||||||
genericGroupCompletedRebalancesSensor.add(new Meter(
|
classicGroupCompletedRebalancesSensor.add(new Meter(
|
||||||
metrics.metricName("group-completed-rebalance-rate",
|
metrics.metricName("group-completed-rebalance-rate",
|
||||||
METRICS_GROUP,
|
METRICS_GROUP,
|
||||||
"The rate of generic group completed rebalances"),
|
"The rate of classic group completed rebalances"),
|
||||||
metrics.metricName("group-completed-rebalance-count",
|
metrics.metricName("group-completed-rebalance-count",
|
||||||
METRICS_GROUP,
|
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);
|
Sensor classicGroupPreparingRebalancesSensor = metrics.sensor(CLASSIC_GROUP_REBALANCES_SENSOR_NAME);
|
||||||
genericGroupPreparingRebalancesSensor.add(new Meter(
|
classicGroupPreparingRebalancesSensor.add(new Meter(
|
||||||
metrics.metricName("group-rebalance-rate",
|
metrics.metricName("group-rebalance-rate",
|
||||||
METRICS_GROUP,
|
METRICS_GROUP,
|
||||||
"The rate of generic group preparing rebalances"),
|
"The rate of classic group preparing rebalances"),
|
||||||
metrics.metricName("group-rebalance-count",
|
metrics.metricName("group-rebalance-count",
|
||||||
METRICS_GROUP,
|
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);
|
Sensor consumerGroupRebalanceSensor = metrics.sensor(CONSUMER_GROUP_REBALANCES_SENSOR_NAME);
|
||||||
consumerGroupRebalanceSensor.add(new Meter(
|
consumerGroupRebalanceSensor.add(new Meter(
|
||||||
|
@ -208,8 +208,8 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
Utils.mkEntry(OFFSET_COMMITS_SENSOR_NAME, offsetCommitsSensor),
|
Utils.mkEntry(OFFSET_COMMITS_SENSOR_NAME, offsetCommitsSensor),
|
||||||
Utils.mkEntry(OFFSET_EXPIRED_SENSOR_NAME, offsetExpiredSensor),
|
Utils.mkEntry(OFFSET_EXPIRED_SENSOR_NAME, offsetExpiredSensor),
|
||||||
Utils.mkEntry(OFFSET_DELETIONS_SENSOR_NAME, offsetDeletionsSensor),
|
Utils.mkEntry(OFFSET_DELETIONS_SENSOR_NAME, offsetDeletionsSensor),
|
||||||
Utils.mkEntry(GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME, genericGroupCompletedRebalancesSensor),
|
Utils.mkEntry(CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME, classicGroupCompletedRebalancesSensor),
|
||||||
Utils.mkEntry(GENERIC_GROUP_REBALANCES_SENSOR_NAME, genericGroupPreparingRebalancesSensor),
|
Utils.mkEntry(CLASSIC_GROUP_REBALANCES_SENSOR_NAME, classicGroupPreparingRebalancesSensor),
|
||||||
Utils.mkEntry(CONSUMER_GROUP_REBALANCES_SENSOR_NAME, consumerGroupRebalanceSensor)
|
Utils.mkEntry(CONSUMER_GROUP_REBALANCES_SENSOR_NAME, consumerGroupRebalanceSensor)
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
@ -218,12 +218,12 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numOffsets).sum();
|
return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numOffsets).sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Long numGenericGroups() {
|
private Long numClassicGroups() {
|
||||||
return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numGenericGroups).sum();
|
return shards.values().stream().mapToLong(GroupCoordinatorMetricsShard::numClassicGroups).sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
private Long numGenericGroups(GenericGroupState state) {
|
private Long numClassicGroups(ClassicGroupState state) {
|
||||||
return shards.values().stream().mapToLong(shard -> shard.numGenericGroups(state)).sum();
|
return shards.values().stream().mapToLong(shard -> shard.numClassicGroups(state)).sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
private long numConsumerGroups() {
|
private long numConsumerGroups() {
|
||||||
|
@ -238,16 +238,16 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
public void close() {
|
public void close() {
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
NUM_OFFSETS,
|
NUM_OFFSETS,
|
||||||
NUM_GENERIC_GROUPS,
|
NUM_CLASSIC_GROUPS,
|
||||||
NUM_GENERIC_GROUPS_PREPARING_REBALANCE,
|
NUM_CLASSIC_GROUPS_PREPARING_REBALANCE,
|
||||||
NUM_GENERIC_GROUPS_COMPLETING_REBALANCE,
|
NUM_CLASSIC_GROUPS_COMPLETING_REBALANCE,
|
||||||
NUM_GENERIC_GROUPS_STABLE,
|
NUM_CLASSIC_GROUPS_STABLE,
|
||||||
NUM_GENERIC_GROUPS_DEAD,
|
NUM_CLASSIC_GROUPS_DEAD,
|
||||||
NUM_GENERIC_GROUPS_EMPTY
|
NUM_CLASSIC_GROUPS_EMPTY
|
||||||
).forEach(registry::removeMetric);
|
).forEach(registry::removeMetric);
|
||||||
|
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
genericGroupCountMetricName,
|
classicGroupCountMetricName,
|
||||||
consumerGroupCountMetricName,
|
consumerGroupCountMetricName,
|
||||||
consumerGroupCountEmptyMetricName,
|
consumerGroupCountEmptyMetricName,
|
||||||
consumerGroupCountAssigningMetricName,
|
consumerGroupCountAssigningMetricName,
|
||||||
|
@ -260,8 +260,8 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
OFFSET_COMMITS_SENSOR_NAME,
|
OFFSET_COMMITS_SENSOR_NAME,
|
||||||
OFFSET_EXPIRED_SENSOR_NAME,
|
OFFSET_EXPIRED_SENSOR_NAME,
|
||||||
OFFSET_DELETIONS_SENSOR_NAME,
|
OFFSET_DELETIONS_SENSOR_NAME,
|
||||||
GENERIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME,
|
CLASSIC_GROUP_COMPLETED_REBALANCES_SENSOR_NAME,
|
||||||
GENERIC_GROUP_REBALANCES_SENSOR_NAME,
|
CLASSIC_GROUP_REBALANCES_SENSOR_NAME,
|
||||||
CONSUMER_GROUP_REBALANCES_SENSOR_NAME
|
CONSUMER_GROUP_REBALANCES_SENSOR_NAME
|
||||||
).forEach(metrics::removeSensor);
|
).forEach(metrics::removeSensor);
|
||||||
}
|
}
|
||||||
|
@ -309,51 +309,51 @@ public class GroupCoordinatorMetrics extends CoordinatorMetrics implements AutoC
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
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
|
@Override
|
||||||
public Long value() {
|
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
|
@Override
|
||||||
public Long value() {
|
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
|
@Override
|
||||||
public Long value() {
|
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
|
@Override
|
||||||
public Long value() {
|
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
|
@Override
|
||||||
public Long value() {
|
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
|
@Override
|
||||||
public Long value() {
|
public Long value() {
|
||||||
return numGenericGroups(GenericGroupState.EMPTY);
|
return numClassicGroups(ClassicGroupState.EMPTY);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
metrics.addMetric(
|
metrics.addMetric(
|
||||||
genericGroupCountMetricName,
|
classicGroupCountMetricName,
|
||||||
(Gauge<Long>) (config, now) -> numGenericGroups()
|
(Gauge<Long>) (config, now) -> numClassicGroups()
|
||||||
);
|
);
|
||||||
|
|
||||||
metrics.addMetric(
|
metrics.addMetric(
|
||||||
|
|
|
@ -20,7 +20,7 @@ import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.metrics.Sensor;
|
import org.apache.kafka.common.metrics.Sensor;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState;
|
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.SnapshotRegistry;
|
||||||
import org.apache.kafka.timeline.TimelineLong;
|
import org.apache.kafka.timeline.TimelineLong;
|
||||||
|
|
||||||
|
@ -56,9 +56,9 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
/**
|
/**
|
||||||
* 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.
|
* Consumer group size gauge counters keyed by the metric name.
|
||||||
|
@ -76,9 +76,9 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
private final TimelineGaugeCounter numOffsetsTimelineGaugeCounter;
|
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.
|
* The topic partition.
|
||||||
|
@ -92,14 +92,14 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
) {
|
) {
|
||||||
Objects.requireNonNull(snapshotRegistry);
|
Objects.requireNonNull(snapshotRegistry);
|
||||||
numOffsetsTimelineGaugeCounter = new TimelineGaugeCounter(new TimelineLong(snapshotRegistry), new AtomicLong(0));
|
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(
|
this.classicGroupGauges = Utils.mkMap(
|
||||||
Utils.mkEntry(GenericGroupState.PREPARING_REBALANCE, new AtomicLong(0)),
|
Utils.mkEntry(ClassicGroupState.PREPARING_REBALANCE, new AtomicLong(0)),
|
||||||
Utils.mkEntry(GenericGroupState.COMPLETING_REBALANCE, new AtomicLong(0)),
|
Utils.mkEntry(ClassicGroupState.COMPLETING_REBALANCE, new AtomicLong(0)),
|
||||||
Utils.mkEntry(GenericGroupState.STABLE, new AtomicLong(0)),
|
Utils.mkEntry(ClassicGroupState.STABLE, new AtomicLong(0)),
|
||||||
Utils.mkEntry(GenericGroupState.DEAD, new AtomicLong(0)),
|
Utils.mkEntry(ClassicGroupState.DEAD, new AtomicLong(0)),
|
||||||
Utils.mkEntry(GenericGroupState.EMPTY, new AtomicLong(0))
|
Utils.mkEntry(ClassicGroupState.EMPTY, new AtomicLong(0))
|
||||||
);
|
);
|
||||||
|
|
||||||
this.consumerGroupGauges = Utils.mkMap(
|
this.consumerGroupGauges = Utils.mkMap(
|
||||||
|
@ -119,8 +119,8 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
this.topicPartition = Objects.requireNonNull(topicPartition);
|
this.topicPartition = Objects.requireNonNull(topicPartition);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void incrementNumGenericGroups(GenericGroupState state) {
|
public void incrementNumClassicGroups(ClassicGroupState state) {
|
||||||
AtomicLong counter = genericGroupGauges.get(state);
|
AtomicLong counter = classicGroupGauges.get(state);
|
||||||
if (counter != null) {
|
if (counter != null) {
|
||||||
counter.incrementAndGet();
|
counter.incrementAndGet();
|
||||||
}
|
}
|
||||||
|
@ -159,12 +159,12 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* 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) {
|
public void decrementNumClassicGroups(ClassicGroupState state) {
|
||||||
AtomicLong counter = genericGroupGauges.get(state);
|
AtomicLong counter = classicGroupGauges.get(state);
|
||||||
if (counter != null) {
|
if (counter != null) {
|
||||||
counter.decrementAndGet();
|
counter.decrementAndGet();
|
||||||
}
|
}
|
||||||
|
@ -192,14 +192,14 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* 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) {
|
public long numClassicGroups(ClassicGroupState state) {
|
||||||
AtomicLong counter = genericGroupGauges.get(state);
|
AtomicLong counter = classicGroupGauges.get(state);
|
||||||
if (counter != null) {
|
if (counter != null) {
|
||||||
return counter.get();
|
return counter.get();
|
||||||
}
|
}
|
||||||
|
@ -207,10 +207,10 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The total number of generic groups.
|
* @return The total number of classic groups.
|
||||||
*/
|
*/
|
||||||
public long numGenericGroups() {
|
public long numClassicGroups() {
|
||||||
return genericGroupGauges.values().stream()
|
return classicGroupGauges.values().stream()
|
||||||
.mapToLong(AtomicLong::get).sum();
|
.mapToLong(AtomicLong::get).sum();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -230,7 +230,7 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return The total number of generic groups.
|
* @return The total number of consumer groups.
|
||||||
*/
|
*/
|
||||||
public long numConsumerGroups() {
|
public long numConsumerGroups() {
|
||||||
return consumerGroupGauges.values().stream()
|
return consumerGroupGauges.values().stream()
|
||||||
|
@ -268,9 +268,9 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
gaugeCounter.atomicLong.set(value);
|
gaugeCounter.atomicLong.set(value);
|
||||||
});
|
});
|
||||||
|
|
||||||
synchronized (numGenericGroupsTimelineCounter.timelineLong) {
|
synchronized (numClassicGroupsTimelineCounter.timelineLong) {
|
||||||
long value = numGenericGroupsTimelineCounter.timelineLong.get(offset);
|
long value = numClassicGroupsTimelineCounter.timelineLong.get(offset);
|
||||||
numGenericGroupsTimelineCounter.atomicLong.set(value);
|
numClassicGroupsTimelineCounter.atomicLong.set(value);
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized (numOffsetsTimelineGaugeCounter.timelineLong) {
|
synchronized (numOffsetsTimelineGaugeCounter.timelineLong) {
|
||||||
|
@ -280,51 +280,51 @@ public class GroupCoordinatorMetricsShard implements CoordinatorMetricsShard {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* 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.
|
* the counter accordingly.
|
||||||
*
|
*
|
||||||
* @param oldState The previous state. null value means that it's a new group.
|
* @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.
|
* @param newState The next state. null value means that the group has been removed.
|
||||||
*/
|
*/
|
||||||
public void onGenericGroupStateTransition(
|
public void onClassicGroupStateTransition(
|
||||||
GenericGroupState oldState,
|
ClassicGroupState oldState,
|
||||||
GenericGroupState newState
|
ClassicGroupState newState
|
||||||
) {
|
) {
|
||||||
if (newState != null) {
|
if (newState != null) {
|
||||||
switch (newState) {
|
switch (newState) {
|
||||||
case PREPARING_REBALANCE:
|
case PREPARING_REBALANCE:
|
||||||
incrementNumGenericGroups(GenericGroupState.PREPARING_REBALANCE);
|
incrementNumClassicGroups(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
break;
|
break;
|
||||||
case COMPLETING_REBALANCE:
|
case COMPLETING_REBALANCE:
|
||||||
incrementNumGenericGroups(GenericGroupState.COMPLETING_REBALANCE);
|
incrementNumClassicGroups(ClassicGroupState.COMPLETING_REBALANCE);
|
||||||
break;
|
break;
|
||||||
case STABLE:
|
case STABLE:
|
||||||
incrementNumGenericGroups(GenericGroupState.STABLE);
|
incrementNumClassicGroups(ClassicGroupState.STABLE);
|
||||||
break;
|
break;
|
||||||
case DEAD:
|
case DEAD:
|
||||||
incrementNumGenericGroups(GenericGroupState.DEAD);
|
incrementNumClassicGroups(ClassicGroupState.DEAD);
|
||||||
break;
|
break;
|
||||||
case EMPTY:
|
case EMPTY:
|
||||||
incrementNumGenericGroups(GenericGroupState.EMPTY);
|
incrementNumClassicGroups(ClassicGroupState.EMPTY);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (oldState != null) {
|
if (oldState != null) {
|
||||||
switch (oldState) {
|
switch (oldState) {
|
||||||
case PREPARING_REBALANCE:
|
case PREPARING_REBALANCE:
|
||||||
decrementNumGenericGroups(GenericGroupState.PREPARING_REBALANCE);
|
decrementNumClassicGroups(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
break;
|
break;
|
||||||
case COMPLETING_REBALANCE:
|
case COMPLETING_REBALANCE:
|
||||||
decrementNumGenericGroups(GenericGroupState.COMPLETING_REBALANCE);
|
decrementNumClassicGroups(ClassicGroupState.COMPLETING_REBALANCE);
|
||||||
break;
|
break;
|
||||||
case STABLE:
|
case STABLE:
|
||||||
decrementNumGenericGroups(GenericGroupState.STABLE);
|
decrementNumClassicGroups(ClassicGroupState.STABLE);
|
||||||
break;
|
break;
|
||||||
case DEAD:
|
case DEAD:
|
||||||
decrementNumGenericGroups(GenericGroupState.DEAD);
|
decrementNumClassicGroups(ClassicGroupState.DEAD);
|
||||||
break;
|
break;
|
||||||
case EMPTY:
|
case EMPTY:
|
||||||
decrementNumGenericGroups(GenericGroupState.EMPTY);
|
decrementNumClassicGroups(ClassicGroupState.EMPTY);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -53,11 +53,11 @@ public class GroupCoordinatorConfigTest {
|
||||||
assertEquals(Collections.singletonList(assignor), config.consumerGroupAssignors);
|
assertEquals(Collections.singletonList(assignor), config.consumerGroupAssignors);
|
||||||
assertEquals(2222, config.offsetsTopicSegmentBytes);
|
assertEquals(2222, config.offsetsTopicSegmentBytes);
|
||||||
assertEquals(3333, config.offsetMetadataMaxSize);
|
assertEquals(3333, config.offsetMetadataMaxSize);
|
||||||
assertEquals(60, config.genericGroupMaxSize);
|
assertEquals(60, config.classicGroupMaxSize);
|
||||||
assertEquals(3000, config.genericGroupInitialRebalanceDelayMs);
|
assertEquals(3000, config.classicGroupInitialRebalanceDelayMs);
|
||||||
assertEquals(5 * 60 * 1000, config.genericGroupNewMemberJoinTimeoutMs);
|
assertEquals(5 * 60 * 1000, config.classicGroupNewMemberJoinTimeoutMs);
|
||||||
assertEquals(120, config.genericGroupMinSessionTimeoutMs);
|
assertEquals(120, config.classicGroupMinSessionTimeoutMs);
|
||||||
assertEquals(10 * 60 * 1000, config.genericGroupMaxSessionTimeoutMs);
|
assertEquals(10 * 60 * 1000, config.classicGroupMaxSessionTimeoutMs);
|
||||||
assertEquals(10 * 60 * 1000, config.offsetsRetentionCheckIntervalMs);
|
assertEquals(10 * 60 * 1000, config.offsetsRetentionCheckIntervalMs);
|
||||||
assertEquals(24 * 60 * 60 * 1000L, config.offsetsRetentionMs);
|
assertEquals(24 * 60 * 60 * 1000L, config.offsetsRetentionMs);
|
||||||
assertEquals(5000, config.offsetCommitTimeoutMs);
|
assertEquals(5000, config.offsetCommitTimeoutMs);
|
||||||
|
|
|
@ -355,7 +355,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleWriteOperation(
|
when(runtime.scheduleWriteOperation(
|
||||||
ArgumentMatchers.eq("generic-group-join"),
|
ArgumentMatchers.eq("classic-group-join"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
|
@ -388,7 +388,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleWriteOperation(
|
when(runtime.scheduleWriteOperation(
|
||||||
ArgumentMatchers.eq("generic-group-join"),
|
ArgumentMatchers.eq("classic-group-join"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
|
@ -495,7 +495,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleWriteOperation(
|
when(runtime.scheduleWriteOperation(
|
||||||
ArgumentMatchers.eq("generic-group-sync"),
|
ArgumentMatchers.eq("classic-group-sync"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
|
@ -528,7 +528,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleWriteOperation(
|
when(runtime.scheduleWriteOperation(
|
||||||
ArgumentMatchers.eq("generic-group-sync"),
|
ArgumentMatchers.eq("classic-group-sync"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
|
@ -619,7 +619,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleReadOperation(
|
when(runtime.scheduleReadOperation(
|
||||||
ArgumentMatchers.eq("generic-group-heartbeat"),
|
ArgumentMatchers.eq("classic-group-heartbeat"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
)).thenReturn(CompletableFuture.completedFuture(
|
)).thenReturn(CompletableFuture.completedFuture(
|
||||||
|
@ -651,7 +651,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleReadOperation(
|
when(runtime.scheduleReadOperation(
|
||||||
ArgumentMatchers.eq("generic-group-heartbeat"),
|
ArgumentMatchers.eq("classic-group-heartbeat"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
)).thenReturn(FutureUtils.failedFuture(
|
)).thenReturn(FutureUtils.failedFuture(
|
||||||
|
@ -683,7 +683,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleReadOperation(
|
when(runtime.scheduleReadOperation(
|
||||||
ArgumentMatchers.eq("generic-group-heartbeat"),
|
ArgumentMatchers.eq("classic-group-heartbeat"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
)).thenReturn(FutureUtils.failedFuture(
|
)).thenReturn(FutureUtils.failedFuture(
|
||||||
|
@ -1240,7 +1240,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleWriteOperation(
|
when(runtime.scheduleWriteOperation(
|
||||||
ArgumentMatchers.eq("generic-group-leave"),
|
ArgumentMatchers.eq("classic-group-leave"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
|
@ -1281,7 +1281,7 @@ public class GroupCoordinatorServiceTest {
|
||||||
service.startup(() -> 1);
|
service.startup(() -> 1);
|
||||||
|
|
||||||
when(runtime.scheduleWriteOperation(
|
when(runtime.scheduleWriteOperation(
|
||||||
ArgumentMatchers.eq("generic-group-leave"),
|
ArgumentMatchers.eq("classic-group-leave"),
|
||||||
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
|
||||||
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
ArgumentMatchers.eq(Duration.ofMillis(5000)),
|
||||||
ArgumentMatchers.any()
|
ArgumentMatchers.any()
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -52,9 +52,9 @@ import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroup;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroupMember;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroupState;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
import org.apache.kafka.coordinator.group.runtime.CoordinatorResult;
|
||||||
import org.apache.kafka.image.MetadataImage;
|
import org.apache.kafka.image.MetadataImage;
|
||||||
|
@ -547,11 +547,11 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create a dead group.
|
// Create a dead group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
group.transitionTo(GenericGroupState.DEAD);
|
group.transitionTo(ClassicGroupState.DEAD);
|
||||||
|
|
||||||
// Verify that the request is rejected with the correct exception.
|
// Verify that the request is rejected with the correct exception.
|
||||||
assertThrows(CoordinatorNotAvailableException.class, () -> context.commitOffset(
|
assertThrows(CoordinatorNotAvailableException.class, () -> context.commitOffset(
|
||||||
|
@ -577,7 +577,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -606,7 +606,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -615,7 +615,7 @@ public class OffsetMetadataManagerTest {
|
||||||
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
|
|
||||||
|
@ -643,7 +643,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -676,7 +676,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -709,7 +709,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -718,7 +718,7 @@ public class OffsetMetadataManagerTest {
|
||||||
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
|
|
||||||
|
@ -746,7 +746,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -755,7 +755,7 @@ public class OffsetMetadataManagerTest {
|
||||||
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
|
|
||||||
|
@ -781,7 +781,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -790,10 +790,10 @@ public class OffsetMetadataManagerTest {
|
||||||
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
group.transitionTo(GenericGroupState.STABLE);
|
group.transitionTo(ClassicGroupState.STABLE);
|
||||||
|
|
||||||
CoordinatorResult<OffsetCommitResponseData, Record> result = context.commitOffset(
|
CoordinatorResult<OffsetCommitResponseData, Record> result = context.commitOffset(
|
||||||
new OffsetCommitRequestData()
|
new OffsetCommitRequestData()
|
||||||
|
@ -849,24 +849,24 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create a group.
|
// Create a group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
|
||||||
// Add member.
|
// Add member.
|
||||||
GenericGroupMember member = mkGenericMember("member", Optional.empty());
|
ClassicGroupMember member = mkGenericMember("member", Optional.empty());
|
||||||
group.add(member);
|
group.add(member);
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
group.transitionTo(GenericGroupState.STABLE);
|
group.transitionTo(ClassicGroupState.STABLE);
|
||||||
|
|
||||||
// Schedule session timeout. This would be normally done when
|
// Schedule session timeout. This would be normally done when
|
||||||
// the group transitions to stable.
|
// 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
|
// Advance time by half of the session timeout. No timeouts are
|
||||||
// expired.
|
// expired.
|
||||||
|
@ -952,7 +952,7 @@ public class OffsetMetadataManagerTest {
|
||||||
);
|
);
|
||||||
|
|
||||||
// A generic should have been created.
|
// A generic should have been created.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
@ -1492,20 +1492,20 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create a group.
|
// Create a group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
|
||||||
// Add member.
|
// Add member.
|
||||||
GenericGroupMember member = mkGenericMember("member", Optional.empty());
|
ClassicGroupMember member = mkGenericMember("member", Optional.empty());
|
||||||
group.add(member);
|
group.add(member);
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
group.transitionTo(GenericGroupState.STABLE);
|
group.transitionTo(ClassicGroupState.STABLE);
|
||||||
|
|
||||||
CoordinatorResult<TxnOffsetCommitResponseData, Record> result = context.commitTransactionalOffset(
|
CoordinatorResult<TxnOffsetCommitResponseData, Record> result = context.commitTransactionalOffset(
|
||||||
new TxnOffsetCommitRequestData()
|
new TxnOffsetCommitRequestData()
|
||||||
|
@ -1585,7 +1585,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -1614,20 +1614,20 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create a group.
|
// Create a group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
|
||||||
// Add member.
|
// Add member.
|
||||||
GenericGroupMember member = mkGenericMember("member", Optional.empty());
|
ClassicGroupMember member = mkGenericMember("member", Optional.empty());
|
||||||
group.add(member);
|
group.add(member);
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
group.transitionTo(GenericGroupState.STABLE);
|
group.transitionTo(ClassicGroupState.STABLE);
|
||||||
|
|
||||||
assertThrows(IllegalGenerationException.class, () -> context.commitTransactionalOffset(
|
assertThrows(IllegalGenerationException.class, () -> context.commitTransactionalOffset(
|
||||||
new TxnOffsetCommitRequestData()
|
new TxnOffsetCommitRequestData()
|
||||||
|
@ -1653,11 +1653,11 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create a dead group.
|
// Create a dead group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"group",
|
"group",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
group.transitionTo(GenericGroupState.DEAD);
|
group.transitionTo(ClassicGroupState.DEAD);
|
||||||
|
|
||||||
List<OffsetFetchRequestData.OffsetFetchRequestTopics> request = Arrays.asList(
|
List<OffsetFetchRequestData.OffsetFetchRequestTopics> request = Arrays.asList(
|
||||||
new OffsetFetchRequestData.OffsetFetchRequestTopics()
|
new OffsetFetchRequestData.OffsetFetchRequestTopics()
|
||||||
|
@ -1861,11 +1861,11 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create a dead group.
|
// Create a dead group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"group",
|
"group",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
group.transitionTo(GenericGroupState.DEAD);
|
group.transitionTo(ClassicGroupState.DEAD);
|
||||||
|
|
||||||
assertEquals(Collections.emptyList(), context.fetchAllOffsets("group", Long.MAX_VALUE));
|
assertEquals(Collections.emptyList(), context.fetchAllOffsets("group", Long.MAX_VALUE));
|
||||||
}
|
}
|
||||||
|
@ -2083,7 +2083,7 @@ public class OffsetMetadataManagerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testGenericGroupOffsetDelete() {
|
public void testGenericGroupOffsetDelete() {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -2095,7 +2095,7 @@ public class OffsetMetadataManagerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testGenericGroupOffsetDeleteWithErrors() {
|
public void testGenericGroupOffsetDeleteWithErrors() {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -2155,8 +2155,8 @@ public class OffsetMetadataManagerTest {
|
||||||
public void testDeleteGroupAllOffsets(Group.GroupType groupType) {
|
public void testDeleteGroupAllOffsets(Group.GroupType groupType) {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
switch (groupType) {
|
switch (groupType) {
|
||||||
case GENERIC:
|
case CLASSIC:
|
||||||
context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -2496,7 +2496,7 @@ public class OffsetMetadataManagerTest {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
||||||
|
|
||||||
// Create an empty group.
|
// Create an empty group.
|
||||||
GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
|
ClassicGroup group = context.groupMetadataManager.getOrMaybeCreateClassicGroup(
|
||||||
"foo",
|
"foo",
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
|
@ -2505,10 +2505,10 @@ public class OffsetMetadataManagerTest {
|
||||||
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
group.add(mkGenericMember("member", Optional.of("new-instance-id")));
|
||||||
|
|
||||||
// Transition to next generation.
|
// Transition to next generation.
|
||||||
group.transitionTo(GenericGroupState.PREPARING_REBALANCE);
|
group.transitionTo(ClassicGroupState.PREPARING_REBALANCE);
|
||||||
group.initNextGeneration();
|
group.initNextGeneration();
|
||||||
assertEquals(1, group.generationId());
|
assertEquals(1, group.generationId());
|
||||||
group.transitionTo(GenericGroupState.STABLE);
|
group.transitionTo(ClassicGroupState.STABLE);
|
||||||
|
|
||||||
CoordinatorResult<OffsetCommitResponseData, Record> result = context.commitOffset(
|
CoordinatorResult<OffsetCommitResponseData, Record> result = context.commitOffset(
|
||||||
new OffsetCommitRequestData()
|
new OffsetCommitRequestData()
|
||||||
|
@ -2588,7 +2588,7 @@ public class OffsetMetadataManagerTest {
|
||||||
@Test
|
@Test
|
||||||
public void testOffsetDeletionsSensor() {
|
public void testOffsetDeletionsSensor() {
|
||||||
OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build();
|
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", 0, 100L, 0);
|
||||||
context.commitOffset("foo", "bar", 1, 150L, 0);
|
context.commitOffset("foo", "bar", 1, 150L, 0);
|
||||||
|
@ -2659,11 +2659,11 @@ public class OffsetMetadataManagerTest {
|
||||||
));
|
));
|
||||||
}
|
}
|
||||||
|
|
||||||
private GenericGroupMember mkGenericMember(
|
private ClassicGroupMember mkGenericMember(
|
||||||
String memberId,
|
String memberId,
|
||||||
Optional<String> groupInstanceId
|
Optional<String> groupInstanceId
|
||||||
) {
|
) {
|
||||||
return new GenericGroupMember(
|
return new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
groupInstanceId,
|
groupInstanceId,
|
||||||
"client-id",
|
"client-id",
|
||||||
|
|
|
@ -42,9 +42,9 @@ import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroup;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroupMember;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroupState;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroupState;
|
||||||
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
|
||||||
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
import org.apache.kafka.server.common.ApiMessageAndVersion;
|
||||||
import org.apache.kafka.server.common.MetadataVersion;
|
import org.apache.kafka.server.common.MetadataVersion;
|
||||||
|
@ -525,10 +525,10 @@ public class RecordHelpersTest {
|
||||||
.setMembers(expectedMembers),
|
.setMembers(expectedMembers),
|
||||||
expectedGroupMetadataValueVersion));
|
expectedGroupMetadataValueVersion));
|
||||||
|
|
||||||
GenericGroup group = new GenericGroup(
|
ClassicGroup group = new ClassicGroup(
|
||||||
new LogContext(),
|
new LogContext(),
|
||||||
"group-id",
|
"group-id",
|
||||||
GenericGroupState.PREPARING_REBALANCE,
|
ClassicGroupState.PREPARING_REBALANCE,
|
||||||
time,
|
time,
|
||||||
mock(GroupCoordinatorMetricsShard.class)
|
mock(GroupCoordinatorMetricsShard.class)
|
||||||
);
|
);
|
||||||
|
@ -541,7 +541,7 @@ public class RecordHelpersTest {
|
||||||
.setName("range")
|
.setName("range")
|
||||||
.setMetadata(member.subscription()));
|
.setMetadata(member.subscription()));
|
||||||
|
|
||||||
group.add(new GenericGroupMember(
|
group.add(new ClassicGroupMember(
|
||||||
member.memberId(),
|
member.memberId(),
|
||||||
Optional.of(member.groupInstanceId()),
|
Optional.of(member.groupInstanceId()),
|
||||||
member.clientId(),
|
member.clientId(),
|
||||||
|
@ -550,7 +550,7 @@ public class RecordHelpersTest {
|
||||||
member.sessionTimeout(),
|
member.sessionTimeout(),
|
||||||
"consumer",
|
"consumer",
|
||||||
protocols,
|
protocols,
|
||||||
GenericGroupMember.EMPTY_ASSIGNMENT
|
ClassicGroupMember.EMPTY_ASSIGNMENT
|
||||||
));
|
));
|
||||||
|
|
||||||
assignment.put(member.memberId(), member.assignment());
|
assignment.put(member.memberId(), member.assignment());
|
||||||
|
@ -596,10 +596,10 @@ public class RecordHelpersTest {
|
||||||
.setAssignment(new byte[]{1, 2})
|
.setAssignment(new byte[]{1, 2})
|
||||||
);
|
);
|
||||||
|
|
||||||
GenericGroup group = new GenericGroup(
|
ClassicGroup group = new ClassicGroup(
|
||||||
new LogContext(),
|
new LogContext(),
|
||||||
"group-id",
|
"group-id",
|
||||||
GenericGroupState.PREPARING_REBALANCE,
|
ClassicGroupState.PREPARING_REBALANCE,
|
||||||
time,
|
time,
|
||||||
mock(GroupCoordinatorMetricsShard.class)
|
mock(GroupCoordinatorMetricsShard.class)
|
||||||
);
|
);
|
||||||
|
@ -610,7 +610,7 @@ public class RecordHelpersTest {
|
||||||
.setName("range")
|
.setName("range")
|
||||||
.setMetadata(null));
|
.setMetadata(null));
|
||||||
|
|
||||||
group.add(new GenericGroupMember(
|
group.add(new ClassicGroupMember(
|
||||||
member.memberId(),
|
member.memberId(),
|
||||||
Optional.of(member.groupInstanceId()),
|
Optional.of(member.groupInstanceId()),
|
||||||
member.clientId(),
|
member.clientId(),
|
||||||
|
@ -648,10 +648,10 @@ public class RecordHelpersTest {
|
||||||
.setAssignment(null)
|
.setAssignment(null)
|
||||||
);
|
);
|
||||||
|
|
||||||
GenericGroup group = new GenericGroup(
|
ClassicGroup group = new ClassicGroup(
|
||||||
new LogContext(),
|
new LogContext(),
|
||||||
"group-id",
|
"group-id",
|
||||||
GenericGroupState.PREPARING_REBALANCE,
|
ClassicGroupState.PREPARING_REBALANCE,
|
||||||
time,
|
time,
|
||||||
mock(GroupCoordinatorMetricsShard.class)
|
mock(GroupCoordinatorMetricsShard.class)
|
||||||
);
|
);
|
||||||
|
@ -662,7 +662,7 @@ public class RecordHelpersTest {
|
||||||
.setName("range")
|
.setName("range")
|
||||||
.setMetadata(member.subscription()));
|
.setMetadata(member.subscription()));
|
||||||
|
|
||||||
group.add(new GenericGroupMember(
|
group.add(new ClassicGroupMember(
|
||||||
member.memberId(),
|
member.memberId(),
|
||||||
Optional.of(member.groupInstanceId()),
|
Optional.of(member.groupInstanceId()),
|
||||||
member.clientId(),
|
member.clientId(),
|
||||||
|
@ -708,10 +708,10 @@ public class RecordHelpersTest {
|
||||||
.setMembers(expectedMembers),
|
.setMembers(expectedMembers),
|
||||||
expectedGroupMetadataValueVersion));
|
expectedGroupMetadataValueVersion));
|
||||||
|
|
||||||
GenericGroup group = new GenericGroup(
|
ClassicGroup group = new ClassicGroup(
|
||||||
new LogContext(),
|
new LogContext(),
|
||||||
"group-id",
|
"group-id",
|
||||||
GenericGroupState.PREPARING_REBALANCE,
|
ClassicGroupState.PREPARING_REBALANCE,
|
||||||
time,
|
time,
|
||||||
mock(GroupCoordinatorMetricsShard.class)
|
mock(GroupCoordinatorMetricsShard.class)
|
||||||
);
|
);
|
||||||
|
|
|
@ -15,7 +15,7 @@
|
||||||
* limitations under the License.
|
* 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.DescribeGroupsResponseData;
|
||||||
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol;
|
import org.apache.kafka.common.message.JoinGroupRequestData.JoinGroupRequestProtocol;
|
||||||
|
@ -28,14 +28,14 @@ import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
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.assertArrayEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
|
|
||||||
public class GenericGroupMemberTest {
|
public class ClassicGroupMemberTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMatchesSupportedProtocols() {
|
public void testMatchesSupportedProtocols() {
|
||||||
|
@ -44,7 +44,7 @@ public class GenericGroupMemberTest {
|
||||||
.setName("range")
|
.setName("range")
|
||||||
.setMetadata(new byte[]{0}));
|
.setMetadata(new byte[]{0}));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -101,7 +101,7 @@ public class GenericGroupMemberTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -135,7 +135,7 @@ public class GenericGroupMemberTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[]{1}));
|
.setMetadata(new byte[]{1}));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -153,7 +153,7 @@ public class GenericGroupMemberTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMetadataRaisesOnUnsupportedProtocol() {
|
public void testMetadataRaisesOnUnsupportedProtocol() {
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -180,7 +180,7 @@ public class GenericGroupMemberTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[]{1}));
|
.setMetadata(new byte[]{1}));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -199,7 +199,7 @@ public class GenericGroupMemberTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHasValidGroupInstanceId() {
|
public void testHasValidGroupInstanceId() {
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -229,12 +229,12 @@ public class GenericGroupMemberTest {
|
||||||
expectedProtocolNames.add("range");
|
expectedProtocolNames.add("range");
|
||||||
expectedProtocolNames.add("roundrobin");
|
expectedProtocolNames.add("roundrobin");
|
||||||
|
|
||||||
assertEquals(expectedProtocolNames, GenericGroupMember.plainProtocolSet(protocolCollection));
|
assertEquals(expectedProtocolNames, ClassicGroupMember.plainProtocolSet(protocolCollection));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHasHeartbeatSatisfied() {
|
public void testHasHeartbeatSatisfied() {
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -262,7 +262,7 @@ public class GenericGroupMemberTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testDescribeNoMetadata() {
|
public void testDescribeNoMetadata() {
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
||||||
|
@ -295,7 +295,7 @@ public class GenericGroupMemberTest {
|
||||||
.setMetadata(new byte[]{0})
|
.setMetadata(new byte[]{0})
|
||||||
).iterator());
|
).iterator());
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
"member",
|
"member",
|
||||||
Optional.of("group-instance-id"),
|
Optional.of("group-instance-id"),
|
||||||
"client-id",
|
"client-id",
|
|
@ -14,7 +14,7 @@
|
||||||
* See the License for the specific language governing permissions and
|
* See the License for the specific language governing permissions and
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.coordinator.group.generic;
|
package org.apache.kafka.coordinator.group.classic;
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
|
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
|
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol;
|
||||||
|
@ -51,11 +51,11 @@ import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
|
|
||||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
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.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.EMPTY;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPARING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
|
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.assertDoesNotThrow;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
|
@ -67,7 +67,7 @@ import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.times;
|
import static org.mockito.Mockito.times;
|
||||||
import static org.mockito.Mockito.verify;
|
import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
public class GenericGroupTest {
|
public class ClassicGroupTest {
|
||||||
private final String protocolType = "consumer";
|
private final String protocolType = "consumer";
|
||||||
private final String groupInstanceId = "groupInstanceId";
|
private final String groupInstanceId = "groupInstanceId";
|
||||||
private final String memberId = "memberId";
|
private final String memberId = "memberId";
|
||||||
|
@ -82,11 +82,11 @@ public class GenericGroupTest {
|
||||||
new TopicPartition("__consumer_offsets", 0)
|
new TopicPartition("__consumer_offsets", 0)
|
||||||
);
|
);
|
||||||
|
|
||||||
private GenericGroup group = null;
|
private ClassicGroup group = null;
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
public void initialize() {
|
public void initialize() {
|
||||||
group = new GenericGroup(logContext, "groupId", EMPTY, Time.SYSTEM, metrics);
|
group = new ClassicGroup(logContext, "groupId", EMPTY, Time.SYSTEM, metrics);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -242,7 +242,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member1 = new GenericGroupMember(
|
ClassicGroupMember member1 = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -263,7 +263,7 @@ public class GenericGroupTest {
|
||||||
.setName("range")
|
.setName("range")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member2 = new GenericGroupMember(
|
ClassicGroupMember member2 = new ClassicGroupMember(
|
||||||
"member2",
|
"member2",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -279,7 +279,7 @@ public class GenericGroupTest {
|
||||||
assertTrue(group.selectProtocol().equals("range") ||
|
assertTrue(group.selectProtocol().equals("range") ||
|
||||||
group.selectProtocol().equals("roundrobin"));
|
group.selectProtocol().equals("roundrobin"));
|
||||||
|
|
||||||
GenericGroupMember member3 = new GenericGroupMember(
|
ClassicGroupMember member3 = new ClassicGroupMember(
|
||||||
"member3",
|
"member3",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -311,7 +311,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member1 = new GenericGroupMember(
|
ClassicGroupMember member1 = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -333,7 +333,7 @@ public class GenericGroupTest {
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
|
|
||||||
GenericGroupMember member2 = new GenericGroupMember(
|
ClassicGroupMember member2 = new ClassicGroupMember(
|
||||||
"member2",
|
"member2",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -359,7 +359,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member1 = new GenericGroupMember(
|
ClassicGroupMember member1 = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -394,7 +394,7 @@ public class GenericGroupTest {
|
||||||
Collections.singletonList("foo")
|
Collections.singletonList("foo")
|
||||||
)).array()));
|
)).array()));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -425,7 +425,7 @@ public class GenericGroupTest {
|
||||||
.setName("range")
|
.setName("range")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember memberWithFaultyProtocol = new GenericGroupMember(
|
ClassicGroupMember memberWithFaultyProtocol = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -454,7 +454,7 @@ public class GenericGroupTest {
|
||||||
.setName("range")
|
.setName("range")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
|
ClassicGroupMember memberWithNonConsumerProtocol = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -480,7 +480,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -523,7 +523,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -568,7 +568,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.of(groupInstanceId),
|
Optional.of(groupInstanceId),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -606,7 +606,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -640,7 +640,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -670,7 +670,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -701,7 +701,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -728,7 +728,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -754,7 +754,7 @@ public class GenericGroupTest {
|
||||||
assertFalse(group.hasMemberId(memberId));
|
assertFalse(group.hasMemberId(memberId));
|
||||||
assertTrue(group.isPendingMember(memberId));
|
assertTrue(group.isPendingMember(memberId));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -788,7 +788,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.of(groupInstanceId),
|
Optional.of(groupInstanceId),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -826,7 +826,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -851,7 +851,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.of(groupInstanceId),
|
Optional.of(groupInstanceId),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -876,7 +876,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -902,7 +902,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember leader = new GenericGroupMember(
|
ClassicGroupMember leader = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -917,7 +917,7 @@ public class GenericGroupTest {
|
||||||
assertTrue(group.isLeader(memberId));
|
assertTrue(group.isLeader(memberId));
|
||||||
assertFalse(leader.isAwaitingJoin());
|
assertFalse(leader.isAwaitingJoin());
|
||||||
|
|
||||||
GenericGroupMember newLeader = new GenericGroupMember(
|
ClassicGroupMember newLeader = new ClassicGroupMember(
|
||||||
"new-leader",
|
"new-leader",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -929,7 +929,7 @@ public class GenericGroupTest {
|
||||||
);
|
);
|
||||||
group.add(newLeader, new CompletableFuture<>());
|
group.add(newLeader, new CompletableFuture<>());
|
||||||
|
|
||||||
GenericGroupMember newMember = new GenericGroupMember(
|
ClassicGroupMember newMember = new ClassicGroupMember(
|
||||||
"new-member",
|
"new-member",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -952,7 +952,7 @@ public class GenericGroupTest {
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
|
|
||||||
GenericGroupMember leader = new GenericGroupMember(
|
ClassicGroupMember leader = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -967,7 +967,7 @@ public class GenericGroupTest {
|
||||||
assertTrue(group.isLeader(memberId));
|
assertTrue(group.isLeader(memberId));
|
||||||
assertTrue(leader.isAwaitingJoin());
|
assertTrue(leader.isAwaitingJoin());
|
||||||
|
|
||||||
GenericGroupMember newMember = new GenericGroupMember(
|
ClassicGroupMember newMember = new ClassicGroupMember(
|
||||||
"new-member",
|
"new-member",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -989,7 +989,7 @@ public class GenericGroupTest {
|
||||||
group.validateOffsetCommit("", "", -1, false);
|
group.validateOffsetCommit("", "", -1, false);
|
||||||
|
|
||||||
// Add a member.
|
// Add a member.
|
||||||
group.add(new GenericGroupMember(
|
group.add(new ClassicGroupMember(
|
||||||
"member-id",
|
"member-id",
|
||||||
Optional.of("instance-id"),
|
Optional.of("instance-id"),
|
||||||
"",
|
"",
|
||||||
|
@ -1066,7 +1066,7 @@ public class GenericGroupTest {
|
||||||
protocols.add(new JoinGroupRequestProtocol()
|
protocols.add(new JoinGroupRequestProtocol()
|
||||||
.setName("roundrobin")
|
.setName("roundrobin")
|
||||||
.setMetadata(new byte[0]));
|
.setMetadata(new byte[0]));
|
||||||
GenericGroupMember member = new GenericGroupMember(
|
ClassicGroupMember member = new ClassicGroupMember(
|
||||||
memberId,
|
memberId,
|
||||||
Optional.of(groupInstanceId),
|
Optional.of(groupInstanceId),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -1116,7 +1116,7 @@ public class GenericGroupTest {
|
||||||
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty());
|
OffsetAndMetadata offsetAndMetadata = new OffsetAndMetadata(15000L, OptionalInt.empty(), "", commitTimestamp, OptionalLong.empty());
|
||||||
MockTime time = new MockTime();
|
MockTime time = new MockTime();
|
||||||
long currentStateTimestamp = time.milliseconds();
|
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.
|
// 1. Test no protocol type. Simple consumer case, Base timestamp based off of last commit timestamp.
|
||||||
Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
|
Optional<OffsetExpirationCondition> offsetExpirationCondition = group.offsetExpirationCondition();
|
||||||
|
@ -1133,7 +1133,7 @@ public class GenericGroupTest {
|
||||||
.setMetadata(ConsumerProtocol.serializeSubscription(
|
.setMetadata(ConsumerProtocol.serializeSubscription(
|
||||||
new ConsumerPartitionAssignor.Subscription(Collections.singletonList("topic"))).array()));
|
new ConsumerPartitionAssignor.Subscription(Collections.singletonList("topic"))).array()));
|
||||||
|
|
||||||
GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
|
ClassicGroupMember memberWithNonConsumerProtocol = new ClassicGroupMember(
|
||||||
"memberWithNonConsumerProtocol",
|
"memberWithNonConsumerProtocol",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -1161,7 +1161,7 @@ public class GenericGroupTest {
|
||||||
|
|
||||||
// 4. Test consumer protocol type + subscribed topics + Stable state. Base timestamp based off of last commit timestamp.
|
// 4. Test consumer protocol type + subscribed topics + Stable state. Base timestamp based off of last commit timestamp.
|
||||||
group.remove("memberWithNonConsumerProtocol");
|
group.remove("memberWithNonConsumerProtocol");
|
||||||
GenericGroupMember memberWithConsumerProtocol = new GenericGroupMember(
|
ClassicGroupMember memberWithConsumerProtocol = new ClassicGroupMember(
|
||||||
"memberWithConsumerProtocol",
|
"memberWithConsumerProtocol",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -1191,7 +1191,7 @@ public class GenericGroupTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIsSubscribedToTopic() {
|
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
|
// 1. group has no protocol type => not subscribed
|
||||||
assertFalse(group.isSubscribedToTopic("topic"));
|
assertFalse(group.isSubscribedToTopic("topic"));
|
||||||
|
@ -1203,7 +1203,7 @@ public class GenericGroupTest {
|
||||||
.setMetadata(ConsumerProtocol.serializeSubscription(
|
.setMetadata(ConsumerProtocol.serializeSubscription(
|
||||||
new ConsumerPartitionAssignor.Subscription(Collections.singletonList("topic"))).array()));
|
new ConsumerPartitionAssignor.Subscription(Collections.singletonList("topic"))).array()));
|
||||||
|
|
||||||
GenericGroupMember memberWithNonConsumerProtocol = new GenericGroupMember(
|
ClassicGroupMember memberWithNonConsumerProtocol = new ClassicGroupMember(
|
||||||
"memberWithNonConsumerProtocol",
|
"memberWithNonConsumerProtocol",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -1223,7 +1223,7 @@ public class GenericGroupTest {
|
||||||
|
|
||||||
// 3. group uses consumer group protocol type but empty members => not subscribed
|
// 3. group uses consumer group protocol type but empty members => not subscribed
|
||||||
group.remove("memberWithNonConsumerProtocol");
|
group.remove("memberWithNonConsumerProtocol");
|
||||||
GenericGroupMember memberWithConsumerProtocol = new GenericGroupMember(
|
ClassicGroupMember memberWithConsumerProtocol = new ClassicGroupMember(
|
||||||
"memberWithConsumerProtocol",
|
"memberWithConsumerProtocol",
|
||||||
Optional.empty(),
|
Optional.empty(),
|
||||||
clientId,
|
clientId,
|
||||||
|
@ -1258,24 +1258,24 @@ public class GenericGroupTest {
|
||||||
// Confirm metrics is not updated when a new GenericGroup is created but only when the group transitions
|
// Confirm metrics is not updated when a new GenericGroup is created but only when the group transitions
|
||||||
// its state.
|
// its state.
|
||||||
GroupCoordinatorMetricsShard metrics = mock(GroupCoordinatorMetricsShard.class);
|
GroupCoordinatorMetricsShard metrics = mock(GroupCoordinatorMetricsShard.class);
|
||||||
GenericGroup group = new GenericGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM, metrics);
|
ClassicGroup group = new ClassicGroup(new LogContext(), "groupId", EMPTY, Time.SYSTEM, metrics);
|
||||||
verify(metrics, times(0)).onGenericGroupStateTransition(any(), any());
|
verify(metrics, times(0)).onClassicGroupStateTransition(any(), any());
|
||||||
|
|
||||||
group.transitionTo(PREPARING_REBALANCE);
|
group.transitionTo(PREPARING_REBALANCE);
|
||||||
verify(metrics, times(1)).onGenericGroupStateTransition(EMPTY, PREPARING_REBALANCE);
|
verify(metrics, times(1)).onClassicGroupStateTransition(EMPTY, PREPARING_REBALANCE);
|
||||||
|
|
||||||
group.transitionTo(COMPLETING_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);
|
group.transitionTo(STABLE);
|
||||||
verify(metrics, times(1)).onGenericGroupStateTransition(COMPLETING_REBALANCE, STABLE);
|
verify(metrics, times(1)).onClassicGroupStateTransition(COMPLETING_REBALANCE, STABLE);
|
||||||
|
|
||||||
group.transitionTo(DEAD);
|
group.transitionTo(DEAD);
|
||||||
verify(metrics, times(1)).onGenericGroupStateTransition(STABLE, DEAD);
|
verify(metrics, times(1)).onClassicGroupStateTransition(STABLE, DEAD);
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertState(GenericGroup group, GenericGroupState targetState) {
|
private void assertState(ClassicGroup group, ClassicGroupState targetState) {
|
||||||
Set<GenericGroupState> otherStates = new HashSet<>();
|
Set<ClassicGroupState> otherStates = new HashSet<>();
|
||||||
otherStates.add(STABLE);
|
otherStates.add(STABLE);
|
||||||
otherStates.add(PREPARING_REBALANCE);
|
otherStates.add(PREPARING_REBALANCE);
|
||||||
otherStates.add(COMPLETING_REBALANCE);
|
otherStates.add(COMPLETING_REBALANCE);
|
|
@ -24,19 +24,19 @@ import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
|
||||||
import org.apache.kafka.coordinator.group.generic.GenericGroup;
|
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
|
||||||
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.SnapshotRegistry;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.COMPLETING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.DEAD;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.DEAD;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.EMPTY;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.EMPTY;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.PREPARING_REBALANCE;
|
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.PREPARING_REBALANCE;
|
||||||
import static org.apache.kafka.coordinator.group.generic.GenericGroupState.STABLE;
|
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.assertGaugeValue;
|
||||||
import static org.apache.kafka.coordinator.group.metrics.MetricsTestUtils.metricName;
|
import static org.apache.kafka.coordinator.group.metrics.MetricsTestUtils.metricName;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
|
@ -106,39 +106,39 @@ public class GroupCoordinatorMetricsShardTest {
|
||||||
coordinatorMetrics.activateMetricsShard(shard);
|
coordinatorMetrics.activateMetricsShard(shard);
|
||||||
|
|
||||||
LogContext logContext = new LogContext();
|
LogContext logContext = new LogContext();
|
||||||
GenericGroup group0 = new GenericGroup(logContext, "groupId0", EMPTY, Time.SYSTEM, shard);
|
ClassicGroup group0 = new ClassicGroup(logContext, "groupId0", EMPTY, Time.SYSTEM, shard);
|
||||||
GenericGroup group1 = new GenericGroup(logContext, "groupId1", EMPTY, Time.SYSTEM, shard);
|
ClassicGroup group1 = new ClassicGroup(logContext, "groupId1", EMPTY, Time.SYSTEM, shard);
|
||||||
GenericGroup group2 = new GenericGroup(logContext, "groupId2", EMPTY, Time.SYSTEM, shard);
|
ClassicGroup group2 = new ClassicGroup(logContext, "groupId2", EMPTY, Time.SYSTEM, shard);
|
||||||
GenericGroup group3 = new GenericGroup(logContext, "groupId3", 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(PREPARING_REBALANCE);
|
||||||
group0.transitionTo(COMPLETING_REBALANCE);
|
group0.transitionTo(COMPLETING_REBALANCE);
|
||||||
group1.transitionTo(PREPARING_REBALANCE);
|
group1.transitionTo(PREPARING_REBALANCE);
|
||||||
group2.transitionTo(DEAD);
|
group2.transitionTo(DEAD);
|
||||||
|
|
||||||
assertEquals(1, shard.numGenericGroups(GenericGroupState.EMPTY));
|
assertEquals(1, shard.numClassicGroups(ClassicGroupState.EMPTY));
|
||||||
assertEquals(1, shard.numGenericGroups(GenericGroupState.PREPARING_REBALANCE));
|
assertEquals(1, shard.numClassicGroups(ClassicGroupState.PREPARING_REBALANCE));
|
||||||
assertEquals(1, shard.numGenericGroups(GenericGroupState.COMPLETING_REBALANCE));
|
assertEquals(1, shard.numClassicGroups(ClassicGroupState.COMPLETING_REBALANCE));
|
||||||
assertEquals(1, shard.numGenericGroups(GenericGroupState.DEAD));
|
assertEquals(1, shard.numClassicGroups(ClassicGroupState.DEAD));
|
||||||
assertEquals(0, shard.numGenericGroups(GenericGroupState.STABLE));
|
assertEquals(0, shard.numClassicGroups(ClassicGroupState.STABLE));
|
||||||
|
|
||||||
group0.transitionTo(STABLE);
|
group0.transitionTo(STABLE);
|
||||||
group1.transitionTo(COMPLETING_REBALANCE);
|
group1.transitionTo(COMPLETING_REBALANCE);
|
||||||
group3.transitionTo(DEAD);
|
group3.transitionTo(DEAD);
|
||||||
|
|
||||||
assertEquals(0, shard.numGenericGroups(GenericGroupState.EMPTY));
|
assertEquals(0, shard.numClassicGroups(ClassicGroupState.EMPTY));
|
||||||
assertEquals(0, shard.numGenericGroups(GenericGroupState.PREPARING_REBALANCE));
|
assertEquals(0, shard.numClassicGroups(ClassicGroupState.PREPARING_REBALANCE));
|
||||||
assertEquals(1, shard.numGenericGroups(GenericGroupState.COMPLETING_REBALANCE));
|
assertEquals(1, shard.numClassicGroups(ClassicGroupState.COMPLETING_REBALANCE));
|
||||||
assertEquals(2, shard.numGenericGroups(GenericGroupState.DEAD));
|
assertEquals(2, shard.numClassicGroups(ClassicGroupState.DEAD));
|
||||||
assertEquals(1, shard.numGenericGroups(GenericGroupState.STABLE));
|
assertEquals(1, shard.numClassicGroups(ClassicGroupState.STABLE));
|
||||||
|
|
||||||
assertGaugeValue(
|
assertGaugeValue(
|
||||||
metrics,
|
metrics,
|
||||||
metrics.metricName("group-count", "group-coordinator-metrics", Collections.singletonMap("protocol", "generic")),
|
metrics.metricName("group-count", "group-coordinator-metrics", Collections.singletonMap("protocol", "classic")),
|
||||||
4
|
4
|
||||||
);
|
);
|
||||||
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 4);
|
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 4);
|
||||||
|
|
|
@ -24,7 +24,7 @@ import org.apache.kafka.common.utils.LogContext;
|
||||||
import org.apache.kafka.common.utils.MockTime;
|
import org.apache.kafka.common.utils.MockTime;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.coordinator.group.consumer.ConsumerGroup.ConsumerGroupState;
|
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.SnapshotRegistry;
|
||||||
import org.junit.jupiter.api.Test;
|
import org.junit.jupiter.api.Test;
|
||||||
|
|
||||||
|
@ -33,9 +33,9 @@ import java.util.Collections;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.stream.IntStream;
|
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.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.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_COMMITS_SENSOR_NAME;
|
||||||
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.OFFSET_EXPIRED_SENSOR_NAME;
|
import static org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics.OFFSET_EXPIRED_SENSOR_NAME;
|
||||||
|
@ -69,7 +69,7 @@ public class GroupCoordinatorMetricsTest {
|
||||||
metrics.metricName(
|
metrics.metricName(
|
||||||
"group-count",
|
"group-count",
|
||||||
GroupCoordinatorMetrics.METRICS_GROUP,
|
GroupCoordinatorMetrics.METRICS_GROUP,
|
||||||
Collections.singletonMap("protocol", "generic")),
|
Collections.singletonMap("protocol", "classic")),
|
||||||
metrics.metricName(
|
metrics.metricName(
|
||||||
"group-count",
|
"group-count",
|
||||||
GroupCoordinatorMetrics.METRICS_GROUP,
|
GroupCoordinatorMetrics.METRICS_GROUP,
|
||||||
|
@ -132,11 +132,11 @@ public class GroupCoordinatorMetricsTest {
|
||||||
coordinatorMetrics.activateMetricsShard(shard0);
|
coordinatorMetrics.activateMetricsShard(shard0);
|
||||||
coordinatorMetrics.activateMetricsShard(shard1);
|
coordinatorMetrics.activateMetricsShard(shard1);
|
||||||
|
|
||||||
IntStream.range(0, 5).forEach(__ -> shard0.incrementNumGenericGroups(GenericGroupState.PREPARING_REBALANCE));
|
IntStream.range(0, 5).forEach(__ -> shard0.incrementNumClassicGroups(ClassicGroupState.PREPARING_REBALANCE));
|
||||||
IntStream.range(0, 1).forEach(__ -> shard0.decrementNumGenericGroups(GenericGroupState.COMPLETING_REBALANCE));
|
IntStream.range(0, 1).forEach(__ -> shard0.decrementNumClassicGroups(ClassicGroupState.COMPLETING_REBALANCE));
|
||||||
IntStream.range(0, 5).forEach(__ -> shard1.incrementNumGenericGroups(GenericGroupState.STABLE));
|
IntStream.range(0, 5).forEach(__ -> shard1.incrementNumClassicGroups(ClassicGroupState.STABLE));
|
||||||
IntStream.range(0, 4).forEach(__ -> shard1.incrementNumGenericGroups(GenericGroupState.DEAD));
|
IntStream.range(0, 4).forEach(__ -> shard1.incrementNumClassicGroups(ClassicGroupState.DEAD));
|
||||||
IntStream.range(0, 4).forEach(__ -> shard1.decrementNumGenericGroups(GenericGroupState.EMPTY));
|
IntStream.range(0, 4).forEach(__ -> shard1.decrementNumClassicGroups(ClassicGroupState.EMPTY));
|
||||||
|
|
||||||
IntStream.range(0, 5).forEach(__ -> shard0.incrementNumConsumerGroups(ConsumerGroupState.ASSIGNING));
|
IntStream.range(0, 5).forEach(__ -> shard0.incrementNumConsumerGroups(ConsumerGroupState.ASSIGNING));
|
||||||
IntStream.range(0, 5).forEach(__ -> shard1.incrementNumConsumerGroups(ConsumerGroupState.RECONCILING));
|
IntStream.range(0, 5).forEach(__ -> shard1.incrementNumConsumerGroups(ConsumerGroupState.RECONCILING));
|
||||||
|
@ -146,12 +146,12 @@ public class GroupCoordinatorMetricsTest {
|
||||||
IntStream.range(0, 2).forEach(__ -> shard1.incrementNumOffsets());
|
IntStream.range(0, 2).forEach(__ -> shard1.incrementNumOffsets());
|
||||||
IntStream.range(0, 1).forEach(__ -> shard1.decrementNumOffsets());
|
IntStream.range(0, 1).forEach(__ -> shard1.decrementNumOffsets());
|
||||||
|
|
||||||
assertEquals(4, shard0.numGenericGroups());
|
assertEquals(4, shard0.numClassicGroups());
|
||||||
assertEquals(5, shard1.numGenericGroups());
|
assertEquals(5, shard1.numClassicGroups());
|
||||||
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 9);
|
assertGaugeValue(registry, metricName("GroupMetadataManager", "NumGroups"), 9);
|
||||||
assertGaugeValue(
|
assertGaugeValue(
|
||||||
metrics,
|
metrics,
|
||||||
metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "generic")),
|
metrics.metricName("group-count", METRICS_GROUP, Collections.singletonMap("protocol", "classic")),
|
||||||
9
|
9
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -182,7 +182,7 @@ public class GroupCoordinatorMetricsTest {
|
||||||
new SnapshotRegistry(new LogContext()), new TopicPartition("__consumer_offsets", 0)
|
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-rate", GroupCoordinatorMetrics.METRICS_GROUP), 1.0 / 3.0);
|
||||||
assertMetricValue(metrics, metrics.metricName("group-completed-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), 10);
|
assertMetricValue(metrics, metrics.metricName("group-completed-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), 10);
|
||||||
|
|
||||||
|
@ -194,7 +194,7 @@ public class GroupCoordinatorMetricsTest {
|
||||||
assertMetricValue(metrics, metrics.metricName("offset-expiration-rate", GroupCoordinatorMetrics.METRICS_GROUP), 1.0);
|
assertMetricValue(metrics, metrics.metricName("offset-expiration-rate", GroupCoordinatorMetrics.METRICS_GROUP), 1.0);
|
||||||
assertMetricValue(metrics, metrics.metricName("offset-expiration-count", GroupCoordinatorMetrics.METRICS_GROUP), 30);
|
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-rate", GroupCoordinatorMetrics.METRICS_GROUP), 4.0 / 3.0);
|
||||||
assertMetricValue(metrics, metrics.metricName("group-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), 40);
|
assertMetricValue(metrics, metrics.metricName("group-rebalance-count", GroupCoordinatorMetrics.METRICS_GROUP), 40);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue