From b4674c002fb7b6cd7f75be5197f924357a46ff4c Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Tue, 21 Jan 2025 21:51:39 +0800 Subject: [PATCH 01/44] KAFKA-18585 Fix fail test ValuesTest#shouldConvertDateValues (#18611) Reviewers: Divij Vaidya , Andrew Schofield --- .../java/org/apache/kafka/connect/data/ValuesTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java index 4c62393e3ea04..e81b1f8a06ec1 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/data/ValuesTest.java @@ -887,7 +887,9 @@ public void shouldConvertTimeValues() { @Test public void shouldConvertDateValues() { - java.util.Date current = new java.util.Date(); + LocalDateTime localTime = LocalDateTime.now(); + ZoneOffset zoneOffset = ZoneId.systemDefault().getRules().getOffset(localTime); + java.util.Date current = new java.util.Date(localTime.toEpochSecond(zoneOffset) * 1000); long currentMillis = current.getTime() % MILLIS_PER_DAY; long days = current.getTime() / MILLIS_PER_DAY; @@ -901,8 +903,10 @@ public void shouldConvertDateValues() { assertEquals(currentDate, d2); // ISO8601 strings - accept a string matching pattern "yyyy-MM-dd" + LocalDateTime localTimeTruncated = localTime.truncatedTo(ChronoUnit.DAYS); java.util.Date d3 = Values.convertToDate(Date.SCHEMA, LocalDate.ofEpochDay(days).format(DateTimeFormatter.ISO_LOCAL_DATE)); - assertEquals(currentDate, d3); + LocalDateTime date3 = LocalDateTime.ofInstant(Instant.ofEpochMilli(d3.getTime()), ZoneId.systemDefault()); + assertEquals(localTimeTruncated, date3); // Days as string java.util.Date d4 = Values.convertToDate(Date.SCHEMA, Long.toString(days)); From 7cbfd22bdef9c45c1f9bd20cbf741bf262e3cafb Mon Sep 17 00:00:00 2001 From: Andrew Schofield Date: Tue, 21 Jan 2025 13:56:40 +0000 Subject: [PATCH 02/44] MINOR: Improve javadoc for ListShareGroupOffsetsResult (#18650) Reviewers: Lianet Magrans , PoAn Yang , Apoorv Mittal --- .../kafka/clients/admin/ListShareGroupOffsetsResult.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java b/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java index cdcd4075af068..5ec11fad6427f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/ListShareGroupOffsetsResult.java @@ -45,7 +45,7 @@ public class ListShareGroupOffsetsResult { /** * Return the future when the requests for all groups succeed. * - * @return - Future which yields all Map<String, Map<TopicPartition, Long>> objects, if requests for all the groups succeed. + * @return Future which yields all {@code Map>} objects, if requests for all the groups succeed. */ public KafkaFuture>> all() { return KafkaFuture.allOf(futures.values().toArray(new KafkaFuture[0])).thenApply( @@ -65,8 +65,10 @@ public KafkaFuture>> all() { } /** - * @param groupId - The groupId for which the Map<TopicPartition, Long> is needed - * @return - Future which yields a map of topic partitions to offsets for the specified group. + * Return a future which yields a map of topic partitions to offsets for the specified group. + * + * @param groupId The group ID. + * @return Future which yields a map of topic partitions to offsets for the specified group. */ public KafkaFuture> partitionsToOffset(String groupId) { if (!futures.containsKey(groupId)) { From 3d49159c841e7653e3951af4ffc3524d17339295 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Tue, 21 Jan 2025 23:23:39 +0800 Subject: [PATCH 03/44] KAFKA-18598: Remove ControllerMetadataMetrics ZK-related Metrics (#18629) Reviewers: Christo Lolov --- docs/zk2kraft.html | 12 +++++ .../metrics/ControllerMetadataMetrics.java | 46 +------------------ .../ControllerMetadataMetricsPublisher.java | 9 ---- .../metrics/ControllerMetricsChanges.java | 13 ------ .../ControllerMetadataMetricsTest.java | 2 - .../metrics/ControllerMetricsChangesTest.java | 26 ----------- 6 files changed, 14 insertions(+), 94 deletions(-) diff --git a/docs/zk2kraft.html b/docs/zk2kraft.html index 210feee581e02..f85b59b3bdc9b 100644 --- a/docs/zk2kraft.html +++ b/docs/zk2kraft.html @@ -164,5 +164,17 @@
Removal metrics
In Kraft mode, Zookeeper is not used, so the metrics is removed.

+
  • +

    + Remove the metrics which is monitoring the number of Zookeeper migrations. +

    +
      +
    • kafka.controller:type=KafkaController,name=MigratingZkBrokerCount
    • +
    • kafka.controller:type=KafkaController,name=ZkMigrationState
    • +
    +

    + Kafka remove all zookeeper dependencies, so the metrics is removed. +

    +
  • \ No newline at end of file diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java index 884bc709cf6f8..890fbb30e6385 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java @@ -42,8 +42,6 @@ public final class ControllerMetadataMetrics implements AutoCloseable { "KafkaController", "FencedBrokerCount"); private static final MetricName ACTIVE_BROKER_COUNT = getMetricName( "KafkaController", "ActiveBrokerCount"); - private static final MetricName MIGRATING_ZK_BROKER_COUNT = getMetricName( - "KafkaController", "MigratingZkBrokerCount"); private static final MetricName GLOBAL_TOPIC_COUNT = getMetricName( "KafkaController", "GlobalTopicCount"); private static final MetricName GLOBAL_PARTITION_COUNT = getMetricName( @@ -54,21 +52,17 @@ public final class ControllerMetadataMetrics implements AutoCloseable { "KafkaController", "PreferredReplicaImbalanceCount"); private static final MetricName METADATA_ERROR_COUNT = getMetricName( "KafkaController", "MetadataErrorCount"); - private static final MetricName ZK_MIGRATION_STATE = getMetricName( - "KafkaController", "ZkMigrationState"); private static final MetricName UNCLEAN_LEADER_ELECTIONS_PER_SEC = getMetricName( "ControllerStats", "UncleanLeaderElectionsPerSec"); private final Optional registry; private final AtomicInteger fencedBrokerCount = new AtomicInteger(0); private final AtomicInteger activeBrokerCount = new AtomicInteger(0); - private final AtomicInteger migratingZkBrokerCount = new AtomicInteger(0); private final AtomicInteger globalTopicCount = new AtomicInteger(0); private final AtomicInteger globalPartitionCount = new AtomicInteger(0); private final AtomicInteger offlinePartitionCount = new AtomicInteger(0); private final AtomicInteger preferredReplicaImbalanceCount = new AtomicInteger(0); private final AtomicInteger metadataErrorCount = new AtomicInteger(0); - private final AtomicInteger zkMigrationState = new AtomicInteger(-1); private Optional uncleanLeaderElectionMeter = Optional.empty(); @@ -121,20 +115,6 @@ public Integer value() { return metadataErrorCount(); } })); - registry.ifPresent(r -> r.newGauge(ZK_MIGRATION_STATE, new Gauge() { - @Override - public Integer value() { - return (int) zkMigrationState(); - } - })); - - registry.ifPresent(r -> r.newGauge(MIGRATING_ZK_BROKER_COUNT, new Gauge() { - @Override - public Integer value() { - return migratingZkBrokerCount(); - } - })); - registry.ifPresent(r -> uncleanLeaderElectionMeter = Optional.of(registry.get().newMeter(UNCLEAN_LEADER_ELECTIONS_PER_SEC, "elections", TimeUnit.SECONDS))); } @@ -162,19 +142,7 @@ public void addToActiveBrokerCount(int brokerCountDelta) { public int activeBrokerCount() { return this.activeBrokerCount.get(); } - - public void setMigratingZkBrokerCount(int brokerCount) { - this.migratingZkBrokerCount.set(brokerCount); - } - - public void addToMigratingZkBrokerCount(int brokerCountDelta) { - this.migratingZkBrokerCount.addAndGet(brokerCountDelta); - } - - public int migratingZkBrokerCount() { - return this.migratingZkBrokerCount.get(); - } - + public void setGlobalTopicCount(int topicCount) { this.globalTopicCount.set(topicCount); } @@ -230,15 +198,7 @@ public void incrementMetadataErrorCount() { public int metadataErrorCount() { return this.metadataErrorCount.get(); } - - public void setZkMigrationState(byte migrationStateValue) { - this.zkMigrationState.set(migrationStateValue); - } - - public byte zkMigrationState() { - return zkMigrationState.byteValue(); - } - + public void updateUncleanLeaderElection(int count) { this.uncleanLeaderElectionMeter.ifPresent(m -> m.mark(count)); } @@ -248,13 +208,11 @@ public void close() { registry.ifPresent(r -> Arrays.asList( FENCED_BROKER_COUNT, ACTIVE_BROKER_COUNT, - MIGRATING_ZK_BROKER_COUNT, GLOBAL_TOPIC_COUNT, GLOBAL_PARTITION_COUNT, OFFLINE_PARTITION_COUNT, PREFERRED_REPLICA_IMBALANCE_COUNT, METADATA_ERROR_COUNT, - ZK_MIGRATION_STATE, UNCLEAN_LEADER_ELECTIONS_PER_SEC ).forEach(r::removeMetric)); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisher.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisher.java index c4aec11079362..565de3c1a9ba5 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsPublisher.java @@ -39,10 +39,6 @@ * behind the latest in-memory state which has not yet been fully persisted to the log. This is * reasonable for metrics, which don't need up-to-the-millisecond update latency. * - * NOTE: the ZK controller has some special rules for calculating preferredReplicaImbalanceCount - * which we haven't implemented here. Specifically, the ZK controller considers reassigning - * partitions to always have their preferred leader, even if they don't. - * All other metrics should be the same, as far as is possible. */ public class ControllerMetadataMetricsPublisher implements MetadataPublisher { private final ControllerMetadataMetrics metrics; @@ -121,20 +117,15 @@ private void publishSnapshot(MetadataImage newImage) { metrics.setGlobalTopicCount(newImage.topics().topicsById().size()); int fencedBrokers = 0; int activeBrokers = 0; - int zkBrokers = 0; for (BrokerRegistration broker : newImage.cluster().brokers().values()) { if (broker.fenced()) { fencedBrokers++; } else { activeBrokers++; } - if (broker.isMigratingZkBroker()) { - zkBrokers++; - } } metrics.setFencedBrokerCount(fencedBrokers); metrics.setActiveBrokerCount(activeBrokers); - metrics.setMigratingZkBrokerCount(zkBrokers); int totalPartitions = 0; int offlinePartitions = 0; diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java index a4b30a9f9f359..dc5b5cf5b93ea 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetricsChanges.java @@ -43,7 +43,6 @@ static int delta(boolean prev, boolean next) { private int fencedBrokersChange = 0; private int activeBrokersChange = 0; - private int migratingZkBrokersChange = 0; private int globalTopicsChange = 0; private int globalPartitionsChange = 0; private int offlinePartitionsChange = 0; @@ -58,10 +57,6 @@ public int activeBrokersChange() { return activeBrokersChange; } - public int migratingZkBrokersChange() { - return migratingZkBrokersChange; - } - public int globalTopicsChange() { return globalTopicsChange; } @@ -81,23 +76,18 @@ public int partitionsWithoutPreferredLeaderChange() { void handleBrokerChange(BrokerRegistration prev, BrokerRegistration next) { boolean wasFenced = false; boolean wasActive = false; - boolean wasZk = false; if (prev != null) { wasFenced = prev.fenced(); wasActive = !prev.fenced(); - wasZk = prev.isMigratingZkBroker(); } boolean isFenced = false; boolean isActive = false; - boolean isZk = false; if (next != null) { isFenced = next.fenced(); isActive = !next.fenced(); - isZk = next.isMigratingZkBroker(); } fencedBrokersChange += delta(wasFenced, isFenced); activeBrokersChange += delta(wasActive, isActive); - migratingZkBrokersChange += delta(wasZk, isZk); } void handleDeletedTopic(TopicImage deletedTopic) { @@ -157,9 +147,6 @@ void apply(ControllerMetadataMetrics metrics) { if (activeBrokersChange != 0) { metrics.addToActiveBrokerCount(activeBrokersChange); } - if (migratingZkBrokersChange != 0) { - metrics.addToMigratingZkBrokerCount(migratingZkBrokersChange); - } if (globalTopicsChange != 0) { metrics.addToGlobalTopicCount(globalTopicsChange); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java index 6ebb582c158ff..f6dd72edb2fe8 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java @@ -43,13 +43,11 @@ public void testMetricNames() { new HashSet<>(Arrays.asList( "kafka.controller:type=KafkaController,name=ActiveBrokerCount", "kafka.controller:type=KafkaController,name=FencedBrokerCount", - "kafka.controller:type=KafkaController,name=MigratingZkBrokerCount", "kafka.controller:type=KafkaController,name=GlobalPartitionCount", "kafka.controller:type=KafkaController,name=GlobalTopicCount", "kafka.controller:type=KafkaController,name=MetadataErrorCount", "kafka.controller:type=KafkaController,name=OfflinePartitionsCount", "kafka.controller:type=KafkaController,name=PreferredReplicaImbalanceCount", - "kafka.controller:type=KafkaController,name=ZkMigrationState", "kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec" ))); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java index 0cfea7b4ce5b6..80f54daf27b10 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java @@ -60,18 +60,6 @@ private static BrokerRegistration brokerRegistration( setInControlledShutdown(false).build(); } - private static BrokerRegistration zkBrokerRegistration( - int brokerId - ) { - return new BrokerRegistration.Builder(). - setId(brokerId). - setEpoch(100L). - setIncarnationId(Uuid.fromString("Pxi6QwS2RFuN8VSKjqJZyQ")). - setFenced(false). - setInControlledShutdown(false). - setIsMigratingZkBroker(true).build(); - } - @Test public void testInitialValues() { ControllerMetricsChanges changes = new ControllerMetricsChanges(); @@ -115,20 +103,6 @@ public void testHandleBrokerUnfencing() { assertEquals(1, changes.activeBrokersChange()); } - @Test - public void testHandleZkBroker() { - ControllerMetricsChanges changes = new ControllerMetricsChanges(); - changes.handleBrokerChange(null, zkBrokerRegistration(1)); - assertEquals(1, changes.migratingZkBrokersChange()); - changes.handleBrokerChange(null, zkBrokerRegistration(2)); - changes.handleBrokerChange(null, zkBrokerRegistration(3)); - assertEquals(3, changes.migratingZkBrokersChange()); - - changes.handleBrokerChange(zkBrokerRegistration(3), brokerRegistration(3, true)); - changes.handleBrokerChange(brokerRegistration(3, true), brokerRegistration(3, false)); - assertEquals(2, changes.migratingZkBrokersChange()); - } - @Test public void testHandleDeletedTopic() { ControllerMetricsChanges changes = new ControllerMetricsChanges(); From faff2de6a5d10b79e1992d1e4357ae9c80c0b56d Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 21 Jan 2025 08:09:43 -0800 Subject: [PATCH 04/44] KAFKA-17561: add processId tag to thread-state metric (#18581) Part of KIP-1091. Reviewers: Bill Bejeck --- docs/ops.html | 2 +- .../processor/internals/StreamThread.java | 3 ++ .../internals/metrics/StreamsMetricsImpl.java | 37 ++++++++++++++----- .../internals/metrics/ThreadMetrics.java | 6 ++- .../processor/internals/StreamThreadTest.java | 10 +++++ .../metrics/StreamsMetricsImplTest.java | 34 ++++++++++++++++- .../internals/metrics/ThreadMetricsTest.java | 3 ++ 7 files changed, 82 insertions(+), 13 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index 228d17734753a..fdb415d13ffd3 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -2826,7 +2826,7 @@

    kafka-configs.sh can be used to create and update credentials after Kafka brokers are started.

    Create initial SCRAM credentials for user admin with password admin-secret: -

    $ bin/kafka-storage.sh format -t $(bin/kafka-storage.sh random-uuid) -c config/kraft/server.properties --add-scram 'SCRAM-SHA-256=[name="admin",password="admin-secret"]'
    +
    $ bin/kafka-storage.sh format -t $(bin/kafka-storage.sh random-uuid) -c config/server.properties --add-scram 'SCRAM-SHA-256=[name="admin",password="admin-secret"]'

    Create SCRAM credentials for user alice with password alice-secret (refer to Configuring Kafka Clients for client configuration):

    $ bin/kafka-configs.sh --bootstrap-server localhost:9092 --alter --add-config 'SCRAM-SHA-256=[iterations=8192,password=alice-secret]' --entity-type users --entity-name alice --command-config client.properties

    The default iteration count of 4096 is used if iterations are not specified. A random salt is created if it's not specified. diff --git a/docs/streams/quickstart.html b/docs/streams/quickstart.html index 710bec4335108..ad14001293dc5 100644 --- a/docs/streams/quickstart.html +++ b/docs/streams/quickstart.html @@ -106,13 +106,13 @@

    $ bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/kraft/reconfig-server.properties +
    $ bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/server.properties

    Start the Kafka Server

    -
    $ bin/kafka-server-start.sh config/kraft/reconfig-server.properties
    +
    $ bin/kafka-server-start.sh config/server.properties

    Step 3: Prepare input topic and start Kafka producer

    diff --git a/docs/upgrade.html b/docs/upgrade.html index 1be5e1836c771..b45e25c400cda 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -82,6 +82,9 @@

    Notable changes in 4
  • The function onNewBatch in org.apache.kafka.clients.producer.Partitioner class was removed.
  • +
  • The default properties files for KRaft mode are no longer stored in the separate config/kraft directory since Zookeeper has been removed. These files have been consolidated with other configuration files. + Now all configuration files are in config directory. +
  • Broker diff --git a/trogdor/README.md b/trogdor/README.md index a44da002eb30a..daebf2f50aa09 100644 --- a/trogdor/README.md +++ b/trogdor/README.md @@ -12,8 +12,8 @@ Running Kafka in Kraft mode: ``` KAFKA_CLUSTER_ID="$(./bin/kafka-storage.sh random-uuid)" -./bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/kraft/reconfig-server.properties -./bin/kafka-server-start.sh config/kraft/reconfig-server.properties &> /tmp/kafka.log & +./bin/kafka-storage.sh format --standalone -t $KAFKA_CLUSTER_ID -c config/server.properties +./bin/kafka-server-start.sh config/server.properties &> /tmp/kafka.log & ``` Then, we want to run a Trogdor Agent, plus a Trogdor Coordinator. From 084fcbd3275ffd22ec0e1841b54de87d986afe0c Mon Sep 17 00:00:00 2001 From: "Chung, Ming-Yen" Date: Wed, 22 Jan 2025 22:50:16 +0800 Subject: [PATCH 18/44] KAFKA-18599: Remove Optional wrapping for forwardingManager in ApiVersionManager (#18630) `forwardingManager` is always present now. Reviewers: Ismael Juma --- .../kafka/server/ApiVersionManager.scala | 6 +- .../scala/kafka/server/BrokerServer.scala | 2 +- .../kafka/server/ApiVersionManagerTest.scala | 61 ++++--------------- 3 files changed, 15 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/kafka/server/ApiVersionManager.scala b/core/src/main/scala/kafka/server/ApiVersionManager.scala index 972af0414e463..fd1c70e509fff 100644 --- a/core/src/main/scala/kafka/server/ApiVersionManager.scala +++ b/core/src/main/scala/kafka/server/ApiVersionManager.scala @@ -46,7 +46,7 @@ object ApiVersionManager { def apply( listenerType: ListenerType, config: KafkaConfig, - forwardingManager: Option[ForwardingManager], + forwardingManager: ForwardingManager, supportedFeatures: BrokerFeatures, metadataCache: MetadataCache, clientMetricsManager: Option[ClientMetricsManager] @@ -129,7 +129,7 @@ class SimpleApiVersionManager( */ class DefaultApiVersionManager( val listenerType: ListenerType, - forwardingManager: Option[ForwardingManager], + forwardingManager: ForwardingManager, brokerFeatures: BrokerFeatures, metadataCache: MetadataCache, val enableUnstableLastVersion: Boolean, @@ -143,7 +143,7 @@ class DefaultApiVersionManager( alterFeatureLevel0: Boolean ): ApiVersionsResponse = { val finalizedFeatures = metadataCache.features() - val controllerApiVersions = forwardingManager.flatMap(_.controllerApiVersions) + val controllerApiVersions = forwardingManager.controllerApiVersions val clientTelemetryEnabled = clientMetricsManager match { case Some(manager) => manager.isTelemetryReceiverConfigured case None => false diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 442e6a403d107..ace134773ae2b 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -253,7 +253,7 @@ class BrokerServer( val apiVersionManager = ApiVersionManager( ListenerType.BROKER, config, - Some(forwardingManager), + forwardingManager, brokerFeatures, metadataCache, Some(clientMetricsManager) diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala index fcfd8a05ae649..341c859bf32da 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala @@ -21,7 +21,7 @@ import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.server.BrokerFeatures import org.apache.kafka.server.common.KRaftVersion -import org.junit.jupiter.api.{Disabled, Test} +import org.junit.jupiter.api.Test import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.EnumSource @@ -36,9 +36,10 @@ class ApiVersionManagerTest { @ParameterizedTest @EnumSource(classOf[ListenerType]) def testApiScope(apiScope: ListenerType): Unit = { + val forwardingManager = Mockito.mock(classOf[ForwardingManager]) val versionManager = new DefaultApiVersionManager( listenerType = apiScope, - forwardingManager = None, + forwardingManager = forwardingManager, brokerFeatures = brokerFeatures, metadataCache = metadataCache, enableUnstableLastVersion = true @@ -54,9 +55,10 @@ class ApiVersionManagerTest { @ParameterizedTest @EnumSource(classOf[ListenerType]) def testDisabledApis(apiScope: ListenerType): Unit = { + val forwardingManager = Mockito.mock(classOf[ForwardingManager]) val versionManager = new DefaultApiVersionManager( listenerType = apiScope, - forwardingManager = None, + forwardingManager = forwardingManager, brokerFeatures = brokerFeatures, metadataCache = metadataCache, enableUnstableLastVersion = false @@ -85,7 +87,7 @@ class ApiVersionManagerTest { val versionManager = new DefaultApiVersionManager( listenerType = ListenerType.ZK_BROKER, - forwardingManager = Some(forwardingManager), + forwardingManager = forwardingManager, brokerFeatures = brokerFeatures, metadataCache = metadataCache, enableUnstableLastVersion = true @@ -103,59 +105,18 @@ class ApiVersionManagerTest { val forwardingManager = Mockito.mock(classOf[ForwardingManager]) Mockito.when(forwardingManager.controllerApiVersions).thenReturn(None) - for (forwardingManagerOpt <- Seq(Some(forwardingManager), None)) { - val versionManager = new DefaultApiVersionManager( - listenerType = ListenerType.BROKER, - forwardingManager = forwardingManagerOpt, - brokerFeatures = brokerFeatures, - metadataCache = metadataCache, - enableUnstableLastVersion = true - ) - assertFalse(versionManager.isApiEnabled(ApiKeys.ENVELOPE, ApiKeys.ENVELOPE.latestVersion)) - assertFalse(versionManager.enabledApis.contains(ApiKeys.ENVELOPE)) - - val apiVersionsResponse = versionManager.apiVersionResponse(throttleTimeMs = 0, false) - val envelopeVersion = apiVersionsResponse.data.apiKeys.find(ApiKeys.ENVELOPE.id) - assertNull(envelopeVersion) - } - } - - @Disabled("Enable after enable KIP-590 forwarding in KAFKA-12886") - @Test - def testEnvelopeEnabledWhenForwardingManagerPresent(): Unit = { - val forwardingManager = Mockito.mock(classOf[ForwardingManager]) - Mockito.when(forwardingManager.controllerApiVersions).thenReturn(None) - val versionManager = new DefaultApiVersionManager( - listenerType = ListenerType.ZK_BROKER, - forwardingManager = Some(forwardingManager), + listenerType = ListenerType.BROKER, + forwardingManager = forwardingManager, brokerFeatures = brokerFeatures, metadataCache = metadataCache, enableUnstableLastVersion = true ) - assertTrue(versionManager.isApiEnabled(ApiKeys.ENVELOPE, ApiKeys.ENVELOPE.latestVersion)) - assertTrue(versionManager.enabledApis.contains(ApiKeys.ENVELOPE)) + assertFalse(versionManager.isApiEnabled(ApiKeys.ENVELOPE, ApiKeys.ENVELOPE.latestVersion)) + assertFalse(versionManager.enabledApis.contains(ApiKeys.ENVELOPE)) val apiVersionsResponse = versionManager.apiVersionResponse(throttleTimeMs = 0, false) val envelopeVersion = apiVersionsResponse.data.apiKeys.find(ApiKeys.ENVELOPE.id) - assertNotNull(envelopeVersion) - assertEquals(ApiKeys.ENVELOPE.oldestVersion, envelopeVersion.minVersion) - assertEquals(ApiKeys.ENVELOPE.latestVersion, envelopeVersion.maxVersion) - } - - @Test - def testEnvelopeDisabledWhenForwardingManagerEmpty(): Unit = { - val versionManager = new DefaultApiVersionManager( - listenerType = ListenerType.ZK_BROKER, - forwardingManager = None, - brokerFeatures = brokerFeatures, - metadataCache = metadataCache, - enableUnstableLastVersion = true - ) - assertTrue(versionManager.isApiEnabled(ApiKeys.ENVELOPE, ApiKeys.ENVELOPE.latestVersion)) - assertTrue(versionManager.enabledApis.contains(ApiKeys.ENVELOPE)) - - val apiVersionsResponse = versionManager.apiVersionResponse(throttleTimeMs = 0, false) - assertNotNull(apiVersionsResponse.data.apiKeys.find(ApiKeys.ENVELOPE.id)) + assertNull(envelopeVersion) } } From 341e535942c378c72a0b1d7aa4eec4895ef15b17 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Wed, 22 Jan 2025 23:12:06 +0800 Subject: [PATCH 19/44] KAFKA-18519: Remove Json.scala, cleanup AclEntry.scala (#18614) Reviewers: Mickael Maison , Ismael Juma , Chia-Ping Tsai --- core/src/main/scala/kafka/utils/Json.scala | 92 -------- .../security/authorizer/AclEntryTest.scala | 48 ---- .../scala/unit/kafka/utils/JsonTest.scala | 134 ----------- .../unit/kafka/utils/json/JsonValueTest.scala | 212 ------------------ gradle/spotbugs-exclude.xml | 7 - .../kafka/jmh/acl/AuthorizerBenchmark.java | 39 ++-- .../kafka/security/authorizer/AclEntry.java | 122 +--------- 7 files changed, 17 insertions(+), 637 deletions(-) delete mode 100644 core/src/main/scala/kafka/utils/Json.scala delete mode 100644 core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala delete mode 100644 core/src/test/scala/unit/kafka/utils/JsonTest.scala delete mode 100644 core/src/test/scala/unit/kafka/utils/json/JsonValueTest.scala diff --git a/core/src/main/scala/kafka/utils/Json.scala b/core/src/main/scala/kafka/utils/Json.scala deleted file mode 100644 index 049941cd01d98..0000000000000 --- a/core/src/main/scala/kafka/utils/Json.scala +++ /dev/null @@ -1,92 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.utils - -import com.fasterxml.jackson.core.{JsonParseException, JsonProcessingException} -import com.fasterxml.jackson.databind.ObjectMapper -import com.fasterxml.jackson.databind.node.MissingNode -import kafka.utils.json.JsonValue - -import scala.reflect.ClassTag - -/** - * Provides methods for parsing JSON with Jackson and encoding to JSON with a simple and naive custom implementation. - */ -object Json { - - private val mapper = new ObjectMapper() - - /** - * Parse a JSON string into a JsonValue if possible. `None` is returned if `input` is not valid JSON. - */ - def parseFull(input: String): Option[JsonValue] = tryParseFull(input).toOption - - /** - * Parse a JSON string into either a generic type T, or a JsonProcessingException in the case of - * exception. - */ - def parseStringAs[T](input: String)(implicit tag: ClassTag[T]): Either[JsonProcessingException, T] = { - try Right(mapper.readValue(input, tag.runtimeClass).asInstanceOf[T]) - catch { case e: JsonProcessingException => Left(e) } - } - - /** - * Parse a JSON byte array into a JsonValue if possible. `None` is returned if `input` is not valid JSON. - */ - def parseBytes(input: Array[Byte]): Option[JsonValue] = - try Option(mapper.readTree(input)).map(JsonValue(_)) - catch { case _: JsonProcessingException => None } - - def tryParseBytes(input: Array[Byte]): Either[JsonProcessingException, JsonValue] = - try Right(mapper.readTree(input)).map(JsonValue(_)) - catch { case e: JsonProcessingException => Left(e) } - - /** - * Parse a JSON byte array into either a generic type T, or a JsonProcessingException in the case of exception. - */ - def parseBytesAs[T](input: Array[Byte])(implicit tag: ClassTag[T]): Either[JsonProcessingException, T] = { - try Right(mapper.readValue(input, tag.runtimeClass).asInstanceOf[T]) - catch { case e: JsonProcessingException => Left(e) } - } - - /** - * Parse a JSON string into a JsonValue if possible. It returns an `Either` where `Left` will be an exception and - * `Right` is the `JsonValue`. - * @param input a JSON string to parse - * @return An `Either` which in case of `Left` means an exception and `Right` is the actual return value. - */ - def tryParseFull(input: String): Either[JsonProcessingException, JsonValue] = - if (input == null || input.isEmpty) - Left(new JsonParseException(MissingNode.getInstance().traverse(), "The input string shouldn't be empty")) - else - try Right(mapper.readTree(input)).map(JsonValue(_)) - catch { case e: JsonProcessingException => Left(e) } - - /** - * Encode an object into a JSON string. This method accepts any type supported by Jackson's ObjectMapper in - * the default configuration. That is, Java collections are supported, but Scala collections are not (to avoid - * a jackson-scala dependency). - */ - def encodeAsString(obj: Any): String = mapper.writeValueAsString(obj) - - /** - * Encode an object into a JSON value in bytes. This method accepts any type supported by Jackson's ObjectMapper in - * the default configuration. That is, Java collections are supported, but Scala collections are not (to avoid - * a jackson-scala dependency). - */ - def encodeAsBytes(obj: Any): Array[Byte] = mapper.writeValueAsBytes(obj) -} diff --git a/core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala b/core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala deleted file mode 100644 index f6867b6233b89..0000000000000 --- a/core/src/test/scala/unit/kafka/security/authorizer/AclEntryTest.scala +++ /dev/null @@ -1,48 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.security.authorizer - -import java.nio.charset.StandardCharsets.UTF_8 -import kafka.utils.Json -import org.apache.kafka.common.acl.AccessControlEntry -import org.apache.kafka.common.acl.AclOperation.READ -import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} -import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.apache.kafka.security.authorizer.AclEntry -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test - -import java.util - -class AclEntryTest { - - val AclJson = """{"version": 1, "acls": [{"host": "host1","permissionType": "Deny","operation": "READ", "principal": "User:alice" }, - { "host": "*" , "permissionType": "Allow", "operation": "Read", "principal": "User:bob" }, - { "host": "host1", "permissionType": "Deny", "operation": "Read" , "principal": "User:bob"}]}""" - - @Test - def testAclJsonConversion(): Unit = { - val acl1 = new AclEntry(new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice").toString, "host1", READ, DENY)) - val acl2 = new AclEntry(new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob").toString, "*", READ, ALLOW)) - val acl3 = new AclEntry(new AccessControlEntry(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob").toString, "host1", READ, DENY)) - - val acls = new util.HashSet[AclEntry](util.Arrays.asList(acl1, acl2, acl3)) - - assertEquals(acls, AclEntry.fromBytes(Json.encodeAsBytes(AclEntry.toJsonCompatibleMap(acls)))) - assertEquals(acls, AclEntry.fromBytes(AclJson.getBytes(UTF_8))) - } -} diff --git a/core/src/test/scala/unit/kafka/utils/JsonTest.scala b/core/src/test/scala/unit/kafka/utils/JsonTest.scala deleted file mode 100644 index aca7d45600d12..0000000000000 --- a/core/src/test/scala/unit/kafka/utils/JsonTest.scala +++ /dev/null @@ -1,134 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package kafka.utils - -import java.nio.charset.StandardCharsets - -import com.fasterxml.jackson.annotation.JsonProperty -import com.fasterxml.jackson.core.{JsonParseException, JsonProcessingException} -import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node._ -import kafka.utils.JsonTest.TestObject -import kafka.utils.json.JsonValue -import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Test - -import scala.jdk.CollectionConverters._ -import scala.collection.Map - -object JsonTest { - case class TestObject(@JsonProperty("foo") foo: String, @JsonProperty("bar") bar: Int) -} - -class JsonTest { - - @Test - def testJsonParse(): Unit = { - val jnf = JsonNodeFactory.instance - - assertEquals(Some(JsonValue(new ObjectNode(jnf))), Json.parseFull("{}")) - assertEquals(Right(JsonValue(new ObjectNode(jnf))), Json.tryParseFull("{}")) - assertEquals(classOf[Left[JsonProcessingException, JsonValue]], Json.tryParseFull(null).getClass) - assertThrows(classOf[IllegalArgumentException], () => Json.tryParseBytes(null)) - - assertEquals(None, Json.parseFull("")) - assertEquals(classOf[Left[JsonProcessingException, JsonValue]], Json.tryParseFull("").getClass) - - assertEquals(None, Json.parseFull("""{"foo":"bar"s}""")) - val tryRes = Json.tryParseFull("""{"foo":"bar"s}""") - assertTrue(tryRes.isInstanceOf[Left[_, JsonValue]]) - - val objectNode = new ObjectNode( - jnf, - Map[String, JsonNode]("foo" -> new TextNode("bar"), "is_enabled" -> BooleanNode.TRUE).asJava - ) - assertEquals(Some(JsonValue(objectNode)), Json.parseFull("""{"foo":"bar", "is_enabled":true}""")) - assertEquals(Right(JsonValue(objectNode)), Json.tryParseFull("""{"foo":"bar", "is_enabled":true}""")) - - val arrayNode = new ArrayNode(jnf) - Vector(1, 2, 3).map(new IntNode(_)).foreach(arrayNode.add) - assertEquals(Some(JsonValue(arrayNode)), Json.parseFull("[1, 2, 3]")) - - // Test with encoder that properly escapes backslash and quotes - val map = Map("foo1" -> """bar1\,bar2""", "foo2" -> """\bar""").asJava - val encoded = Json.encodeAsString(map) - val decoded = Json.parseFull(encoded) - assertEquals(decoded, Json.parseFull("""{"foo1":"bar1\\,bar2", "foo2":"\\bar"}""")) - } - - @Test - def testEncodeAsString(): Unit = { - assertEquals("null", Json.encodeAsString(null)) - assertEquals("1", Json.encodeAsString(1)) - assertEquals("1", Json.encodeAsString(1L)) - assertEquals("1", Json.encodeAsString(1.toByte)) - assertEquals("1", Json.encodeAsString(1.toShort)) - assertEquals("1.0", Json.encodeAsString(1.0)) - assertEquals(""""str"""", Json.encodeAsString("str")) - assertEquals("true", Json.encodeAsString(true)) - assertEquals("false", Json.encodeAsString(false)) - assertEquals("[]", Json.encodeAsString(Seq().asJava)) - assertEquals("[null]", Json.encodeAsString(Seq(null).asJava)) - assertEquals("[1,2,3]", Json.encodeAsString(Seq(1,2,3).asJava)) - assertEquals("""[1,"2",[3],null]""", Json.encodeAsString(Seq(1,"2",Seq(3).asJava,null).asJava)) - assertEquals("{}", Json.encodeAsString(Map().asJava)) - assertEquals("""{"a":1,"b":2,"c":null}""", Json.encodeAsString(Map("a" -> 1, "b" -> 2, "c" -> null).asJava)) - assertEquals("""{"a":[1,2],"c":[3,4]}""", Json.encodeAsString(Map("a" -> Seq(1,2).asJava, "c" -> Seq(3,4).asJava).asJava)) - assertEquals("""{"a":[1,2],"b":[3,4],"c":null}""", Json.encodeAsString(Map("a" -> Seq(1,2).asJava, "b" -> Seq(3,4).asJava, "c" -> null).asJava)) - assertEquals(""""str1\\,str2"""", Json.encodeAsString("""str1\,str2""")) - assertEquals(""""\"quoted\""""", Json.encodeAsString(""""quoted"""")) - } - - @Test - def testEncodeAsBytes(): Unit = { - assertEquals("null", new String(Json.encodeAsBytes(null), StandardCharsets.UTF_8)) - assertEquals("1", new String(Json.encodeAsBytes(1), StandardCharsets.UTF_8)) - assertEquals("1", new String(Json.encodeAsBytes(1L), StandardCharsets.UTF_8)) - assertEquals("1", new String(Json.encodeAsBytes(1.toByte), StandardCharsets.UTF_8)) - assertEquals("1", new String(Json.encodeAsBytes(1.toShort), StandardCharsets.UTF_8)) - assertEquals("1.0", new String(Json.encodeAsBytes(1.0), StandardCharsets.UTF_8)) - assertEquals(""""str"""", new String(Json.encodeAsBytes("str"), StandardCharsets.UTF_8)) - assertEquals("true", new String(Json.encodeAsBytes(true), StandardCharsets.UTF_8)) - assertEquals("false", new String(Json.encodeAsBytes(false), StandardCharsets.UTF_8)) - assertEquals("[]", new String(Json.encodeAsBytes(Seq().asJava), StandardCharsets.UTF_8)) - assertEquals("[null]", new String(Json.encodeAsBytes(Seq(null).asJava), StandardCharsets.UTF_8)) - assertEquals("[1,2,3]", new String(Json.encodeAsBytes(Seq(1,2,3).asJava), StandardCharsets.UTF_8)) - assertEquals("""[1,"2",[3],null]""", new String(Json.encodeAsBytes(Seq(1,"2",Seq(3).asJava,null).asJava), StandardCharsets.UTF_8)) - assertEquals("{}", new String(Json.encodeAsBytes(Map().asJava), StandardCharsets.UTF_8)) - assertEquals("""{"a":1,"b":2,"c":null}""", new String(Json.encodeAsBytes(Map("a" -> 1, "b" -> 2, "c" -> null).asJava), StandardCharsets.UTF_8)) - assertEquals("""{"a":[1,2],"c":[3,4]}""", new String(Json.encodeAsBytes(Map("a" -> Seq(1,2).asJava, "c" -> Seq(3,4).asJava).asJava), StandardCharsets.UTF_8)) - assertEquals("""{"a":[1,2],"b":[3,4],"c":null}""", new String(Json.encodeAsBytes(Map("a" -> Seq(1,2).asJava, "b" -> Seq(3,4).asJava, "c" -> null).asJava), StandardCharsets.UTF_8)) - assertEquals(""""str1\\,str2"""", new String(Json.encodeAsBytes("""str1\,str2"""), StandardCharsets.UTF_8)) - assertEquals(""""\"quoted\""""", new String(Json.encodeAsBytes(""""quoted""""), StandardCharsets.UTF_8)) - } - - @Test - def testParseTo(): Unit = { - val foo = "baz" - val bar = 1 - - val result = Json.parseStringAs[TestObject](s"""{"foo": "$foo", "bar": $bar}""") - - assertEquals(Right(TestObject(foo, bar)), result) - } - - @Test - def testParseToWithInvalidJson(): Unit = { - val result = Json.parseStringAs[TestObject]("{invalid json}") - assertEquals(Left(classOf[JsonParseException]), result.left.map(_.getClass)) - } -} diff --git a/core/src/test/scala/unit/kafka/utils/json/JsonValueTest.scala b/core/src/test/scala/unit/kafka/utils/json/JsonValueTest.scala deleted file mode 100644 index 8194b298b4e5a..0000000000000 --- a/core/src/test/scala/unit/kafka/utils/json/JsonValueTest.scala +++ /dev/null @@ -1,212 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.utils.json - -import scala.collection.Seq - -import com.fasterxml.jackson.databind.{ObjectMapper, JsonMappingException} -import org.junit.jupiter.api.Test -import org.junit.jupiter.api.Assertions._ - -import kafka.utils.Json - -class JsonValueTest { - - private val json = """ - |{ - | "boolean": false, - | "int": 1234, - | "long": 3000000000, - | "double": 16.244355, - | "string": "string", - | "number_as_string": "123", - | "array": [4.0, 11.1, 44.5], - | "object": { - | "a": true, - | "b": false - | }, - | "null": null - |} - """.stripMargin - - private def parse(s: String): JsonValue = - Json.parseFull(s).getOrElse(sys.error("Failed to parse json: " + s)) - - private def assertTo[T: DecodeJson](expected: T, jsonValue: JsonObject => JsonValue): Unit = { - val parsed = jsonValue(parse(json).asJsonObject) - assertEquals(Right(expected), parsed.toEither[T]) - assertEquals(expected, parsed.to[T]) - } - - private def assertToFails[T: DecodeJson](jsonValue: JsonObject => JsonValue): Unit = { - val parsed = jsonValue(parse(json).asJsonObject) - assertTrue(parsed.toEither[T].isLeft) - assertThrow[JsonMappingException](parsed.to[T]) - } - - def assertThrow[E <: Throwable : Manifest](body: => Unit): Unit = { - import scala.util.control.Exception._ - val klass = manifest[E].runtimeClass - catchingPromiscuously(klass).opt(body).foreach { _ => - fail("Expected `" + klass + "` to be thrown, but no exception was thrown") - } - } - - @Test - def testAsJsonObject(): Unit = { - val parsed = parse(json).asJsonObject - val obj = parsed("object") - assertEquals(obj, obj.asJsonObject) - assertThrow[JsonMappingException](parsed("array").asJsonObject) - } - - @Test - def testAsJsonObjectOption(): Unit = { - val parsed = parse(json).asJsonObject - assertTrue(parsed("object").asJsonObjectOption.isDefined) - assertEquals(None, parsed("array").asJsonObjectOption) - } - - @Test - def testAsJsonArray(): Unit = { - val parsed = parse(json).asJsonObject - val array = parsed("array") - assertEquals(array, array.asJsonArray) - assertThrow[JsonMappingException](parsed("object").asJsonArray) - } - - @Test - def testAsJsonArrayOption(): Unit = { - val parsed = parse(json).asJsonObject - assertTrue(parsed("array").asJsonArrayOption.isDefined) - assertEquals(None, parsed("object").asJsonArrayOption) - } - - @Test - def testJsonObjectGet(): Unit = { - val parsed = parse(json).asJsonObject - assertEquals(Some(parse("""{"a":true,"b":false}""")), parsed.get("object")) - assertEquals(None, parsed.get("aaaaa")) - } - - @Test - def testJsonObjectApply(): Unit = { - val parsed = parse(json).asJsonObject - assertEquals(parse("""{"a":true,"b":false}"""), parsed("object")) - assertThrow[JsonMappingException](parsed("aaaaaaaa")) - } - - @Test - def testJsonObjectIterator(): Unit = { - assertEquals( - Vector("a" -> parse("true"), "b" -> parse("false")), - parse(json).asJsonObject("object").asJsonObject.iterator.toVector - ) - } - - @Test - def testJsonArrayIterator(): Unit = { - assertEquals(Vector("4.0", "11.1", "44.5").map(parse), parse(json).asJsonObject("array").asJsonArray.iterator.toVector) - } - - @Test - def testJsonValueEquals(): Unit = { - - assertEquals(parse(json), parse(json)) - - assertEquals(parse("""{"blue": true, "red": false}"""), parse("""{"red": false, "blue": true}""")) - assertNotEquals(parse("""{"blue": true, "red": true}"""), parse("""{"red": false, "blue": true}""")) - - assertEquals(parse("""[1, 2, 3]"""), parse("""[1, 2, 3]""")) - assertNotEquals(parse("""[1, 2, 3]"""), parse("""[2, 1, 3]""")) - - assertEquals(parse("1344"), parse("1344")) - assertNotEquals(parse("1344"), parse("144")) - - } - - @Test - def testJsonValueHashCode(): Unit = { - assertEquals(new ObjectMapper().readTree(json).hashCode, parse(json).hashCode) - } - - @Test - def testJsonValueToString(): Unit = { - val js = """{"boolean":false,"int":1234,"array":[4.0,11.1,44.5],"object":{"a":true,"b":false}}""" - assertEquals(js, parse(js).toString) - } - - @Test - def testDecodeBoolean(): Unit = { - assertTo[Boolean](false, _("boolean")) - assertToFails[Boolean](_("int")) - } - - @Test - def testDecodeString(): Unit = { - assertTo[String]("string", _("string")) - assertTo[String]("123", _("number_as_string")) - assertToFails[String](_("int")) - assertToFails[String](_("array")) - } - - @Test - def testDecodeInt(): Unit = { - assertTo[Int](1234, _("int")) - assertToFails[Int](_("long")) - } - - @Test - def testDecodeLong(): Unit = { - assertTo[Long](3000000000L, _("long")) - assertTo[Long](1234, _("int")) - assertToFails[Long](_("string")) - } - - @Test - def testDecodeDouble(): Unit = { - assertTo[Double](16.244355, _("double")) - assertTo[Double](1234.0, _("int")) - assertTo[Double](3000000000L, _("long")) - assertToFails[Double](_("string")) - } - - @Test - def testDecodeSeq(): Unit = { - assertTo[Seq[Double]](Seq(4.0, 11.1, 44.5), _("array")) - assertToFails[Seq[Double]](_("string")) - assertToFails[Seq[Double]](_("object")) - assertToFails[Seq[String]](_("array")) - } - - @Test - def testDecodeMap(): Unit = { - assertTo[Map[String, Boolean]](Map("a" -> true, "b" -> false), _("object")) - assertToFails[Map[String, Int]](_("object")) - assertToFails[Map[String, String]](_("object")) - assertToFails[Map[String, Double]](_("array")) - } - - @Test - def testDecodeOption(): Unit = { - assertTo[Option[Int]](None, _("null")) - assertTo[Option[Int]](Some(1234), _("int")) - assertToFails[Option[String]](_("int")) - } - -} diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index ddd0c398b5bac..3745dd8e4f848 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -151,13 +151,6 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read - - - - - - - diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java index 051d4d7724fec..d2fb9018df219 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/acl/AuthorizerBenchmark.java @@ -35,7 +35,6 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.metadata.authorizer.StandardAcl; import org.apache.kafka.metadata.authorizer.StandardAuthorizer; -import org.apache.kafka.security.authorizer.AclEntry; import org.apache.kafka.server.authorizer.Action; import org.openjdk.jmh.annotations.Benchmark; @@ -113,14 +112,14 @@ public void setup() throws Exception { } private void prepareAclCache() { - Map> aclEntries = new HashMap<>(); + Map> aclEntries = new HashMap<>(); for (int resourceId = 0; resourceId < resourceCount; resourceId++) { ResourcePattern resource = new ResourcePattern( (resourceId % 10 == 0) ? ResourceType.GROUP : ResourceType.TOPIC, resourceNamePrefix + resourceId, (resourceId % 5 == 0) ? PatternType.PREFIXED : PatternType.LITERAL); - Set entries = aclEntries.computeIfAbsent(resource, k -> new HashSet<>()); + Set entries = aclEntries.computeIfAbsent(resource, k -> new HashSet<>()); for (int aclId = 0; aclId < aclCount; aclId++) { // The principal in the request context we are using @@ -129,36 +128,31 @@ private void prepareAclCache() { AccessControlEntry allowAce = new AccessControlEntry( principalName, "*", AclOperation.READ, AclPermissionType.ALLOW); - entries.add(new AclEntry(allowAce)); + entries.add(new AccessControlEntry(allowAce.principal(), allowAce.host(), allowAce.operation(), allowAce.permissionType())); if (shouldDeny()) { - // dominantly deny the resource - AccessControlEntry denyAce = new AccessControlEntry( - principalName, "*", AclOperation.READ, AclPermissionType.DENY); - entries.add(new AclEntry(denyAce)); + entries.add(new AccessControlEntry(principalName, "*", AclOperation.READ, AclPermissionType.DENY)); } } } ResourcePattern resourcePrefix = new ResourcePattern(ResourceType.TOPIC, resourceNamePrefix, PatternType.PREFIXED); - Set entriesPrefix = aclEntries.computeIfAbsent(resourcePrefix, k -> new HashSet<>()); + Set entriesPrefix = aclEntries.computeIfAbsent(resourcePrefix, k -> new HashSet<>()); for (int hostId = 0; hostId < hostPreCount; hostId++) { AccessControlEntry allowAce = new AccessControlEntry(principal.toString(), "127.0.0." + hostId, AclOperation.READ, AclPermissionType.ALLOW); - entriesPrefix.add(new AclEntry(allowAce)); + entriesPrefix.add(new AccessControlEntry(allowAce.principal(), allowAce.host(), allowAce.operation(), allowAce.permissionType())); if (shouldDeny()) { - // dominantly deny the resource - AccessControlEntry denyAce = new AccessControlEntry(principal.toString(), "127.0.0." + hostId, - AclOperation.READ, AclPermissionType.DENY); - entriesPrefix.add(new AclEntry(denyAce)); + entriesPrefix.add(new AccessControlEntry(principal.toString(), "127.0.0." + hostId, + AclOperation.READ, AclPermissionType.DENY)); } } ResourcePattern resourceWildcard = new ResourcePattern(ResourceType.TOPIC, ResourcePattern.WILDCARD_RESOURCE, PatternType.LITERAL); - Set entriesWildcard = aclEntries.computeIfAbsent(resourceWildcard, k -> new HashSet<>()); + Set entriesWildcard = aclEntries.computeIfAbsent(resourceWildcard, k -> new HashSet<>()); // get dynamic entries number for wildcard acl for (int hostId = 0; hostId < resourceCount / 10; hostId++) { String hostName = "127.0.0" + hostId; @@ -170,23 +164,22 @@ private void prepareAclCache() { AccessControlEntry allowAce = new AccessControlEntry(principal.toString(), hostName, AclOperation.READ, AclPermissionType.ALLOW); - entriesWildcard.add(new AclEntry(allowAce)); + entriesWildcard.add(new AccessControlEntry(allowAce.principal(), allowAce.host(), allowAce.operation(), allowAce.permissionType())); if (shouldDeny()) { - AccessControlEntry denyAce = new AccessControlEntry(principal.toString(), hostName, - AclOperation.READ, AclPermissionType.DENY); - entriesWildcard.add(new AclEntry(denyAce)); + entriesWildcard.add(new AccessControlEntry(principal.toString(), hostName, + AclOperation.READ, AclPermissionType.DENY)); } } setupAcls(aclEntries); } - private void setupAcls(Map> aclEntries) { - for (Map.Entry> entryMap : aclEntries.entrySet()) { + private void setupAcls(Map> aclEntries) { + for (Map.Entry> entryMap : aclEntries.entrySet()) { ResourcePattern resourcePattern = entryMap.getKey(); - for (AclEntry aclEntry : entryMap.getValue()) { - StandardAcl standardAcl = StandardAcl.fromAclBinding(new AclBinding(resourcePattern, aclEntry)); + for (AccessControlEntry accessControlEntry : entryMap.getValue()) { + StandardAcl standardAcl = StandardAcl.fromAclBinding(new AclBinding(resourcePattern, accessControlEntry)); authorizer.addAcl(Uuid.randomUuid(), standardAcl); } authorizer.completeInitialLoad(); diff --git a/server/src/main/java/org/apache/kafka/security/authorizer/AclEntry.java b/server/src/main/java/org/apache/kafka/security/authorizer/AclEntry.java index 012be7f55a95d..4b1b12aefdcff 100644 --- a/server/src/main/java/org/apache/kafka/security/authorizer/AclEntry.java +++ b/server/src/main/java/org/apache/kafka/security/authorizer/AclEntry.java @@ -16,28 +16,15 @@ */ package org.apache.kafka.security.authorizer; -import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.acl.AclOperation; -import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.security.auth.KafkaPrincipal; -import org.apache.kafka.common.utils.SecurityUtils; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.server.util.Json; -import org.apache.kafka.server.util.json.DecodeJson; -import org.apache.kafka.server.util.json.JsonObject; -import org.apache.kafka.server.util.json.JsonValue; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; -import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -54,99 +41,16 @@ import static org.apache.kafka.common.acl.AclOperation.READ; import static org.apache.kafka.common.acl.AclOperation.WRITE; -public class AclEntry extends AccessControlEntry { - private static final DecodeJson.DecodeInteger INT = new DecodeJson.DecodeInteger(); - private static final DecodeJson.DecodeString STRING = new DecodeJson.DecodeString(); +public class AclEntry { public static final KafkaPrincipal WILDCARD_PRINCIPAL = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "*"); public static final String WILDCARD_PRINCIPAL_STRING = WILDCARD_PRINCIPAL.toString(); public static final String WILDCARD_HOST = "*"; public static final String WILDCARD_RESOURCE = ResourcePattern.WILDCARD_RESOURCE; - public static final String RESOURCE_SEPARATOR = ":"; - public static final Set RESOURCE_TYPES = Arrays.stream(ResourceType.values()) - .filter(t -> !(t == ResourceType.UNKNOWN || t == ResourceType.ANY)) - .collect(Collectors.toSet()); public static final Set ACL_OPERATIONS = Arrays.stream(AclOperation.values()) .filter(t -> !(t == AclOperation.UNKNOWN || t == AclOperation.ANY)) .collect(Collectors.toSet()); - private static final String PRINCIPAL_KEY = "principal"; - private static final String PERMISSION_TYPE_KEY = "permissionType"; - private static final String OPERATION_KEY = "operation"; - private static final String HOSTS_KEY = "host"; - public static final String VERSION_KEY = "version"; - public static final int CURRENT_VERSION = 1; - private static final String ACLS_KEY = "acls"; - - public final AccessControlEntry ace; - public final KafkaPrincipal kafkaPrincipal; - - public AclEntry(AccessControlEntry ace) { - super(ace.principal(), ace.host(), ace.operation(), ace.permissionType()); - this.ace = ace; - - kafkaPrincipal = ace.principal() == null - ? null - : SecurityUtils.parseKafkaPrincipal(ace.principal()); - } - - /** - * Parse JSON representation of ACLs - * @param bytes of acls json string - * - *

    - { - "version": 1, - "acls": [ - { - "host":"host1", - "permissionType": "Deny", - "operation": "Read", - "principal": "User:alice" - } - ] - } - *

    - * - * @return set of AclEntry objects from the JSON string - */ - public static Set fromBytes(byte[] bytes) throws IOException { - if (bytes == null || bytes.length == 0) - return Collections.emptySet(); - - Optional jsonValue = Json.parseBytes(bytes); - if (jsonValue.isEmpty()) - return Collections.emptySet(); - - JsonObject js = jsonValue.get().asJsonObject(); - - //the acl json version. - Utils.require(js.apply(VERSION_KEY).to(INT) == CURRENT_VERSION); - - Set res = new HashSet<>(); - - Iterator aclsIter = js.apply(ACLS_KEY).asJsonArray().iterator(); - while (aclsIter.hasNext()) { - JsonObject itemJs = aclsIter.next().asJsonObject(); - KafkaPrincipal principal = SecurityUtils.parseKafkaPrincipal(itemJs.apply(PRINCIPAL_KEY).to(STRING)); - AclPermissionType permissionType = SecurityUtils.permissionType(itemJs.apply(PERMISSION_TYPE_KEY).to(STRING)); - String host = itemJs.apply(HOSTS_KEY).to(STRING); - AclOperation operation = SecurityUtils.operation(itemJs.apply(OPERATION_KEY).to(STRING)); - - res.add(new AclEntry(new AccessControlEntry(principal.toString(), - host, operation, permissionType))); - } - - return res; - } - - public static Map toJsonCompatibleMap(Set acls) { - Map res = new HashMap<>(); - res.put(AclEntry.VERSION_KEY, AclEntry.CURRENT_VERSION); - res.put(AclEntry.ACLS_KEY, acls.stream().map(AclEntry::toMap).collect(Collectors.toList())); - return res; - } - public static Set supportedOperations(ResourceType resourceType) { switch (resourceType) { case TOPIC: @@ -182,28 +86,4 @@ public static Errors authorizationError(ResourceType resourceType) { throw new IllegalArgumentException("Authorization error type not known"); } } - - public Map toMap() { - Map res = new HashMap<>(); - res.put(AclEntry.PRINCIPAL_KEY, principal()); - res.put(AclEntry.PERMISSION_TYPE_KEY, SecurityUtils.permissionTypeName(permissionType())); - res.put(AclEntry.OPERATION_KEY, SecurityUtils.operationName(operation())); - res.put(AclEntry.HOSTS_KEY, host()); - return res; - } - - @Override - public int hashCode() { - return ace.hashCode(); - } - - @Override - public boolean equals(Object o) { - return super.equals(o); // to keep spotbugs happy - } - - @Override - public String toString() { - return String.format("%s has %s permission for operations: %s from hosts: %s", principal(), permissionType().name(), operation(), host()); - } } From f4d90398cc3f839b4fd6da30e7a58d12c71d3047 Mon Sep 17 00:00:00 2001 From: Xiaobing Fang Date: Thu, 23 Jan 2025 00:50:39 +0800 Subject: [PATCH 20/44] MINOR: Fix `LogCleanerManagerTest.testLogsUnderCleanupIneligibleForCompaction()` for `LogMessageTimestampType = "LogAppendTime"` (#12333) While setting Defaults.LogMessageTimestampType to "LogAppendTime", `LogCleanerManagerTest.testLogsUnderCleanupIneligibleForCompaction()` fails with a InvalidTimestampException. This PR fixes this by regenerating the records instead of previous approach of re-using same records in the test. Reviewers: Divij Vaidya , Kvicii --------- Co-authored-by: fangxiaobing --- core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 2315acc3fb31c..155b96f2a3aa5 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -358,7 +358,7 @@ class LogCleanerManagerTest extends Logging { log.appendAsLeader(records, leaderEpoch = 0) log.roll() - log.appendAsLeader(records, leaderEpoch = 0) + log.appendAsLeader(TestUtils.singletonRecords("test2".getBytes, key="test2".getBytes), leaderEpoch = 0) log.updateHighWatermark(2L) // simulate cleanup thread working on the log partition From 410065a65df2716e563429577dc2d3bcaa0c7c9a Mon Sep 17 00:00:00 2001 From: Lianet Magrans <98415067+lianetm@users.noreply.github.com> Date: Wed, 22 Jan 2025 18:02:38 +0100 Subject: [PATCH 21/44] KAFKA-18517: Enable ConsumerBounceTest to run for new async consumer (#18532) Reviewers: Andrew Schofield , Kirk True --- .../kafka/api/ConsumerBounceTest.scala | 45 +++++++++++++------ 1 file changed, 31 insertions(+), 14 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index a8dbe0ecdaa5d..3b257e3f7a50a 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.util.ShutdownableThread import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, Disabled, Test, TestInfo} +import org.junit.jupiter.api.{AfterEach, Disabled, TestInfo} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.MethodSource @@ -59,7 +59,12 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG -> "1", GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG -> "10", // set small enough session timeout GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG -> "0", + + // Tests will run for CONSUMER and CLASSIC group protocol, so set the group max size property + // required for each. + GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG -> maxGroupSize.toString, GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG -> maxGroupSize.toString, + ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG -> "false", ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG -> "true", ReplicationConfigs.UNCLEAN_LEADER_ELECTION_INTERVAL_MS_CONFIG -> "50", @@ -94,7 +99,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testConsumptionWithBrokerFailures(quorum: String, groupProtocol: String): Unit = consumeWithBrokerFailures(10) /* @@ -139,7 +144,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testSeekAndCommitWithBrokerFailures(quorum: String, groupProtocol: String): Unit = seekAndCommitWithBrokerFailures(5) def seekAndCommitWithBrokerFailures(numIters: Int): Unit = { @@ -183,7 +188,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { } @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testSubscribeWhenTopicUnavailable(quorum: String, groupProtocol: String): Unit = { val numRecords = 1000 val newtopic = "newtopic" @@ -243,7 +248,7 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { checkCloseGoodPath(numRecords, "group1") checkCloseWithCoordinatorFailure(numRecords, "group2", "group3") - checkCloseWithClusterFailure(numRecords, "group4", "group5") + checkCloseWithClusterFailure(numRecords, "group4", "group5", groupProtocol) } /** @@ -297,12 +302,15 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { * there is no coordinator, but close should timeout and return. If close is invoked with a very * large timeout, close should timeout after request timeout. */ - private def checkCloseWithClusterFailure(numRecords: Int, group1: String, group2: String): Unit = { + private def checkCloseWithClusterFailure(numRecords: Int, group1: String, group2: String, + groupProtocol: String): Unit = { val consumer1 = createConsumerAndReceive(group1, manualAssign = false, numRecords) val requestTimeout = 6000 - this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "5000") - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + if (groupProtocol.equals(GroupProtocol.CLASSIC.name)) { + this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "5000") + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + } this.consumerConfig.setProperty(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) val consumer2 = createConsumerAndReceive(group2, manualAssign = true, numRecords) @@ -319,9 +327,10 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { * the group should be forced to rebalance when it becomes hosted on a Coordinator with the new config. * Then, 1 consumer should be left out of the group. */ - @Test + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) @Disabled // TODO: To be re-enabled once we can make it less flaky (KAFKA-13421) - def testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup(): Unit = { + def testRollingBrokerRestartsWithSmallerMaxGroupSizeConfigDisruptsBigGroup(quorum: String, groupProtocol: String): Unit = { val group = "group-max-size-test" val topic = "group-max-size-test" val maxGroupSize = 2 @@ -329,7 +338,9 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { val partitionCount = consumerCount * 2 this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "60000") - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + if (groupProtocol.equals(GroupProtocol.CLASSIC.name)) { + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + } this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") val partitions = createTopicPartitions(topic, numPartitions = partitionCount, replicationFactor = brokerCount) @@ -361,12 +372,14 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { * When we have the consumer group max size configured to X, the X+1th consumer trying to join should receive a fatal exception */ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) def testConsumerReceivesFatalExceptionWhenGroupPassesMaxSize(quorum: String, groupProtocol: String): Unit = { val group = "fatal-exception-test" val topic = "fatal-exception-test" this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "60000") - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + if (groupProtocol.equals(GroupProtocol.CLASSIC.name)) { + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + } this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") val partitions = createTopicPartitions(topic, numPartitions = maxGroupSize, replicationFactor = brokerCount) @@ -401,11 +414,15 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging { */ @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly")) + // TODO: enable for all protocols after fix for not generating/blocking on unneeded + // FindCoordinator on close for the new consumer def testCloseDuringRebalance(quorum: String, groupProtocol: String): Unit = { val topic = "closetest" createTopic(topic, 10, brokerCount) this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "60000") - this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + if (groupProtocol.equals(GroupProtocol.CLASSIC.name)) { + this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "1000") + } this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") checkCloseDuringRebalance("group1", topic, executor, brokersAvailableDuringClose = true) } From 94a1bfb1281f06263976b1ba8bba8c5ac5d7f2ce Mon Sep 17 00:00:00 2001 From: Justine Olshan Date: Wed, 22 Jan 2025 13:44:08 -0800 Subject: [PATCH 22/44] KAFKA-18575: Transaction Version 2 doesn't correctly handle race condition with completing and new transaction(#18604) There is a subtle race condition with transactions V2 if a transaction is still completing when checking if we need to add a partition, but it completes when the request reaches the coordinator. One approach was to remove the verification for TV2 and just check the epoch on write, but a simpler one is to simply return concurrent transactions from the partition leader (before attempting to add the partition). I've done this and added a test for this behavior. Locally, I reproduced the race but adding a 1 second sleep when handling the WriteTxnMarkersRequest and a 2 second delay before adding the partition to the AddPartitionsToTxnManager. Without this change, the race happened on every second transaction as the first one completed. With this change, the error went away. As a followup, we may want to clean up some of the code and comments with respect to verification as the code is used by both TV0 + verification and TV2. But that doesn't need to complete for 4.0. This does :) Reviewers: Jeff Kim , Artem Livshits , Calvin Liu --- .../src/main/scala/kafka/log/UnifiedLog.scala | 9 +++-- .../scala/unit/kafka/log/UnifiedLogTest.scala | 36 +++++++++++++++++++ 2 files changed, 43 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index a3267e5ec8c50..3253de11dfec2 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -600,6 +600,11 @@ class UnifiedLog(@volatile var logStartOffset: Long, */ def hasOngoingTransaction(producerId: Long, producerEpoch: Short): Boolean = lock synchronized { val entry = producerStateManager.activeProducers.get(producerId) + // With transactions V2, if we see a future epoch, we are likely in the process of completing the previous transaction. + // Return early with ConcurrentTransactionsException until the transaction completes. + if (entry != null && entry.currentTxnFirstOffset.isPresent && entry.producerEpoch() < producerEpoch) + throw new ConcurrentTransactionsException("The producer attempted to update a transaction " + + "while another concurrent operation on the same transaction was ongoing.") entry != null && entry.currentTxnFirstOffset.isPresent && entry.producerEpoch() == producerEpoch } @@ -1030,7 +1035,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, // transaction is completed or aborted. We can guarantee the transaction coordinator knows about the transaction given step 1 and that the transaction is still // ongoing. If the transaction is expected to be ongoing, we will not set a VerificationGuard. If the transaction is aborted, hasOngoingTransaction is false and // requestVerificationGuard is the sentinel, so we will throw an error. A subsequent produce request (retry) should create verification state and return to phase 1. - if (batch.isTransactional && !hasOngoingTransaction(batch.producerId, batch.producerEpoch()) && batchMissingRequiredVerification(batch, requestVerificationGuard)) + if (batch.isTransactional && !batch.isControlBatch && !hasOngoingTransaction(batch.producerId, batch.producerEpoch()) && batchMissingRequiredVerification(batch, requestVerificationGuard)) throw new InvalidTxnStateException("Record was not part of an ongoing transaction") } @@ -1051,7 +1056,7 @@ class UnifiedLog(@volatile var logStartOffset: Long, } private def batchMissingRequiredVerification(batch: MutableRecordBatch, requestVerificationGuard: VerificationGuard): Boolean = { - producerStateManager.producerStateManagerConfig().transactionVerificationEnabled() && !batch.isControlBatch && + producerStateManager.producerStateManagerConfig().transactionVerificationEnabled() && !verificationGuard(batch.producerId).verify(requestVerificationGuard) } diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index edbc8db0fb2e0..242c939370f65 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -4020,6 +4020,42 @@ class UnifiedLogTest { log.appendAsLeader(transactionalRecords, leaderEpoch = 0, verificationGuard = verificationGuard) } + @Test + def testPreviousTransactionOngoing(): Unit = { + val producerStateManagerConfig = new ProducerStateManagerConfig(86400000, true) + + val producerId = 23L + val producerEpoch = 1.toShort + val logConfig = LogTestUtils.createLogConfig(segmentBytes = 2048 * 5) + val log = createLog(logDir, logConfig, producerStateManagerConfig = producerStateManagerConfig) + + val verificationGuard = log.maybeStartTransactionVerification(producerId, 0, producerEpoch) + assertNotEquals(VerificationGuard.SENTINEL, verificationGuard) + + val transactionalRecords = MemoryRecords.withTransactionalRecords( + Compression.NONE, + producerId, + producerEpoch, + 0, + new SimpleRecord("1".getBytes), + new SimpleRecord("2".getBytes) + ) + log.appendAsLeader(transactionalRecords, origin = AppendOrigin.CLIENT, leaderEpoch = 0, verificationGuard = verificationGuard) + + assertThrows(classOf[ConcurrentTransactionsException], () => log.maybeStartTransactionVerification(producerId, 0, (producerEpoch + 1).toShort)) + assertNotEquals(VerificationGuard.SENTINEL, verificationGuard) + + val endTransactionMarkerRecord = MemoryRecords.withEndTransactionMarker( + producerId, + producerEpoch, + new EndTransactionMarker(ControlRecordType.COMMIT, 0) + ) + + log.appendAsLeader(endTransactionMarkerRecord, origin = AppendOrigin.COORDINATOR, leaderEpoch = 0) + val verificationGuard2 = log.maybeStartTransactionVerification(producerId, 0, (producerEpoch + 1).toShort) + assertNotEquals(VerificationGuard.SENTINEL, verificationGuard2) + } + @Test def testRecoveryPointNotIncrementedOnProducerStateSnapshotFlushFailure(): Unit = { val logConfig = LogTestUtils.createLogConfig() From 32767599c0d9fd28a80fffcce1097e92cd256974 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Thu, 23 Jan 2025 15:05:06 +0800 Subject: [PATCH 23/44] MINOR: add the removed topic metrics to zk2kraft (#18658) Reviewers: Christo Lolov , Chia-Ping Tsai --- docs/zk2kraft.html | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/docs/zk2kraft.html b/docs/zk2kraft.html index f85b59b3bdc9b..123aaca4e18b1 100644 --- a/docs/zk2kraft.html +++ b/docs/zk2kraft.html @@ -176,5 +176,17 @@
    Removal metrics
    Kafka remove all zookeeper dependencies, so the metrics is removed.

  • +
  • +

    + Remove the metrics which is monitoring requests waiting in the topic purgatory +

    +
      +
    • kafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=topic
    • +
    • kafka.server:type=DelayedOperationPurgatory,name=NumDelayedOperations,delayedOperation=topic
    • +
    +

    + In Kraft mode, Zookeeper is not used, so the metrics is removed. +

    +
  • \ No newline at end of file From bc807083fbc14a6d9fd69f8b3303479c2e5bc054 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Thu, 23 Jan 2025 10:49:16 +0100 Subject: [PATCH 24/44] KAFKA-18486; [1/2] Update LocalLeaderEndPointTest (#18666) This patch is a first step towards removing `ReplicaManager#becomeLeaderOrFollower`. It updates the `LocalLeaderEndPointTest` tests. Reviewers: Christo Lolov , Ismael Juma --- .../server/LocalLeaderEndPointTest.scala | 126 ++++++++++-------- 1 file changed, 71 insertions(+), 55 deletions(-) diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala index 86ffdfb681cb5..6e9eeeb5cd79c 100644 --- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala +++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala @@ -20,38 +20,39 @@ package kafka.server import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.{CoreUtils, Logging, TestUtils} import org.apache.kafka.common.compress.Compression -import org.apache.kafka.common.{Node, TopicPartition, Uuid} +import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset +import org.apache.kafka.common.metadata.{PartitionChangeRecord, PartitionRecord, TopicRecord} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} -import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse +import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.server.common.{KRaftVersion, OffsetAndEpoch} import org.apache.kafka.server.network.BrokerEndPoint import org.apache.kafka.server.util.{MockScheduler, MockTime} -import org.apache.kafka.storage.internals.checkpoint.LazyOffsetCheckpoints import org.apache.kafka.storage.internals.log.{AppendOrigin, LogDirFailureChannel} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.Assertions._ import org.mockito.Mockito.mock import java.io.File -import java.util.Collections -import scala.collection.{Map, Seq} +import scala.collection.Map import scala.jdk.CollectionConverters._ class LocalLeaderEndPointTest extends Logging { val time = new MockTime - val topicId: Uuid = Uuid.randomUuid() + val topicId = Uuid.randomUuid() val topic = "test" - val topicPartition = new TopicPartition(topic, 5) + val partition = 5 + val topicPartition = new TopicPartition(topic, partition) val sourceBroker: BrokerEndPoint = new BrokerEndPoint(0, "localhost", 9092) var replicaManager: ReplicaManager = _ var endPoint: LeaderEndPoint = _ var quotaManager: QuotaManagers = _ + var image: MetadataImage = _ @BeforeEach def setUp(): Unit = { @@ -70,16 +71,35 @@ class LocalLeaderEndPointTest extends Logging { quotaManagers = quotaManager, metadataCache = MetadataCache.kRaftMetadataCache(config.brokerId, () => KRaftVersion.KRAFT_VERSION_0), logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size), - alterPartitionManager = alterPartitionManager) - val partition = replicaManager.createPartition(topicPartition) - partition.createLogIfNotExists(isNew = false, isFutureReplica = false, - new LazyOffsetCheckpoints(replicaManager.highWatermarkCheckpoints.asJava), None) - // Make this replica the leader. - val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 0) - replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) + alterPartitionManager = alterPartitionManager + ) + + val delta = new MetadataDelta(MetadataImage.EMPTY) + delta.replay(new TopicRecord() + .setName(topic) + .setTopicId(topicId) + ) + delta.replay(new PartitionRecord() + .setPartitionId(partition) + .setTopicId(topicId) + .setReplicas(List[Integer](sourceBroker.id).asJava) + .setIsr(List[Integer](sourceBroker.id).asJava) + .setLeader(sourceBroker.id) + .setLeaderEpoch(0) + .setPartitionEpoch(0) + ) + + image = delta.apply(MetadataProvenance.EMPTY) + replicaManager.applyDelta(delta.topicsDelta(), image) + replicaManager.getPartitionOrException(topicPartition) .localLogOrException - endPoint = new LocalLeaderEndPoint(sourceBroker, config, replicaManager, QuotaFactory.UNBOUNDED_QUOTA) + endPoint = new LocalLeaderEndPoint( + sourceBroker, + config, + replicaManager, + QuotaFactory.UNBOUNDED_QUOTA + ) } @AfterEach @@ -93,11 +113,10 @@ class LocalLeaderEndPointTest extends Logging { appendRecords(replicaManager, topicPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) assertEquals(new OffsetAndEpoch(3L, 0), endPoint.fetchLatestOffset(topicPartition, currentLeaderEpoch = 0)) - val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) - replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) + bumpLeaderEpoch() appendRecords(replicaManager, topicPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) - assertEquals(new OffsetAndEpoch(6L, 4), endPoint.fetchLatestOffset(topicPartition, currentLeaderEpoch = 7)) + assertEquals(new OffsetAndEpoch(6L, 1), endPoint.fetchLatestOffset(topicPartition, currentLeaderEpoch = 7)) } @Test @@ -106,12 +125,11 @@ class LocalLeaderEndPointTest extends Logging { .onFire(response => assertEquals(Errors.NONE, response.error)) assertEquals(new OffsetAndEpoch(0L, 0), endPoint.fetchEarliestOffset(topicPartition, currentLeaderEpoch = 0)) - val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) - replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) + bumpLeaderEpoch() appendRecords(replicaManager, topicPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) replicaManager.deleteRecords(timeout = 1000L, Map(topicPartition -> 3), _ => ()) - assertEquals(new OffsetAndEpoch(3L, 4), endPoint.fetchEarliestOffset(topicPartition, currentLeaderEpoch = 7)) + assertEquals(new OffsetAndEpoch(3L, 1), endPoint.fetchEarliestOffset(topicPartition, currentLeaderEpoch = 7)) } @Test @@ -120,13 +138,12 @@ class LocalLeaderEndPointTest extends Logging { .onFire(response => assertEquals(Errors.NONE, response.error)) assertEquals(new OffsetAndEpoch(0L, 0), endPoint.fetchEarliestLocalOffset(topicPartition, currentLeaderEpoch = 0)) - val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) - replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) + bumpLeaderEpoch() appendRecords(replicaManager, topicPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) replicaManager.logManager.getLog(topicPartition).foreach(log => log._localLogStartOffset = 3) assertEquals(new OffsetAndEpoch(0L, 0), endPoint.fetchEarliestOffset(topicPartition, currentLeaderEpoch = 7)) - assertEquals(new OffsetAndEpoch(3L, 4), endPoint.fetchEarliestLocalOffset(topicPartition, currentLeaderEpoch = 7)) + assertEquals(new OffsetAndEpoch(3L, 1), endPoint.fetchEarliestLocalOffset(topicPartition, currentLeaderEpoch = 7)) } @Test @@ -137,42 +154,49 @@ class LocalLeaderEndPointTest extends Logging { var result = endPoint.fetchEpochEndOffsets(Map( topicPartition -> new OffsetForLeaderPartition() .setPartition(topicPartition.partition) - .setLeaderEpoch(0))) + .setLeaderEpoch(0) + )) var expected = Map( topicPartition -> new EpochEndOffset() .setPartition(topicPartition.partition) .setErrorCode(Errors.NONE.code) .setLeaderEpoch(0) - .setEndOffset(3L)) + .setEndOffset(3L) + ) assertEquals(expected, result) // Change leader epoch and end offset, and verify the behavior again. - val leaderAndIsrRequest = buildLeaderAndIsrRequest(leaderEpoch = 4) - replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) + bumpLeaderEpoch() + bumpLeaderEpoch() + assertEquals(2, replicaManager.getPartitionOrException(topicPartition).getLeaderEpoch) + appendRecords(replicaManager, topicPartition, records) .onFire(response => assertEquals(Errors.NONE, response.error)) result = endPoint.fetchEpochEndOffsets(Map( topicPartition -> new OffsetForLeaderPartition() .setPartition(topicPartition.partition) - .setLeaderEpoch(4))) + .setLeaderEpoch(2) + )) expected = Map( topicPartition -> new EpochEndOffset() .setPartition(topicPartition.partition) .setErrorCode(Errors.NONE.code) - .setLeaderEpoch(4) - .setEndOffset(6L)) + .setLeaderEpoch(2) + .setEndOffset(6L) + ) assertEquals(expected, result) - // Check missing epoch: 3, we expect the API to return (leader_epoch=0, end_offset=3). + // Check missing epoch: 1, we expect the API to return (leader_epoch=0, end_offset=3). result = endPoint.fetchEpochEndOffsets(Map( topicPartition -> new OffsetForLeaderPartition() .setPartition(topicPartition.partition) - .setLeaderEpoch(3))) + .setLeaderEpoch(1) + )) expected = Map( topicPartition -> new EpochEndOffset() @@ -187,14 +211,16 @@ class LocalLeaderEndPointTest extends Logging { result = endPoint.fetchEpochEndOffsets(Map( topicPartition -> new OffsetForLeaderPartition() .setPartition(topicPartition.partition) - .setLeaderEpoch(5))) + .setLeaderEpoch(5) + )) expected = Map( topicPartition -> new EpochEndOffset() .setPartition(topicPartition.partition) .setErrorCode(Errors.NONE.code) .setLeaderEpoch(-1) - .setEndOffset(-1L)) + .setEndOffset(-1L) + ) assertEquals(expected, result) } @@ -219,22 +245,16 @@ class LocalLeaderEndPointTest extends Logging { } } - private def buildLeaderAndIsrRequest(leaderEpoch: Int): LeaderAndIsrRequest = { - val brokerList = Seq[Integer](sourceBroker.id).asJava - val topicIds = Collections.singletonMap(topic, topicId) - new LeaderAndIsrRequest.Builder(0, 0, 0, - Seq(new LeaderAndIsrRequest.PartitionState() - .setTopicName(topic) - .setPartitionIndex(topicPartition.partition()) - .setControllerEpoch(0) - .setLeader(sourceBroker.id) - .setLeaderEpoch(leaderEpoch) - .setIsr(brokerList) - .setPartitionEpoch(0) - .setReplicas(brokerList) - .setIsNew(false)).asJava, - topicIds, - Set(node(sourceBroker)).asJava).build() + private def bumpLeaderEpoch(): Unit = { + val delta = new MetadataDelta(image) + delta.replay(new PartitionChangeRecord() + .setTopicId(topicId) + .setPartitionId(partition) + .setLeader(sourceBroker.id) + ) + + image = delta.apply(MetadataProvenance.EMPTY) + replicaManager.applyDelta(delta.topicsDelta, image) } private def appendRecords(replicaManager: ReplicaManager, @@ -260,10 +280,6 @@ class LocalLeaderEndPointTest extends Logging { result } - private def node(endPoint: BrokerEndPoint): Node = { - new Node(endPoint.id, endPoint.host, endPoint.port) - } - private def records: MemoryRecords = { MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("first message".getBytes()), From 27552e7b1195e05b22a5ae2d99c043f172c724fe Mon Sep 17 00:00:00 2001 From: Pramithas Dhakal Date: Thu, 23 Jan 2025 16:24:45 +0545 Subject: [PATCH 25/44] MINOR: Make `forceUnmap` method protected (#18638) Reviewers: Divij Vaidya --- .../org/apache/kafka/storage/internals/log/AbstractIndex.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java index 9c7c645c4a705..bf1bd802f5e0c 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/AbstractIndex.java @@ -399,8 +399,8 @@ protected void safeForceUnmap() { /** * Forcefully free the buffer's mmap. */ - // Visible for testing, we can make this protected once OffsetIndexTest is in the same package as this class - public void forceUnmap() throws IOException { + // Visible for testing + protected void forceUnmap() throws IOException { try { ByteBufferUnmapper.unmap(file.getAbsolutePath(), mmap); } finally { From a783dc69b9bb7da1cc5e517ad33a54690ca5ce61 Mon Sep 17 00:00:00 2001 From: qingbozhang Date: Thu, 23 Jan 2025 19:15:41 +0800 Subject: [PATCH 26/44] MINOR: Replace Stream#collect with Stream#toList api from JDK 16 (#18655) Reviewers: Divij Vaidya --- .../java/org/apache/kafka/storage/internals/log/LogLoader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java index 89780686995fb..285e9fb11f759 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogLoader.java @@ -357,7 +357,7 @@ private void loadSegmentFiles() throws IOException { // segments that come before it File[] files = dir.listFiles(); if (files == null) files = new File[0]; - List sortedFiles = Arrays.stream(files).filter(File::isFile).sorted().collect(Collectors.toList()); + List sortedFiles = Arrays.stream(files).filter(File::isFile).sorted().toList(); for (File file : sortedFiles) { if (LogFileUtils.isIndexFile(file)) { // if it is an index file, make sure it has a corresponding .log file From bdc92fd5a1deb00c719c65ceaa9bc5fb65eed0a8 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Thu, 23 Jan 2025 19:53:10 +0800 Subject: [PATCH 27/44] MINOR: Cleanup zk condition in TransactionsTest, QuorumTestHarness and PlaintextConsumerAssignorsTest (#18639) Reviewers: Ken Huang , Chia-Ping Tsai --- .../api/PlaintextConsumerAssignorsTest.scala | 2 - .../kafka/api/TransactionsTest.scala | 40 +++++-------------- .../kafka/server/QuorumTestHarness.scala | 6 +-- 3 files changed, 12 insertions(+), 36 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignorsTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignorsTest.scala index 23f5a1601b4fe..13251efc78837 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignorsTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerAssignorsTest.scala @@ -308,8 +308,6 @@ class PlaintextConsumerAssignorsTest extends AbstractConsumerTest { // Only the classic group protocol supports client-side assignors @ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}.assignmentStrategy={2}") @CsvSource(Array( - "zk, classic, org.apache.kafka.clients.consumer.CooperativeStickyAssignor", - "zk, classic, org.apache.kafka.clients.consumer.RangeAssignor", "kraft, classic, org.apache.kafka.clients.consumer.CooperativeStickyAssignor", "kraft, classic, org.apache.kafka.clients.consumer.RangeAssignor" )) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 66ff64f2cdc03..2733167b46994 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -587,14 +587,9 @@ class TransactionsTest extends IntegrationTestHarness { fail("Should not be able to send messages from a fenced producer.") } catch { case _: InvalidProducerEpochException => - case e: ExecutionException => { - if (quorum == "zk") { - assertTrue(e.getCause.isInstanceOf[ProducerFencedException]) - } else { - // In kraft mode, transactionV2 is used. - assertTrue(e.getCause.isInstanceOf[InvalidProducerEpochException]) - } - } + case e: ExecutionException => + // In kraft mode, transactionV2 is used. + assertTrue(e.getCause.isInstanceOf[InvalidProducerEpochException]) case e: Exception => throw new AssertionError("Got an unexpected exception from a fenced producer.", e) } @@ -622,27 +617,14 @@ class TransactionsTest extends IntegrationTestHarness { // Wait for the expiration cycle to kick in. Thread.sleep(600) - if (quorum == "zk") { - // In zk mode, transaction v1 is used. - try { - // Now that the transaction has expired, the second send should fail with a ProducerFencedException. - producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "2", willBeCommitted = false)).get() - fail("should have raised a ProducerFencedException since the transaction has expired") - } catch { - case _: ProducerFencedException => - case e: ExecutionException => - assertTrue(e.getCause.isInstanceOf[ProducerFencedException]) - } - } else { - try { - // Now that the transaction has expired, the second send should fail with a InvalidProducerEpochException. - producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "2", willBeCommitted = false)).get() - fail("should have raised a InvalidProducerEpochException since the transaction has expired") - } catch { - case _: InvalidProducerEpochException => - case e: ExecutionException => - assertTrue(e.getCause.isInstanceOf[InvalidProducerEpochException]) - } + try { + // Now that the transaction has expired, the second send should fail with a InvalidProducerEpochException. + producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(topic1, null, "2", "2", willBeCommitted = false)).get() + fail("should have raised a InvalidProducerEpochException since the transaction has expired") + } catch { + case _: InvalidProducerEpochException => + case e: ExecutionException => + assertTrue(e.getCause.isInstanceOf[InvalidProducerEpochException]) } // Verify that the first message was aborted and the second one was never written at all. diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 72b76cd152d25..3b400c573f93c 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -270,7 +270,6 @@ abstract class QuorumTestHarness extends Logging { props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:0,$listeners") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, s"CONTROLLER,$listenerNames") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:0") - // Setting the configuration to the same value set on the brokers via TestUtils to keep KRaft based and Zk based controller configs are consistent. props.setProperty(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000") val config = new KafkaConfig(props) @@ -369,7 +368,7 @@ object QuorumTestHarness { /** * Verify that a previous test that doesn't use QuorumTestHarness hasn't left behind an unexpected thread. - * This assumes that brokers, ZooKeeper clients, producers and consumers are not created in another @BeforeClass, + * This assumes that brokers, admin clients, producers and consumers are not created in another @BeforeClass, * which is true for core tests where this harness is used. */ @BeforeAll @@ -437,9 +436,6 @@ object QuorumTestHarness { ) } - // The following is for tests that only work with the classic group protocol because of relying on Zookeeper - def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit: java.util.stream.Stream[Arguments] = stream.Stream.of(Arguments.of("zk", GroupProtocol.CLASSIC.name.toLowerCase(Locale.ROOT))) - // The following parameter groups are to *temporarily* avoid bugs with the CONSUMER group protocol Consumer // implementation that would otherwise cause tests to fail. def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_18034: stream.Stream[Arguments] = getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly From 8000d04dcba4b3b33a91aa0052c9c2ff9d304d65 Mon Sep 17 00:00:00 2001 From: Andrew Schofield Date: Thu, 23 Jan 2025 13:32:59 +0000 Subject: [PATCH 28/44] KAFKA-18488: Additional protocol tests for share consumption (#18601) Reviewers: ShivsundarR , Lianet Magrans --- .../consumer/KafkaShareConsumerTest.java | 335 ++++++++++++++++++ 1 file changed, 335 insertions(+) create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java new file mode 100644 index 0000000000000..63880ace0827a --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java @@ -0,0 +1,335 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer; + +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy; +import org.apache.kafka.clients.consumer.internals.ConsumerMetadata; +import org.apache.kafka.clients.consumer.internals.SubscriptionState; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.internals.ClusterResourceListeners; +import org.apache.kafka.common.message.ShareAcknowledgeResponseData; +import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.message.ShareGroupHeartbeatResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.record.MemoryRecordsBuilder; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.requests.RequestTestUtils; +import org.apache.kafka.common.requests.ShareAcknowledgeRequest; +import org.apache.kafka.common.requests.ShareAcknowledgeResponse; +import org.apache.kafka.common.requests.ShareFetchRequest; +import org.apache.kafka.common.requests.ShareFetchResponse; +import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest; +import org.apache.kafka.common.requests.ShareGroupHeartbeatResponse; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.LogContext; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +// This test exercises the KafkaShareConsumer with the MockClient to validate the Kafka protocol RPCs +@Timeout(value = 120) +@SuppressWarnings({"ClassDataAbstractionCoupling"}) +public class KafkaShareConsumerTest { + + private final String groupId = "test-group"; + private final String clientId1 = "client-id-1"; + + private final String topic1 = "test1"; + private final Uuid topicId1 = Uuid.randomUuid(); + private final TopicPartition t1p0 = new TopicPartition(topic1, 0); + private final TopicIdPartition ti1p0 = new TopicIdPartition(topicId1, t1p0); + + private final Map topicIds = Map.of(topic1, topicId1); + + private final int batchSize = 10; + private final int heartbeatIntervalMs = 1000; + + private final Time time = new MockTime(); + private final SubscriptionState subscription = new SubscriptionState(new LogContext(), AutoOffsetResetStrategy.EARLIEST); + + @Test + public void testVerifyHeartbeats() throws InterruptedException { + ConsumerMetadata metadata = new ConsumerMetadata(0, 0, Long.MAX_VALUE, false, false, + subscription, new LogContext(), new ClusterResourceListeners()); + MockClient client = new MockClient(time, metadata); + + initMetadata(client, Map.of(topic1, 1)); + Node node = metadata.fetch().nodes().get(0); + + Node coordinator = findCoordinator(client, node); + + // The member ID is created by the consumer and then sent in the initial request. The responses + // need to contain the same member ID. + final AtomicReference memberId = new AtomicReference<>(); + final AtomicInteger heartbeatsReceived = new AtomicInteger(); + client.prepareResponseFrom(body -> { + ShareGroupHeartbeatRequest request = (ShareGroupHeartbeatRequest) body; + memberId.set(Uuid.fromString(request.data().memberId())); + boolean matches = request.data().memberEpoch() == 0; + heartbeatsReceived.addAndGet(1); + + client.prepareResponseFrom(body2 -> { + ShareGroupHeartbeatRequest request2 = (ShareGroupHeartbeatRequest) body2; + boolean matches2 = request2.data().memberId().equals(memberId.get().toString()) && request2.data().memberEpoch() == 1; + heartbeatsReceived.addAndGet(1); + return matches2; + }, shareGroupHeartbeatResponse(memberId.get(), 2, ti1p0), coordinator); + + return matches; + }, shareGroupHeartbeatResponse(memberId.get(), 1, ti1p0), coordinator); + + try (KafkaShareConsumer consumer = newShareConsumer(clientId1, metadata, client)) { + consumer.subscribe(Set.of(topic1)); + consumer.poll(Duration.ZERO); + + Thread.sleep(heartbeatIntervalMs); + + assertEquals(2, heartbeatsReceived.get()); + assertTrue(client.futureResponses().isEmpty()); + + consumer.close(Duration.ZERO); + } + } + + @Test + public void testVerifyFetchAndAcknowledgeSync() throws InterruptedException { + ConsumerMetadata metadata = new ConsumerMetadata(0, 0, Long.MAX_VALUE, false, false, + subscription, new LogContext(), new ClusterResourceListeners()); + MockClient client = new MockClient(time, metadata); + + initMetadata(client, Map.of(topic1, 1)); + Node node = metadata.fetch().nodes().get(0); + + Node coordinator = findCoordinator(client, node); + + final AtomicReference memberId = new AtomicReference<>(); + final AtomicBoolean memberLeft = shareGroupHeartbeatGenerator(client, coordinator, memberId, ti1p0); + + // [A] A SHARE_FETCH in a new share session, fetching from topic topicId1, with no acknowledgements included. + // The response includes 2 records which are acquired. + client.prepareResponseFrom(body -> { + ShareFetchRequest request = (ShareFetchRequest) body; + return request.data().groupId().equals(groupId) && + request.data().shareSessionEpoch() == 0 && + request.data().batchSize() == batchSize && + request.data().topics().get(0).topicId().equals(topicId1) && + request.data().topics().get(0).partitions().size() == 1 && + request.data().topics().get(0).partitions().get(0).acknowledgementBatches().isEmpty(); + }, shareFetchResponse(ti1p0, 2), node); + + // [B] A SHARE_ACKNOWLEDGE for the two records acquired in [A]. + client.prepareResponseFrom(body -> { + ShareAcknowledgeRequest request = (ShareAcknowledgeRequest) body; + return request.data().groupId().equals(groupId) && + request.data().shareSessionEpoch() == 1 && + request.data().topics().get(0).partitions().get(0).acknowledgementBatches().get(0).firstOffset() == 0 && + request.data().topics().get(0).partitions().get(0).acknowledgementBatches().get(0).lastOffset() == 1 && + request.data().topics().get(0).partitions().get(0).acknowledgementBatches().get(0).acknowledgeTypes().size() == 1 && + request.data().topics().get(0).partitions().get(0).acknowledgementBatches().get(0).acknowledgeTypes().get(0) == (byte) 1; + }, shareAcknowledgeResponse(ti1p0), node); + + // [C] A SHARE_ACKNOWLEDGE which closes the share session. + client.prepareResponseFrom(body -> { + ShareAcknowledgeRequest request = (ShareAcknowledgeRequest) body; + return request.data().groupId().equals(groupId) && + request.data().shareSessionEpoch() == -1 && + request.data().topics().isEmpty(); + }, shareAcknowledgeResponse(), node); + + try (KafkaShareConsumer consumer = newShareConsumer(clientId1, metadata, client)) { + + consumer.subscribe(Set.of(topic1)); + + // This will be a SHARE_GROUP_HEARTBEAT to establish the membership and then a SHARE_FETCH [A] + consumer.poll(Duration.ofMillis(heartbeatIntervalMs)); + + // This will be a SHARE_ACKNOWLEDGE [B] + consumer.commitSync(); + + // This will be a SHARE_ACKNOWLEDGE [C] and a final SHARE_GROUP_HEARTBEAT to leave the group + consumer.close(Duration.ZERO); + + assertTrue(memberLeft.get()); + assertTrue(client.futureResponses().isEmpty()); + } + } + + private KafkaShareConsumer newShareConsumer(String clientId, + ConsumerMetadata metadata, + KafkaClient client) { + LogContext logContext = new LogContext(); + Deserializer keyDeserializer = new StringDeserializer(); + Deserializer valueDeserializer = new StringDeserializer(); + ConsumerConfig config = newConsumerConfig(clientId); + + return new KafkaShareConsumer<>( + logContext, + clientId, + groupId, + config, + keyDeserializer, + valueDeserializer, + time, + client, + subscription, + metadata + ); + } + + private ConsumerConfig newConsumerConfig(String clientId) { + Map configs = new HashMap<>(); + configs.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId); + configs.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + configs.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, batchSize); + return new ConsumerConfig(configs); + } + + private void initMetadata(MockClient client, Map partitions) { + // It is important that this MetadataResponse contains the topic IDs for the topics which are being + // subscribed to, otherwise the ShareMembershipManager will cause a second MetadataRequest to be sent out + Map metadataTopicIds = new HashMap<>(); + for (String name : partitions.keySet()) { + metadataTopicIds.put(name, topicIds.get(name)); + } + MetadataResponse initialMetadata = RequestTestUtils.metadataUpdateWithIds(1, partitions, metadataTopicIds); + client.updateMetadata(initialMetadata); + } + + private Node findCoordinator(MockClient client, Node node) { + client.prepareResponseFrom(FindCoordinatorResponse.prepareResponse(Errors.NONE, groupId, node), node); + return new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); + } + + // This method generates a sequence of prepared SHARE_GROUP_HEARTBEAT responses with increasing member epochs. + // Each time that a SHARE_GROUP_HEARTBEAT response matches the prepared response matcher, the next prepared + // response is added, until the matching requests member epoch is -1, indicating that the member is leaving + // the group. + private AtomicBoolean shareGroupHeartbeatGenerator(MockClient client, Node coordinator, AtomicReference memberId, TopicIdPartition tip) { + AtomicBoolean memberLeft = new AtomicBoolean(); + AtomicInteger heartbeatsReceived = new AtomicInteger(); + shareGroupHeartbeat(client, coordinator, memberId, 0, tip, heartbeatsReceived, memberLeft); + return memberLeft; + } + + private void shareGroupHeartbeat(MockClient client, Node coordinator, AtomicReference memberId, int memberEpoch, TopicIdPartition tip, AtomicInteger heartbeatsReceived, AtomicBoolean memberLeft) { + client.prepareResponseFrom(body -> { + ShareGroupHeartbeatRequest request = (ShareGroupHeartbeatRequest) body; + if (request.data().memberEpoch() == 0) { + memberId.set(Uuid.fromString(request.data().memberId())); + } + if (request.data().memberEpoch() == -1) { + memberLeft.set(true); + } else { + shareGroupHeartbeat(client, coordinator, memberId, memberEpoch + 1, tip, heartbeatsReceived, memberLeft); + } + heartbeatsReceived.addAndGet(1); + return true; + }, shareGroupHeartbeatResponse(memberId.get(), memberEpoch, tip), coordinator); + } + + private ShareGroupHeartbeatResponse shareGroupHeartbeatResponse(Uuid memberId, int memberEpoch, TopicIdPartition tip) { + if (memberEpoch != -1) { + List assignedPartitions = new LinkedList<>(); + assignedPartitions.add(new ShareGroupHeartbeatResponseData.TopicPartitions().setTopicId(tip.topicId()).setPartitions(List.of(tip.partition()))); + + return new ShareGroupHeartbeatResponse( + new ShareGroupHeartbeatResponseData() + .setMemberId(memberId != null ? memberId.toString() : null) + .setMemberEpoch(memberEpoch) + .setHeartbeatIntervalMs(heartbeatIntervalMs) + .setAssignment(new ShareGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(assignedPartitions)) + ); + } else { + return new ShareGroupHeartbeatResponse( + new ShareGroupHeartbeatResponseData() + .setMemberId(memberId != null ? memberId.toString() : null) + .setMemberEpoch(memberEpoch) + .setHeartbeatIntervalMs(heartbeatIntervalMs) + ); + } + } + + private ShareFetchResponse shareFetchResponse(TopicIdPartition tip, int count) { + MemoryRecords records; + try (MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE, + TimestampType.CREATE_TIME, 0)) { + for (int i = 0; i < count; i++) { + builder.append(0L, ("key-" + i).getBytes(), ("value-" + i).getBytes()); + } + records = builder.build(); + } + ShareFetchResponseData.PartitionData partData = new ShareFetchResponseData.PartitionData() + .setPartitionIndex(tip.partition()) + .setRecords(records) + .setAcquiredRecords(List.of(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(count - 1).setDeliveryCount((short) 1))); + ShareFetchResponseData.ShareFetchableTopicResponse topicResponse = new ShareFetchResponseData.ShareFetchableTopicResponse() + .setTopicId(tip.topicId()) + .setPartitions(List.of(partData)); + return new ShareFetchResponse( + new ShareFetchResponseData() + .setResponses(List.of(topicResponse)) + ); + } + + private ShareAcknowledgeResponse shareAcknowledgeResponse() { + return new ShareAcknowledgeResponse( + new ShareAcknowledgeResponseData() + ); + } + + private ShareAcknowledgeResponse shareAcknowledgeResponse(TopicIdPartition tip) { + ShareAcknowledgeResponseData.PartitionData partData = new ShareAcknowledgeResponseData.PartitionData() + .setPartitionIndex(tip.partition()) + .setErrorCode(Errors.NONE.code()); + ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse topicResponse = new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponse() + .setTopicId(tip.topicId()) + .setPartitions(List.of(partData)); + return new ShareAcknowledgeResponse( + new ShareAcknowledgeResponseData() + .setResponses(List.of(topicResponse)) + ); + } +} \ No newline at end of file From 5946f27ac5bc1f4a5bc162ccc26f130933ab2182 Mon Sep 17 00:00:00 2001 From: Sean Quah Date: Thu, 23 Jan 2025 16:15:21 +0000 Subject: [PATCH 29/44] KAFKA-18484 [2/2]; Handle exceptions during coordinator unload (#18667) Ensure that unloading a coordinator always succeeds. Previously, we have guarded against exceptions from DeferredEvent completions. All that remains is handling exceptions from the onUnloaded() method of the coordinator state machine. Reviewers: David Jacot --- .../common/runtime/CoordinatorRuntime.java | 24 +++++++-- .../runtime/CoordinatorRuntimeTest.java | 52 +++++++++++++++++++ 2 files changed, 72 insertions(+), 4 deletions(-) diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java index b341c0adaeb07..1e9724a57aa8e 100644 --- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java +++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java @@ -742,7 +742,11 @@ private void unload() { deferredEventQueue.failAll(Errors.NOT_COORDINATOR.exception()); failCurrentBatch(Errors.NOT_COORDINATOR.exception()); if (coordinator != null) { - coordinator.onUnloaded(); + try { + coordinator.onUnloaded(); + } catch (Throwable ex) { + log.error("Failed to unload coordinator for {} due to {}.", tp, ex.getMessage(), ex); + } } coordinator = null; } @@ -2415,9 +2419,19 @@ public void scheduleUnloadOperation( try { if (partitionEpoch.isEmpty() || context.epoch < partitionEpoch.getAsInt()) { log.info("Started unloading metadata for {} with epoch {}.", tp, partitionEpoch); - context.transitionTo(CoordinatorState.CLOSED); - coordinators.remove(tp, context); - log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch); + try { + context.transitionTo(CoordinatorState.CLOSED); + log.info("Finished unloading metadata for {} with epoch {}.", tp, partitionEpoch); + } catch (Throwable ex) { + // It's very unlikely that we will ever see an exception here, since we + // already make an effort to catch exceptions in the unload method. + log.error("Failed to unload metadata for {} with epoch {} due to {}.", + tp, partitionEpoch, ex.toString()); + } finally { + // Always remove the coordinator context, otherwise the coordinator + // shard could be permanently stuck. + coordinators.remove(tp, context); + } } else { log.info("Ignored unloading metadata for {} in epoch {} since current epoch is {}.", tp, partitionEpoch, context.epoch); @@ -2498,6 +2512,8 @@ public void close() throws Exception { context.lock.lock(); try { context.transitionTo(CoordinatorState.CLOSED); + } catch (Throwable ex) { + log.warn("Failed to unload metadata for {} due to {}.", tp, ex.getMessage(), ex); } finally { context.lock.unlock(); } diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java index 3c2021a118c44..9e4e6f7bb9b44 100644 --- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java +++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java @@ -1118,6 +1118,58 @@ public void testScheduleUnloadingWithStalePartitionEpoch() { assertEquals(10, ctx.epoch); } + @Test + public void testScheduleUnloadingWithException() { + MockTimer timer = new MockTimer(); + MockPartitionWriter writer = mock(MockPartitionWriter.class); + MockCoordinatorShardBuilderSupplier supplier = mock(MockCoordinatorShardBuilderSupplier.class); + MockCoordinatorShardBuilder builder = mock(MockCoordinatorShardBuilder.class); + MockCoordinatorShard coordinator = mock(MockCoordinatorShard.class); + CoordinatorRuntimeMetrics metrics = mock(CoordinatorRuntimeMetrics.class); + + CoordinatorRuntime runtime = + new CoordinatorRuntime.Builder() + .withTime(timer.time()) + .withTimer(timer) + .withDefaultWriteTimeOut(DEFAULT_WRITE_TIMEOUT) + .withLoader(new MockCoordinatorLoader()) + .withEventProcessor(new DirectEventProcessor()) + .withPartitionWriter(writer) + .withCoordinatorShardBuilderSupplier(supplier) + .withCoordinatorRuntimeMetrics(metrics) + .withCoordinatorMetrics(mock(CoordinatorMetrics.class)) + .withSerializer(new StringSerializer()) + .withExecutorService(mock(ExecutorService.class)) + .build(); + + doThrow(new KafkaException("error")).when(coordinator).onUnloaded(); + when(builder.withSnapshotRegistry(any())).thenReturn(builder); + when(builder.withLogContext(any())).thenReturn(builder); + when(builder.withTime(any())).thenReturn(builder); + when(builder.withTimer(any())).thenReturn(builder); + when(builder.withCoordinatorMetrics(any())).thenReturn(builder); + when(builder.withTopicPartition(any())).thenReturn(builder); + when(builder.withExecutor(any())).thenReturn(builder); + when(builder.build()).thenReturn(coordinator); + when(supplier.get()).thenReturn(builder); + + // Loads the coordinator. It directly transitions to active. + runtime.scheduleLoadOperation(TP, 10); + CoordinatorRuntime.CoordinatorContext ctx = runtime.contextOrThrow(TP); + assertEquals(ACTIVE, ctx.state); + assertEquals(10, ctx.epoch); + + // Schedule the unloading. + runtime.scheduleUnloadOperation(TP, OptionalInt.of(ctx.epoch + 1)); + assertEquals(CLOSED, ctx.state); + + // Verify that onUnloaded is called. + verify(coordinator, times(1)).onUnloaded(); + + // Getting the coordinator context fails because it no longer exists. + assertThrows(NotCoordinatorException.class, () -> runtime.contextOrThrow(TP)); + } + @Test public void testScheduleUnloadingWithDeferredEventExceptions() throws ExecutionException, InterruptedException, TimeoutException { MockTimer timer = new MockTimer(); From 400ecab5189f68881223b7a1d8194214102e6784 Mon Sep 17 00:00:00 2001 From: Karsten Spang <47601460+karstenspang@users.noreply.github.com> Date: Thu, 23 Jan 2025 17:20:30 +0100 Subject: [PATCH 30/44] KAFKA-13810: Document behavior of KafkaProducer.flush() w.r.t callbacks (#12042) Reviewers: Luke Chen , Andrew Eugene Choi --- .../java/org/apache/kafka/clients/producer/KafkaProducer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 648de3ab4b90a..64930b36e27e4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -1154,7 +1154,8 @@ private void ensureValidRecordSize(int size) { /** * Invoking this method makes all buffered records immediately available to send (even if linger.ms is * greater than 0) and blocks on the completion of the requests associated with these records. The post-condition - * of flush() is that any previously sent record will have completed (e.g. Future.isDone() == true). + * of flush() is that any previously sent record will have completed (e.g. Future.isDone() == true + * and callbacks passed to {@link #send(ProducerRecord,Callback)} have been called). * A request is considered completed when it is successfully acknowledged * according to the acks configuration you have specified or else it results in an error. *

    From 7e46087570d70f42b85fa6f0e23d3b2441591eb4 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Fri, 24 Jan 2025 00:33:05 +0800 Subject: [PATCH 31/44] MINOR: rename `resendBrokerRegistrationUnlessZkMode` to `resendBrokerRegistration` (#18645) Reviewers: Chia-Ping Tsai --- .../main/scala/kafka/server/BrokerLifecycleManager.scala | 6 +++--- core/src/main/scala/kafka/server/BrokerServer.scala | 2 +- .../unit/kafka/server/BrokerLifecycleManagerTest.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala index bd01311feddc4..3079af0a307bd 100644 --- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala +++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala @@ -260,11 +260,11 @@ class BrokerLifecycleManager( new OfflineDirBrokerFailureEvent(directory)) } - def resendBrokerRegistrationUnlessZkMode(): Unit = { - eventQueue.append(new ResendBrokerRegistrationUnlessZkModeEvent()) + def resendBrokerRegistration(): Unit = { + eventQueue.append(new ResendBrokerRegistrationEvent()) } - private class ResendBrokerRegistrationUnlessZkModeEvent extends EventQueue.Event { + private class ResendBrokerRegistrationEvent extends EventQueue.Event { override def run(): Unit = { registered = false scheduleNextCommunicationImmediately() diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index ace134773ae2b..3c735be0004c4 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -534,7 +534,7 @@ class BrokerServer( }) metadataPublishers.add(brokerMetadataPublisher) brokerRegistrationTracker = new BrokerRegistrationTracker(config.brokerId, - () => lifecycleManager.resendBrokerRegistrationUnlessZkMode()) + () => lifecycleManager.resendBrokerRegistration()) metadataPublishers.add(brokerRegistrationTracker) diff --git a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala index a7204ffe884c3..5b621671ad6e6 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerLifecycleManagerTest.scala @@ -288,7 +288,7 @@ class BrokerLifecycleManagerTest { assertEquals(1000L, manager.brokerEpoch) // Trigger JBOD MV update - manager.resendBrokerRegistrationUnlessZkMode() + manager.resendBrokerRegistration() // Accept new registration, response sets epoch to 1200 nextRegistrationRequest(1200L) From aea699bdef01b169b73f14c6f1d58df456056e16 Mon Sep 17 00:00:00 2001 From: Lucas Brutschy Date: Thu, 23 Jan 2025 17:35:03 +0100 Subject: [PATCH 32/44] KAFKA-18324: Add CurrentAssignmentBuilder (#18476) Implements the current assignment builder, analogous to the current assignment builder of consumer groups. The main difference is the underlying assigned resource, and slightly different logic around process IDs: We make sure to move a task only to a new client, once the task is not owned anymore by any client with the same process ID (sharing the same state directory) - in any role (active, standby or warm-up). Compared to the feature branch, the main difference is that I refactored the separate treatment of active, standby and warm-up tasks into a compound datatype called TaskTuple (which is used in place of the more specific Assignment class). This also has effects on StreamsGroupMember. Reviewers: Bruno Cadonna , Bill Bejeck --- .../streams/CurrentAssignmentBuilder.java | 451 ++++++++++ .../group/streams/StreamsGroupMember.java | 163 +--- .../{Assignment.java => TasksTuple.java} | 73 +- .../streams/assignor/GroupAssignment.java | 2 +- .../streams/CurrentAssignmentBuilderTest.java | 825 ++++++++++++++++++ .../group/streams/StreamsGroupMemberTest.java | 132 +-- .../group/streams/TaskAssignmentTestUtil.java | 26 +- ...ssignmentTest.java => TasksTupleTest.java} | 95 +- 8 files changed, 1530 insertions(+), 237 deletions(-) create mode 100644 group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java rename group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/{Assignment.java => TasksTuple.java} (50%) create mode 100644 group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java rename group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/{AssignmentTest.java => TasksTupleTest.java} (57%) diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java new file mode 100644 index 0000000000000..3c9ba064a40cd --- /dev/null +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilder.java @@ -0,0 +1,451 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.streams; + +import org.apache.kafka.common.errors.FencedMemberEpochException; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.function.BiPredicate; + +/** + * The CurrentAssignmentBuilder class encapsulates the reconciliation engine of the streams group protocol. Given the current state of a + * member and a desired or target assignment state, the state machine takes the necessary steps to converge them. + */ +public class CurrentAssignmentBuilder { + + /** + * The streams group member which is reconciled. + */ + private final StreamsGroupMember member; + + /** + * The target assignment epoch. + */ + private int targetAssignmentEpoch; + + /** + * The target assignment. + */ + private TasksTuple targetAssignment; + + /** + * A function which returns the current process ID of an active task or null if the active task + * is not assigned. The current process ID is the process ID of the current owner. + */ + private BiFunction currentActiveTaskProcessId; + + /** + * A function which returns the current process IDs of a standby task or null if the standby + * task is not assigned. The current process IDs are the process IDs of all current owners. + */ + private BiFunction> currentStandbyTaskProcessIds; + + /** + * A function which returns the current process IDs of a warmup task or null if the warmup task + * is not assigned. The current process IDs are the process IDs of all current owners. + */ + private BiFunction> currentWarmupTaskProcessIds; + + /** + * The tasks owned by the member. This may be provided by the member in the StreamsGroupHeartbeat request. + */ + private Optional ownedTasks = Optional.empty(); + + /** + * Constructs the CurrentAssignmentBuilder based on the current state of the provided streams group member. + * + * @param member The streams group member that must be reconciled. + */ + public CurrentAssignmentBuilder(StreamsGroupMember member) { + this.member = Objects.requireNonNull(member); + } + + /** + * Sets the target assignment epoch and the target assignment that the streams group member must be reconciled to. + * + * @param targetAssignmentEpoch The target assignment epoch. + * @param targetAssignment The target assignment. + * @return This object. + */ + public CurrentAssignmentBuilder withTargetAssignment(int targetAssignmentEpoch, + TasksTuple targetAssignment) { + this.targetAssignmentEpoch = targetAssignmentEpoch; + this.targetAssignment = Objects.requireNonNull(targetAssignment); + return this; + } + + /** + * Sets a BiFunction which allows to retrieve the current process ID of an active task. This is + * used by the state machine to determine if an active task is free or still used by another + * member, and if there is still a task on a specific process that is not yet revoked. + * + * @param currentActiveTaskProcessId A BiFunction which gets the process ID of a subtopology ID / + * partition ID pair. + * @return This object. + */ + public CurrentAssignmentBuilder withCurrentActiveTaskProcessId(BiFunction currentActiveTaskProcessId) { + this.currentActiveTaskProcessId = Objects.requireNonNull(currentActiveTaskProcessId); + return this; + } + + /** + * Sets a BiFunction which allows to retrieve the current process IDs of a standby task. This is + * used by the state machine to determine if there is still a task on a specific process that is + * not yet revoked. + * + * @param currentStandbyTaskProcessIds A BiFunction which gets the process IDs of a subtopology + * ID / partition ID pair. + * @return This object. + */ + public CurrentAssignmentBuilder withCurrentStandbyTaskProcessIds( + BiFunction> currentStandbyTaskProcessIds + ) { + this.currentStandbyTaskProcessIds = Objects.requireNonNull(currentStandbyTaskProcessIds); + return this; + } + + /** + * Sets a BiFunction which allows to retrieve the current process IDs of a warmup task. This is + * used by the state machine to determine if there is still a task on a specific process that is + * not yet revoked. + * + * @param currentWarmupTaskProcessIds A BiFunction which gets the process IDs of a subtopology ID + * / partition ID pair. + * @return This object. + */ + public CurrentAssignmentBuilder withCurrentWarmupTaskProcessIds(BiFunction> currentWarmupTaskProcessIds) { + this.currentWarmupTaskProcessIds = Objects.requireNonNull(currentWarmupTaskProcessIds); + return this; + } + + /** + * Sets the tasks currently owned by the member. This comes directly from the last StreamsGroupHeartbeat request. This is used to + * determine if the member has revoked the necessary tasks. Passing null into this function means that the member did not provide + * its owned tasks in this heartbeat. + * + * @param ownedAssignment A collection of active, standby and warm-up tasks + * @return This object. + */ + protected CurrentAssignmentBuilder withOwnedAssignment(TasksTuple ownedAssignment) { + this.ownedTasks = Optional.ofNullable(ownedAssignment); + return this; + } + + /** + * Builds the next state for the member or keep the current one if it is not possible to move forward with the current state. + * + * @return A new StreamsGroupMember or the current one. + */ + public StreamsGroupMember build() { + switch (member.state()) { + case STABLE: + // When the member is in the STABLE state, we verify if a newer + // epoch (or target assignment) is available. If it is, we can + // reconcile the member towards it. Otherwise, we return. + if (member.memberEpoch() != targetAssignmentEpoch) { + return computeNextAssignment( + member.memberEpoch(), + member.assignedTasks() + ); + } else { + return member; + } + + case UNREVOKED_TASKS: + // When the member is in the UNREVOKED_TASKS state, we wait + // until the member has revoked the necessary tasks. They are + // considered revoked when they are not anymore reported in the + // owned tasks set in the StreamsGroupHeartbeat API. + + // If the member provides its owned tasks, we verify if it still + // owns any of the revoked tasks. If it did not provide it's + // owned tasks, or we still own some of the revoked tasks, we + // cannot progress. + if ( + ownedTasks.isEmpty() || ownedTasks.get().containsAny(member.tasksPendingRevocation()) + ) { + return member; + } + + // When the member has revoked all the pending tasks, it can + // transition to the next epoch (current + 1) and we can reconcile + // its state towards the latest target assignment. + return computeNextAssignment( + member.memberEpoch() + 1, + member.assignedTasks() + ); + + case UNRELEASED_TASKS: + // When the member is in the UNRELEASED_TASKS, we reconcile the + // member towards the latest target assignment. This will assign any + // of the unreleased tasks when they become available. + return computeNextAssignment( + member.memberEpoch(), + member.assignedTasks() + ); + + case UNKNOWN: + // We could only end up in this state if a new state is added in the + // future and the group coordinator is downgraded. In this case, the + // best option is to fence the member to force it to rejoin the group + // without any tasks and to reconcile it again from scratch. + if ((ownedTasks.isEmpty() || !ownedTasks.get().isEmpty())) { + throw new FencedMemberEpochException( + "The streams group member is in a unknown state. " + + "The member must abandon all its tasks and rejoin."); + } + + return computeNextAssignment( + targetAssignmentEpoch, + member.assignedTasks() + ); + } + + return member; + } + + /** + * Takes the current currentAssignment and the targetAssignment, and generates three + * collections: + * + * - the resultAssignedTasks: the tasks that are assigned in both the current and target + * assignments. + * - the resultTasksPendingRevocation: the tasks that are assigned in the current + * assignment but not in the target assignment. + * - the resultTasksPendingAssignment: the tasks that are assigned in the target assignment but + * not in the current assignment, and can be assigned currently (i.e., they are not owned by + * another member, as defined by the `isUnreleasedTask` predicate). + */ + private boolean computeAssignmentDifference(Map> currentAssignment, + Map> targetAssignment, + Map> resultAssignedTasks, + Map> resultTasksPendingRevocation, + Map> resultTasksPendingAssignment, + BiPredicate isUnreleasedTask) { + boolean hasUnreleasedTasks = false; + + Set allSubtopologyIds = new HashSet<>(targetAssignment.keySet()); + allSubtopologyIds.addAll(currentAssignment.keySet()); + + for (String subtopologyId : allSubtopologyIds) { + hasUnreleasedTasks |= computeAssignmentDifferenceForOneSubtopology( + subtopologyId, + currentAssignment.getOrDefault(subtopologyId, Collections.emptySet()), + targetAssignment.getOrDefault(subtopologyId, Collections.emptySet()), + resultAssignedTasks, + resultTasksPendingRevocation, + resultTasksPendingAssignment, + isUnreleasedTask + ); + } + return hasUnreleasedTasks; + } + + private static boolean computeAssignmentDifferenceForOneSubtopology(final String subtopologyId, + final Set currentTasksForThisSubtopology, + final Set targetTasksForThisSubtopology, + final Map> resultAssignedTasks, + final Map> resultTasksPendingRevocation, + final Map> resultTasksPendingAssignment, + final BiPredicate isUnreleasedTask) { + // Result Assigned Tasks = Current Tasks ∩ Target Tasks + // i.e. we remove all tasks from the current assignment that are not in the target + // assignment + Set resultAssignedTasksForThisSubtopology = new HashSet<>(currentTasksForThisSubtopology); + resultAssignedTasksForThisSubtopology.retainAll(targetTasksForThisSubtopology); + + // Result Tasks Pending Revocation = Current Tasks - Result Assigned Tasks + // i.e. we will ask the member to revoke all tasks in its current assignment that + // are not in the target assignment + Set resultTasksPendingRevocationForThisSubtopology = new HashSet<>(currentTasksForThisSubtopology); + resultTasksPendingRevocationForThisSubtopology.removeAll(resultAssignedTasksForThisSubtopology); + + // Result Tasks Pending Assignment = Target Tasks - Result Assigned Tasks - Unreleased Tasks + // i.e. we will ask the member to assign all tasks in its target assignment, + // except those that are already assigned, and those that are unreleased + Set resultTasksPendingAssignmentForThisSubtopology = new HashSet<>(targetTasksForThisSubtopology); + resultTasksPendingAssignmentForThisSubtopology.removeAll(resultAssignedTasksForThisSubtopology); + boolean hasUnreleasedTasks = resultTasksPendingAssignmentForThisSubtopology.removeIf(taskId -> + isUnreleasedTask.test(subtopologyId, taskId) + ); + + if (!resultAssignedTasksForThisSubtopology.isEmpty()) { + resultAssignedTasks.put(subtopologyId, resultAssignedTasksForThisSubtopology); + } + + if (!resultTasksPendingRevocationForThisSubtopology.isEmpty()) { + resultTasksPendingRevocation.put(subtopologyId, resultTasksPendingRevocationForThisSubtopology); + } + + if (!resultTasksPendingAssignmentForThisSubtopology.isEmpty()) { + resultTasksPendingAssignment.put(subtopologyId, resultTasksPendingAssignmentForThisSubtopology); + } + + return hasUnreleasedTasks; + } + + /** + * Computes the next assignment. + * + * @param memberEpoch The epoch of the member to use. This may be different from + * the epoch in {@link CurrentAssignmentBuilder#member}. + * @param memberAssignedTasks The assigned tasks of the member to use. + * @return A new StreamsGroupMember. + */ + private StreamsGroupMember computeNextAssignment(int memberEpoch, + TasksTuple memberAssignedTasks) { + Map> newActiveAssignedTasks = new HashMap<>(); + Map> newActiveTasksPendingRevocation = new HashMap<>(); + Map> newActiveTasksPendingAssignment = new HashMap<>(); + Map> newStandbyAssignedTasks = new HashMap<>(); + Map> newStandbyTasksPendingRevocation = new HashMap<>(); + Map> newStandbyTasksPendingAssignment = new HashMap<>(); + Map> newWarmupAssignedTasks = new HashMap<>(); + Map> newWarmupTasksPendingRevocation = new HashMap<>(); + Map> newWarmupTasksPendingAssignment = new HashMap<>(); + + boolean hasUnreleasedActiveTasks = computeAssignmentDifference( + memberAssignedTasks.activeTasks(), + targetAssignment.activeTasks(), + newActiveAssignedTasks, + newActiveTasksPendingRevocation, + newActiveTasksPendingAssignment, + (subtopologyId, partitionId) -> + currentActiveTaskProcessId.apply(subtopologyId, partitionId) != null || + currentStandbyTaskProcessIds.apply(subtopologyId, partitionId) + .contains(member.processId()) || + currentWarmupTaskProcessIds.apply(subtopologyId, partitionId) + .contains(member.processId()) + ); + + boolean hasUnreleasedStandbyTasks = computeAssignmentDifference( + memberAssignedTasks.standbyTasks(), + targetAssignment.standbyTasks(), + newStandbyAssignedTasks, + newStandbyTasksPendingRevocation, + newStandbyTasksPendingAssignment, + (subtopologyId, partitionId) -> + Objects.equals(currentActiveTaskProcessId.apply(subtopologyId, partitionId), + member.processId()) || + currentStandbyTaskProcessIds.apply(subtopologyId, partitionId) + .contains(member.processId()) || + currentWarmupTaskProcessIds.apply(subtopologyId, partitionId) + .contains(member.processId()) + ); + + boolean hasUnreleasedWarmupTasks = computeAssignmentDifference( + memberAssignedTasks.warmupTasks(), + targetAssignment.warmupTasks(), + newWarmupAssignedTasks, + newWarmupTasksPendingRevocation, + newWarmupTasksPendingAssignment, + (subtopologyId, partitionId) -> + Objects.equals(currentActiveTaskProcessId.apply(subtopologyId, partitionId), + member.processId()) || + currentStandbyTaskProcessIds.apply(subtopologyId, partitionId) + .contains(member.processId()) || + currentWarmupTaskProcessIds.apply(subtopologyId, partitionId) + .contains(member.processId()) + ); + + return buildNewMember( + memberEpoch, + new TasksTuple( + newActiveTasksPendingRevocation, + newStandbyTasksPendingRevocation, + newWarmupTasksPendingRevocation + ), + new TasksTuple( + newActiveAssignedTasks, + newStandbyAssignedTasks, + newWarmupAssignedTasks + ), + new TasksTuple( + newActiveTasksPendingAssignment, + newStandbyTasksPendingAssignment, + newWarmupTasksPendingAssignment + ), + hasUnreleasedActiveTasks || hasUnreleasedStandbyTasks || hasUnreleasedWarmupTasks + ); + } + + private StreamsGroupMember buildNewMember(final int memberEpoch, + final TasksTuple newTasksPendingRevocation, + final TasksTuple newAssignedTasks, + final TasksTuple newTasksPendingAssignment, + final boolean hasUnreleasedTasks) { + + final boolean hasTasksToBeRevoked = + (!newTasksPendingRevocation.isEmpty()) + && (ownedTasks.isEmpty() || ownedTasks.get().containsAny(newTasksPendingRevocation)); + + if (hasTasksToBeRevoked) { + // If there are tasks to be revoked, the member remains in its current + // epoch and requests the revocation of those tasks. It transitions to + // the UNREVOKED_TASKS state to wait until the client acknowledges the + // revocation of the tasks. + return new StreamsGroupMember.Builder(member) + .setState(MemberState.UNREVOKED_TASKS) + .updateMemberEpoch(memberEpoch) + .setAssignedTasks(newAssignedTasks) + .setTasksPendingRevocation(newTasksPendingRevocation) + .build(); + } else if (!newTasksPendingAssignment.isEmpty()) { + // If there are tasks to be assigned, the member transitions to the + // target epoch and requests the assignment of those tasks. Note that + // the tasks are directly added to the assigned tasks set. The + // member transitions to the STABLE state or to the UNRELEASED_TASKS + // state depending on whether there are unreleased tasks or not. + MemberState newState = + hasUnreleasedTasks + ? MemberState.UNRELEASED_TASKS + : MemberState.STABLE; + return new StreamsGroupMember.Builder(member) + .setState(newState) + .updateMemberEpoch(targetAssignmentEpoch) + .setAssignedTasks(newAssignedTasks.merge(newTasksPendingAssignment)) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + } else if (hasUnreleasedTasks) { + // If there are no tasks to be revoked nor to be assigned but some + // tasks are not available yet, the member transitions to the target + // epoch, to the UNRELEASED_TASKS state and waits. + return new StreamsGroupMember.Builder(member) + .setState(MemberState.UNRELEASED_TASKS) + .updateMemberEpoch(targetAssignmentEpoch) + .setAssignedTasks(newAssignedTasks) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + } else { + // Otherwise, the member transitions to the target epoch and to the + // STABLE state. + return new StreamsGroupMember.Builder(member) + .setState(MemberState.STABLE) + .updateMemberEpoch(targetAssignmentEpoch) + .setAssignedTasks(newAssignedTasks) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + } + } +} diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java index e23df3f5701c4..612e72fabddaa 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMember.java @@ -49,18 +49,8 @@ * @param userEndpoint The user endpoint exposed for Interactive Queries by the Streams client that * contains the member. * @param clientTags Tags of the client of the member used for rack-aware assignment. - * @param assignedActiveTasks Active tasks assigned to the member. - * The key of the map is the subtopology ID and the value is the set of partition IDs. - * @param assignedStandbyTasks Standby tasks assigned to the member. - * The key of the map is the subtopology ID and the value is the set of partition IDs. - * @param assignedWarmupTasks Warm-up tasks assigned to the member. - * The key of the map is the subtopology ID and the value is the set of partition IDs. - * @param activeTasksPendingRevocation Active tasks assigned to the member pending revocation. - * The key of the map is the subtopology ID and the value is the set of partition IDs. - * @param standbyTasksPendingRevocation Standby tasks assigned to the member pending revocation. - * The key of the map is the subtopology ID and the value is the set of partition IDs. - * @param warmupTasksPendingRevocation Warm-up tasks assigned to the member pending revocation. - * The key of the map is the subtopology ID and the value is the set of partition IDs. + * @param assignedTasks Tasks assigned to the member. + * @param tasksPendingRevocation Tasks owned by the member pending revocation. */ @SuppressWarnings("checkstyle:JavaNCSS") public record StreamsGroupMember(String memberId, @@ -76,22 +66,12 @@ public record StreamsGroupMember(String memberId, String processId, Optional userEndpoint, Map clientTags, - Map> assignedActiveTasks, - Map> assignedStandbyTasks, - Map> assignedWarmupTasks, - Map> activeTasksPendingRevocation, - Map> standbyTasksPendingRevocation, - Map> warmupTasksPendingRevocation) { + TasksTuple assignedTasks, + TasksTuple tasksPendingRevocation) { public StreamsGroupMember { Objects.requireNonNull(memberId, "memberId cannot be null"); clientTags = clientTags != null ? Collections.unmodifiableMap(clientTags) : null; - assignedActiveTasks = assignedActiveTasks != null ? Collections.unmodifiableMap(assignedActiveTasks) : null; - assignedStandbyTasks = assignedStandbyTasks != null ? Collections.unmodifiableMap(assignedStandbyTasks) : null; - assignedWarmupTasks = assignedWarmupTasks != null ? Collections.unmodifiableMap(assignedWarmupTasks) : null; - activeTasksPendingRevocation = activeTasksPendingRevocation != null ? Collections.unmodifiableMap(activeTasksPendingRevocation) : null; - standbyTasksPendingRevocation = standbyTasksPendingRevocation != null ? Collections.unmodifiableMap(standbyTasksPendingRevocation) : null; - warmupTasksPendingRevocation = warmupTasksPendingRevocation != null ? Collections.unmodifiableMap(warmupTasksPendingRevocation) : null; } /** @@ -114,12 +94,8 @@ public static class Builder { private String processId = null; private Optional userEndpoint = null; private Map clientTags = null; - private Map> assignedActiveTasks = null; - private Map> assignedStandbyTasks = null; - private Map> assignedWarmupTasks = null; - private Map> activeTasksPendingRevocation = null; - private Map> standbyTasksPendingRevocation = null; - private Map> warmupTasksPendingRevocation = null; + private TasksTuple assignedTasks = null; + private TasksTuple tasksPendingRevocation = null; public Builder(String memberId) { this.memberId = Objects.requireNonNull(memberId, "memberId cannot be null"); @@ -141,12 +117,8 @@ public Builder(StreamsGroupMember member) { this.userEndpoint = member.userEndpoint; this.clientTags = member.clientTags; this.state = member.state; - this.assignedActiveTasks = member.assignedActiveTasks; - this.assignedStandbyTasks = member.assignedStandbyTasks; - this.assignedWarmupTasks = member.assignedWarmupTasks; - this.activeTasksPendingRevocation = member.activeTasksPendingRevocation; - this.standbyTasksPendingRevocation = member.standbyTasksPendingRevocation; - this.warmupTasksPendingRevocation = member.warmupTasksPendingRevocation; + this.assignedTasks = member.assignedTasks; + this.tasksPendingRevocation = member.tasksPendingRevocation; } public Builder updateMemberEpoch(int memberEpoch) { @@ -251,50 +223,13 @@ public Builder maybeUpdateClientTags(Optional> clientTags) { return this; } - public Builder setAssignment(Assignment assignment) { - this.assignedActiveTasks = assignment.activeTasks(); - this.assignedStandbyTasks = assignment.standbyTasks(); - this.assignedWarmupTasks = assignment.warmupTasks(); + public Builder setAssignedTasks(TasksTuple assignedTasks) { + this.assignedTasks = assignedTasks; return this; } - public Builder setAssignedActiveTasks(Map> assignedActiveTasks) { - this.assignedActiveTasks = assignedActiveTasks; - return this; - } - - public Builder setAssignedStandbyTasks(Map> assignedStandbyTasks) { - this.assignedStandbyTasks = assignedStandbyTasks; - return this; - } - - public Builder setAssignedWarmupTasks(Map> assignedWarmupTasks) { - this.assignedWarmupTasks = assignedWarmupTasks; - return this; - } - - public Builder setAssignmentPendingRevocation(Assignment assignment) { - this.activeTasksPendingRevocation = assignment.activeTasks(); - this.standbyTasksPendingRevocation = assignment.standbyTasks(); - this.warmupTasksPendingRevocation = assignment.warmupTasks(); - return this; - } - - public Builder setActiveTasksPendingRevocation( - Map> activeTasksPendingRevocation) { - this.activeTasksPendingRevocation = activeTasksPendingRevocation; - return this; - } - - public Builder setStandbyTasksPendingRevocation( - Map> standbyTasksPendingRevocation) { - this.standbyTasksPendingRevocation = standbyTasksPendingRevocation; - return this; - } - - public Builder setWarmupTasksPendingRevocation( - Map> warmupTasksPendingRevocation) { - this.warmupTasksPendingRevocation = warmupTasksPendingRevocation; + public Builder setTasksPendingRevocation(TasksTuple tasksPendingRevocation) { + this.tasksPendingRevocation = tasksPendingRevocation; return this; } @@ -318,15 +253,20 @@ public Builder updateWith(StreamsGroupCurrentMemberAssignmentValue record) { setMemberEpoch(record.memberEpoch()); setPreviousMemberEpoch(record.previousMemberEpoch()); setState(MemberState.fromValue(record.state())); - setAssignedActiveTasks(assignmentFromTaskIds(record.activeTasks())); - setAssignedStandbyTasks(assignmentFromTaskIds(record.standbyTasks())); - setAssignedWarmupTasks(assignmentFromTaskIds(record.warmupTasks())); - setActiveTasksPendingRevocation( - assignmentFromTaskIds(record.activeTasksPendingRevocation())); - setStandbyTasksPendingRevocation( - assignmentFromTaskIds(record.standbyTasksPendingRevocation())); - setWarmupTasksPendingRevocation( - assignmentFromTaskIds(record.warmupTasksPendingRevocation())); + setAssignedTasks( + new TasksTuple( + assignmentFromTaskIds(record.activeTasks()), + assignmentFromTaskIds(record.standbyTasks()), + assignmentFromTaskIds(record.warmupTasks()) + ) + ); + setTasksPendingRevocation( + new TasksTuple( + assignmentFromTaskIds(record.activeTasksPendingRevocation()), + assignmentFromTaskIds(record.standbyTasksPendingRevocation()), + assignmentFromTaskIds(record.warmupTasksPendingRevocation()) + ) + ); return this; } @@ -353,12 +293,8 @@ public StreamsGroupMember build() { processId, userEndpoint, clientTags, - assignedActiveTasks, - assignedStandbyTasks, - assignedWarmupTasks, - activeTasksPendingRevocation, - standbyTasksPendingRevocation, - warmupTasksPendingRevocation + assignedTasks, + tasksPendingRevocation ); } } @@ -377,9 +313,7 @@ public boolean isReconciledTo(int targetAssignmentEpoch) { * * @return The StreamsGroupMember mapped as StreamsGroupDescribeResponseData.Member. */ - public StreamsGroupDescribeResponseData.Member asStreamsGroupDescribeMember( - Assignment targetAssignment - ) { + public StreamsGroupDescribeResponseData.Member asStreamsGroupDescribeMember(TasksTuple targetAssignment) { final StreamsGroupDescribeResponseData.Assignment describedTargetAssignment = new StreamsGroupDescribeResponseData.Assignment(); @@ -395,9 +329,9 @@ public StreamsGroupDescribeResponseData.Member asStreamsGroupDescribeMember( .setMemberId(memberId) .setAssignment( new StreamsGroupDescribeResponseData.Assignment() - .setActiveTasks(taskIdsFromMap(assignedActiveTasks)) - .setStandbyTasks(taskIdsFromMap(assignedStandbyTasks)) - .setWarmupTasks(taskIdsFromMap(assignedWarmupTasks))) + .setActiveTasks(taskIdsFromMap(assignedTasks.activeTasks())) + .setStandbyTasks(taskIdsFromMap(assignedTasks.standbyTasks())) + .setWarmupTasks(taskIdsFromMap(assignedTasks.warmupTasks()))) .setTargetAssignment(describedTargetAssignment) .setClientHost(clientHost) .setClientId(clientId) @@ -419,9 +353,7 @@ public StreamsGroupDescribeResponseData.Member asStreamsGroupDescribeMember( ); } - private static List taskIdsFromMap( - Map> tasks - ) { + private static List taskIdsFromMap(Map> tasks) { List taskIds = new ArrayList<>(); tasks.forEach((subtopologyId, partitionSet) -> { taskIds.add(new StreamsGroupDescribeResponseData.TaskIds() @@ -432,32 +364,9 @@ private static List taskIdsFromMap( } /** - * @return True if the two provided members have different assigned active tasks. - */ - public static boolean hasAssignedActiveTasksChanged( - StreamsGroupMember member1, - StreamsGroupMember member2 - ) { - return !member1.assignedActiveTasks().equals(member2.assignedActiveTasks()); - } - - /** - * @return True if the two provided members have different assigned active tasks. - */ - public static boolean hasAssignedStandbyTasksChanged( - StreamsGroupMember member1, - StreamsGroupMember member2 - ) { - return !member1.assignedStandbyTasks().equals(member2.assignedStandbyTasks()); - } - - /** - * @return True if the two provided members have different assigned active tasks. + * @return True if the two provided members have different assigned tasks. */ - public static boolean hasAssignedWarmupTasksChanged( - StreamsGroupMember member1, - StreamsGroupMember member2 - ) { - return !member1.assignedWarmupTasks().equals(member2.assignedWarmupTasks()); + public static boolean hasAssignedTasksChanged(StreamsGroupMember member1, StreamsGroupMember member2) { + return !member1.assignedTasks().equals(member2.assignedTasks()); } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/Assignment.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java similarity index 50% rename from group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/Assignment.java rename to group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java index da377d19ccd52..ea4c6f81a52f6 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/Assignment.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TasksTuple.java @@ -19,6 +19,7 @@ import org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMemberValue; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Map; import java.util.Objects; @@ -26,45 +27,89 @@ import java.util.stream.Collectors; /** - * An immutable assignment for a member. + * An immutable tuple containing active, standby and warm-up tasks. * - * @param activeTasks Active tasks assigned to the member. + * @param activeTasks Active tasks. * The key of the map is the subtopology ID and the value is the set of partition IDs. - * @param standbyTasks Standby tasks assigned to the member. + * @param standbyTasks Standby tasks. * The key of the map is the subtopology ID and the value is the set of partition IDs. - * @param warmupTasks Warm-up tasks assigned to the member. + * @param warmupTasks Warm-up tasks. * The key of the map is the subtopology ID and the value is the set of partition IDs. */ -public record Assignment(Map> activeTasks, +public record TasksTuple(Map> activeTasks, Map> standbyTasks, Map> warmupTasks) { - public Assignment { + public TasksTuple { activeTasks = Collections.unmodifiableMap(Objects.requireNonNull(activeTasks)); standbyTasks = Collections.unmodifiableMap(Objects.requireNonNull(standbyTasks)); warmupTasks = Collections.unmodifiableMap(Objects.requireNonNull(warmupTasks)); } /** - * An empty assignment. + * An empty task tuple. */ - public static final Assignment EMPTY = new Assignment( + public static final TasksTuple EMPTY = new TasksTuple( Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap() ); /** - * Creates a {{@link org.apache.kafka.coordinator.group.streams.Assignment}} from a + * @return true if all collections in the tuple are empty. + */ + public boolean isEmpty() { + return activeTasks.isEmpty() && standbyTasks.isEmpty() && warmupTasks.isEmpty(); + } + + /** + * Merges this task tuple with another task tuple. + * + * @param other The other task tuple. + * @return A new task tuple, containing all active tasks, standby tasks and warm-up tasks from both tuples. + */ + public TasksTuple merge(TasksTuple other) { + Map> mergedActiveTasks = merge(activeTasks, other.activeTasks); + Map> mergedStandbyTasks = merge(standbyTasks, other.standbyTasks); + Map> mergedWarmupTasks = merge(warmupTasks, other.warmupTasks); + return new TasksTuple(mergedActiveTasks, mergedStandbyTasks, mergedWarmupTasks); + } + + private static Map> merge(final Map> tasks1, final Map> tasks2) { + HashMap> result = new HashMap<>(); + tasks1.forEach((subtopologyId, tasks) -> + result.put(subtopologyId, new HashSet<>(tasks))); + tasks2.forEach((subtopologyId, tasks) -> result + .computeIfAbsent(subtopologyId, __ -> new HashSet<>()) + .addAll(tasks)); + return result; + } + + /** + * Checks if this task tuple contains any of the tasks in another task tuple. + * + * @param other The other task tuple. + * @return true if there is at least one active, standby or warm-up task that is present in both tuples. + */ + public boolean containsAny(TasksTuple other) { + return activeTasks.entrySet().stream().anyMatch( + entry -> other.activeTasks.containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.activeTasks.get(entry.getKey())) + ) || standbyTasks.entrySet().stream().anyMatch( + entry -> other.standbyTasks.containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.standbyTasks.get(entry.getKey())) + ) || warmupTasks.entrySet().stream().anyMatch( + entry -> other.warmupTasks.containsKey(entry.getKey()) && !Collections.disjoint(entry.getValue(), other.warmupTasks.get(entry.getKey())) + ); + } + + /** + * Creates a {{@link TasksTuple}} from a * {{@link org.apache.kafka.coordinator.group.generated.StreamsGroupTargetAssignmentMemberValue}}. * * @param record The record. - * @return A {{@link org.apache.kafka.coordinator.group.streams.Assignment}}. + * @return A {{@link TasksTuple}}. */ - public static Assignment fromRecord( - StreamsGroupTargetAssignmentMemberValue record - ) { - return new Assignment( + public static TasksTuple fromTargetAssignmentRecord(StreamsGroupTargetAssignmentMemberValue record) { + return new TasksTuple( record.activeTasks().stream() .collect(Collectors.toMap( StreamsGroupTargetAssignmentMemberValue.TaskIds::subtopologyId, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/GroupAssignment.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/GroupAssignment.java index a97cdc33b7909..c4ac5803b35d0 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/GroupAssignment.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/assignor/GroupAssignment.java @@ -22,7 +22,7 @@ /** * The task assignment for a streams group. * - * @param members The member assignments keyed by member id. + * @param members The member assignments keyed by member ID. */ public record GroupAssignment(Map members) { diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java new file mode 100644 index 0000000000000..4d55a8419fb52 --- /dev/null +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/CurrentAssignmentBuilderTest.java @@ -0,0 +1,825 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.coordinator.group.streams; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.FencedMemberEpochException; +import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import java.util.Collections; + +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; +import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksTuple; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class CurrentAssignmentBuilderTest { + + private static final String SUBTOPOLOGY_ID1 = Uuid.randomUuid().toString(); + private static final String SUBTOPOLOGY_ID2 = Uuid.randomUuid().toString(); + private static final String PROCESS_ID = "process_id"; + private static final String MEMBER_NAME = "member"; + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testStableToStable(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks( + mkTasksTuple( + taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple( + taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testStableToStableAtTargetEpoch(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks( + mkTasksTuple( + taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple( + taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testStableToStableWithNewTasks(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2, 4), + mkTasks(SUBTOPOLOGY_ID2, 3, 4, 7))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2, 4), + mkTasks(SUBTOPOLOGY_ID2, 3, 4, 7))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testStableToUnrevokedTasks(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 4, 5))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2), + mkTasks(SUBTOPOLOGY_ID2, 4))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1), + mkTasks(SUBTOPOLOGY_ID2, 3))) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testStableToUnrevokedWithEmptyAssignment(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks( + mkTasksTuple( + taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, TasksTuple.EMPTY) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(TasksTuple.EMPTY) + .setTasksPendingRevocation( + mkTasksTuple( + taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testStableToUnreleasedTasks(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2, 4), + mkTasks(SUBTOPOLOGY_ID2, 3, 4, 7))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testStableToUnreleasedTasksWithOwnedTasksNotHavingRevokedTasks(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 4))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3, 5))) + .withCurrentActiveTaskProcessId((subtopologyId, __) -> + SUBTOPOLOGY_ID2.equals(subtopologyId) ? PROCESS_ID : null + ) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .withOwnedAssignment(mkTasksTuple(taskRole)) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2), + mkTasks(SUBTOPOLOGY_ID2, 3))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnrevokedTasksToStable(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1), + mkTasks(SUBTOPOLOGY_ID2, 4))) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .withOwnedAssignment(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testRemainsInUnrevokedTasks(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1), + mkTasks(SUBTOPOLOGY_ID2, 4))) + .build(); + + CurrentAssignmentBuilder currentAssignmentBuilder = new CurrentAssignmentBuilder( + member) + .withTargetAssignment(memberEpoch + 2, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()); + + assertEquals( + member, + currentAssignmentBuilder + .withOwnedAssignment(null) + .build() + ); + + assertEquals( + member, + currentAssignmentBuilder + .withOwnedAssignment(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .build() + ); + + assertEquals( + member, + currentAssignmentBuilder + .withOwnedAssignment(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 4, 5, 6))) + .build() + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnrevokedTasksToUnrevokedTasks(TaskRole taskRole) { + final int memberEpoch = 10; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1), + mkTasks(SUBTOPOLOGY_ID2, 4))) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 2, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withOwnedAssignment(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2), + mkTasks(SUBTOPOLOGY_ID2, 5))) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnrevokedTasksToUnreleasedTasks(TaskRole taskRole) { + final int memberEpoch = 11; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch - 1) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 1), + mkTasks(SUBTOPOLOGY_ID2, 4))) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3, 4), + mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .withOwnedAssignment(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6)) + ) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnreleasedTasksToStable(TaskRole taskRole) { + final int memberEpoch = 11; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId("process1") + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.singleton(PROCESS_ID)) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> + Collections.singleton(PROCESS_ID)) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId("process1") + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnreleasedTasksToStableWithNewTasks(TaskRole taskRole) { + int memberEpoch = 11; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId("process1") + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3, 4), + mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId("process1") + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3, 4), + mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnreleasedTasksToUnreleasedTasks(TaskRole taskRole) { + int memberEpoch = 11; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3, 4), + mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.singleton(PROCESS_ID)) + .withCurrentWarmupTaskProcessIds( + (subtopologyId, partitionId) -> Collections.singleton(PROCESS_ID)) + .build(); + + assertEquals(member, updatedMember); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnreleasedTasksToUnreleasedTasksOtherUnreleasedTaskRole(TaskRole taskRole) { + int memberEpoch = 11; + + // The unreleased task is owned by a task of a different role on the same process. + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3, 4), + mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> null) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> (taskRole == TaskRole.STANDBY) + ? Collections.emptySet() : Collections.singleton(PROCESS_ID)) + .withCurrentWarmupTaskProcessIds( + (subtopologyId, partitionId) -> (taskRole == TaskRole.STANDBY) + ? Collections.singleton(PROCESS_ID) : Collections.emptySet()) + .build(); + + assertEquals(member, updatedMember); + } + + @Test + public void testUnreleasedTasksToUnreleasedTasksAnyActiveOwner() { + int memberEpoch = 11; + + // The unreleased task remains unreleased, because it is owned by any other instance in + // an active role, no matter the process. + // The task that is not unreleased can be assigned. + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .build(); + + StreamsGroupMember expectedMember = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch, mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 2, 3, 4), + mkTasks(SUBTOPOLOGY_ID2, 5, 6, 7))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> + (subtopologyId.equals(SUBTOPOLOGY_ID1) && partitionId == 4) ? "anyOtherProcess" + : null) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals(expectedMember, updatedMember); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnreleasedTasksToUnrevokedTasks(TaskRole taskRole) { + int memberEpoch = 11; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNRELEASED_TASKS) + .setProcessId("process1") + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2, 3), + mkTasks(SUBTOPOLOGY_ID2, 5, 6))) + .setTasksPendingRevocation(mkTasksTuple(TaskRole.ACTIVE, + mkTasks(SUBTOPOLOGY_ID1, 4), + mkTasks(SUBTOPOLOGY_ID2, 7))) + .build(); + + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNREVOKED_TASKS) + .setProcessId("process1") + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2), + mkTasks(SUBTOPOLOGY_ID2, 5))) + .build(), + updatedMember + ); + } + + @ParameterizedTest + @EnumSource(TaskRole.class) + public void testUnknownState(TaskRole taskRole) { + int memberEpoch = 11; + + StreamsGroupMember member = new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.UNKNOWN) + .setMemberEpoch(memberEpoch) + .setPreviousMemberEpoch(memberEpoch) + .setProcessId(PROCESS_ID) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .setTasksPendingRevocation(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 2), + mkTasks(SUBTOPOLOGY_ID2, 5))) + .build(); + + // When the member is in an unknown state, the member is first to force + // a reset of the client side member state. + assertThrows(FencedMemberEpochException.class, () -> new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .build()); + + // Then the member rejoins with no owned tasks. + StreamsGroupMember updatedMember = new CurrentAssignmentBuilder(member) + .withTargetAssignment(memberEpoch + 1, mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .withCurrentActiveTaskProcessId((subtopologyId, partitionId) -> PROCESS_ID) + .withCurrentStandbyTaskProcessIds( + (subtopologyId, partitionId) -> Collections.emptySet()) + .withCurrentWarmupTaskProcessIds((subtopologyId, partitionId) -> Collections.emptySet()) + .withOwnedAssignment(mkTasksTuple(taskRole)) + .build(); + + assertEquals( + new StreamsGroupMember.Builder(MEMBER_NAME) + .setState(MemberState.STABLE) + .setProcessId(PROCESS_ID) + .setMemberEpoch(memberEpoch + 1) + .setPreviousMemberEpoch(memberEpoch) + .setAssignedTasks(mkTasksTuple(taskRole, + mkTasks(SUBTOPOLOGY_ID1, 3), + mkTasks(SUBTOPOLOGY_ID2, 6))) + .setTasksPendingRevocation(TasksTuple.EMPTY) + .build(), + updatedMember + ); + } +} diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java index 8c6d3d9088aba..f6c33df6f130b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupMemberTest.java @@ -25,14 +25,12 @@ import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalInt; -import java.util.Set; import java.util.stream.Collectors; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -40,8 +38,10 @@ import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public class StreamsGroupMemberTest { @@ -70,12 +70,18 @@ public class StreamsGroupMemberTest { private static final List TASKS4 = List.of(3, 2, 1); private static final List TASKS5 = List.of(6, 5, 4); private static final List TASKS6 = List.of(9, 7); - private static final Map> ASSIGNED_ACTIVE_TASKS = mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS1.toArray(Integer[]::new))); - private static final Map> ASSIGNED_STANDBY_TASKS = mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS2.toArray(Integer[]::new))); - private static final Map> ASSIGNED_WARMUP_TASKS = mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS3.toArray(Integer[]::new))); - private static final Map> ACTIVE_TASKS_PENDING_REVOCATION = mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS4.toArray(Integer[]::new))); - private static final Map> STANDBY_TASKS_PENDING_REVOCATION = mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS5.toArray(Integer[]::new))); - private static final Map> WARMUP_TASKS_PENDING_REVOCATION = mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS6.toArray(Integer[]::new))); + private static final TasksTuple ASSIGNED_TASKS = + new TasksTuple( + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS1.toArray(Integer[]::new))), + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS2.toArray(Integer[]::new))), + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS3.toArray(Integer[]::new))) + ); + private static final TasksTuple TASKS_PENDING_REVOCATION = + new TasksTuple( + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS4.toArray(Integer[]::new))), + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY1, TASKS5.toArray(Integer[]::new))), + mkTasksPerSubtopology(mkTasks(SUBTOPOLOGY2, TASKS6.toArray(Integer[]::new))) + ); @Test public void testBuilderWithMemberIdIsNull() { @@ -112,12 +118,8 @@ public void testBuilderWithDefaults() { assertNull(member.processId()); assertNull(member.userEndpoint()); assertNull(member.clientTags()); - assertNull(member.assignedActiveTasks()); - assertNull(member.assignedStandbyTasks()); - assertNull(member.assignedWarmupTasks()); - assertNull(member.activeTasksPendingRevocation()); - assertNull(member.standbyTasksPendingRevocation()); - assertNull(member.warmupTasksPendingRevocation()); + assertNull(member.assignedTasks()); + assertNull(member.tasksPendingRevocation()); } @Test @@ -136,12 +138,8 @@ public void testBuilderNewMember() { assertEquals(PROCESS_ID, member.processId()); assertEquals(Optional.of(USER_ENDPOINT), member.userEndpoint()); assertEquals(CLIENT_TAGS, member.clientTags()); - assertEquals(ASSIGNED_ACTIVE_TASKS, member.assignedActiveTasks()); - assertEquals(ASSIGNED_STANDBY_TASKS, member.assignedStandbyTasks()); - assertEquals(ASSIGNED_WARMUP_TASKS, member.assignedWarmupTasks()); - assertEquals(ACTIVE_TASKS_PENDING_REVOCATION, member.activeTasksPendingRevocation()); - assertEquals(STANDBY_TASKS_PENDING_REVOCATION, member.standbyTasksPendingRevocation()); - assertEquals(WARMUP_TASKS_PENDING_REVOCATION, member.warmupTasksPendingRevocation()); + assertEquals(ASSIGNED_TASKS, member.assignedTasks()); + assertEquals(TASKS_PENDING_REVOCATION, member.tasksPendingRevocation()); } @Test @@ -179,12 +177,8 @@ public void testBuilderUpdateWithStreamsGroupMemberMetadataValue() { assertNull(member.memberEpoch()); assertNull(member.previousMemberEpoch()); assertNull(member.state()); - assertNull(member.assignedActiveTasks()); - assertNull(member.assignedStandbyTasks()); - assertNull(member.assignedWarmupTasks()); - assertNull(member.activeTasksPendingRevocation()); - assertNull(member.standbyTasksPendingRevocation()); - assertNull(member.warmupTasksPendingRevocation()); + assertNull(member.assignedTasks()); + assertNull(member.tasksPendingRevocation()); } @Test @@ -208,12 +202,8 @@ public void testBuilderUpdateWithConsumerGroupCurrentMemberAssignmentValue() { assertEquals(record.memberEpoch(), member.memberEpoch()); assertEquals(record.previousMemberEpoch(), member.previousMemberEpoch()); assertEquals(MemberState.fromValue(record.state()), member.state()); - assertEquals(ASSIGNED_ACTIVE_TASKS, member.assignedActiveTasks()); - assertEquals(ASSIGNED_STANDBY_TASKS, member.assignedStandbyTasks()); - assertEquals(ASSIGNED_WARMUP_TASKS, member.assignedWarmupTasks()); - assertEquals(ACTIVE_TASKS_PENDING_REVOCATION, member.activeTasksPendingRevocation()); - assertEquals(STANDBY_TASKS_PENDING_REVOCATION, member.standbyTasksPendingRevocation()); - assertEquals(WARMUP_TASKS_PENDING_REVOCATION, member.warmupTasksPendingRevocation()); + assertEquals(ASSIGNED_TASKS, member.assignedTasks()); + assertEquals(TASKS_PENDING_REVOCATION, member.tasksPendingRevocation()); assertNull(member.instanceId()); assertNull(member.rackId()); assertNull(member.rebalanceTimeoutMs()); @@ -275,12 +265,8 @@ public void testBuilderMaybeUpdateMember() { assertEquals(member.state(), updatedMember.state()); assertEquals(member.clientId(), updatedMember.clientId()); assertEquals(member.clientHost(), updatedMember.clientHost()); - assertEquals(member.assignedActiveTasks(), updatedMember.assignedActiveTasks()); - assertEquals(member.assignedStandbyTasks(), updatedMember.assignedStandbyTasks()); - assertEquals(member.assignedWarmupTasks(), updatedMember.assignedWarmupTasks()); - assertEquals(member.activeTasksPendingRevocation(), updatedMember.activeTasksPendingRevocation()); - assertEquals(member.standbyTasksPendingRevocation(), updatedMember.standbyTasksPendingRevocation()); - assertEquals(member.warmupTasksPendingRevocation(), updatedMember.warmupTasksPendingRevocation()); + assertEquals(member.assignedTasks(), updatedMember.assignedTasks()); + assertEquals(member.tasksPendingRevocation(), updatedMember.tasksPendingRevocation()); } @Test @@ -306,25 +292,8 @@ public void testBuilderUpdateMemberEpoch() { assertEquals(member.processId(), updatedMember.processId()); assertEquals(member.userEndpoint(), updatedMember.userEndpoint()); assertEquals(member.clientTags(), updatedMember.clientTags()); - assertEquals(member.assignedActiveTasks(), updatedMember.assignedActiveTasks()); - assertEquals(member.assignedStandbyTasks(), updatedMember.assignedStandbyTasks()); - assertEquals(member.assignedWarmupTasks(), updatedMember.assignedWarmupTasks()); - assertEquals(member.activeTasksPendingRevocation(), updatedMember.activeTasksPendingRevocation()); - assertEquals(member.standbyTasksPendingRevocation(), updatedMember.standbyTasksPendingRevocation()); - assertEquals(member.warmupTasksPendingRevocation(), updatedMember.warmupTasksPendingRevocation()); - } - - @Test - public void testReturnUnmodifiableFields() { - final StreamsGroupMember member = createStreamsGroupMember(); - - assertThrows(UnsupportedOperationException.class, () -> member.clientTags().put("not allowed", "")); - assertThrows(UnsupportedOperationException.class, () -> member.assignedActiveTasks().put("not allowed", Collections.emptySet())); - assertThrows(UnsupportedOperationException.class, () -> member.assignedStandbyTasks().put("not allowed", Collections.emptySet())); - assertThrows(UnsupportedOperationException.class, () -> member.assignedWarmupTasks().put("not allowed", Collections.emptySet())); - assertThrows(UnsupportedOperationException.class, () -> member.activeTasksPendingRevocation().put("not allowed", Collections.emptySet())); - assertThrows(UnsupportedOperationException.class, () -> member.standbyTasksPendingRevocation().put("not allowed", Collections.emptySet())); - assertThrows(UnsupportedOperationException.class, () -> member.warmupTasksPendingRevocation().put("not allowed", Collections.emptySet())); + assertEquals(member.assignedTasks(), updatedMember.assignedTasks()); + assertEquals(member.tasksPendingRevocation(), updatedMember.tasksPendingRevocation()); } @Test @@ -333,7 +302,7 @@ public void testAsStreamsGroupDescribeMember() { List assignedTasks1 = Arrays.asList(10, 11, 12); List assignedTasks2 = Arrays.asList(13, 14, 15); List assignedTasks3 = Arrays.asList(16, 17, 18); - Assignment targetAssignment = new Assignment( + TasksTuple targetAssignment = new TasksTuple( mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(assignedTasks3))), mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(assignedTasks2))), mkMap(mkEntry(SUBTOPOLOGY3, new HashSet<>(assignedTasks1))) @@ -404,6 +373,45 @@ public void testAsStreamsGroupDescribeWithTargetAssignmentNull() { assertEquals(new StreamsGroupDescribeResponseData.Assignment(), streamsGroupDescribeMember.targetAssignment()); } + @Test + public void testHasAssignedTasksChanged() { + StreamsGroupMember member1 = new StreamsGroupMember.Builder(MEMBER_ID) + .setAssignedTasks(new TasksTuple( + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), + mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) + )) + .build(); + + StreamsGroupMember member2 = new StreamsGroupMember.Builder(MEMBER_ID) + .setAssignedTasks(new TasksTuple( + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS4))), + mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS5))), + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS6))) + )) + .build(); + + assertTrue(StreamsGroupMember.hasAssignedTasksChanged(member1, member2)); + + StreamsGroupMember member3 = new StreamsGroupMember.Builder(MEMBER_ID) + .setAssignedTasks(new TasksTuple( + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), + mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) + )) + .build(); + + StreamsGroupMember member4 = new StreamsGroupMember.Builder(MEMBER_ID) + .setAssignedTasks(new TasksTuple( + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS1))), + mkMap(mkEntry(SUBTOPOLOGY2, new HashSet<>(TASKS2))), + mkMap(mkEntry(SUBTOPOLOGY1, new HashSet<>(TASKS3))) + )) + .build(); + + assertFalse(StreamsGroupMember.hasAssignedTasksChanged(member3, member4)); + } + private StreamsGroupMember createStreamsGroupMember() { return new StreamsGroupMember.Builder(MEMBER_ID) .setMemberEpoch(MEMBER_EPOCH) @@ -418,12 +426,8 @@ private StreamsGroupMember createStreamsGroupMember() { .setProcessId(PROCESS_ID) .setUserEndpoint(USER_ENDPOINT) .setClientTags(CLIENT_TAGS) - .setAssignedActiveTasks(ASSIGNED_ACTIVE_TASKS) - .setAssignedStandbyTasks(ASSIGNED_STANDBY_TASKS) - .setAssignedWarmupTasks(ASSIGNED_WARMUP_TASKS) - .setActiveTasksPendingRevocation(ACTIVE_TASKS_PENDING_REVOCATION) - .setStandbyTasksPendingRevocation(STANDBY_TASKS_PENDING_REVOCATION) - .setWarmupTasksPendingRevocation(WARMUP_TASKS_PENDING_REVOCATION) + .setAssignedTasks(ASSIGNED_TASKS) + .setTasksPendingRevocation(TASKS_PENDING_REVOCATION) .build(); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java index 47668ec84c0f0..f633fec80f7c1 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TaskAssignmentTestUtil.java @@ -17,24 +17,27 @@ package org.apache.kafka.coordinator.group.streams; import java.util.AbstractMap; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; public class TaskAssignmentTestUtil { - public static Assignment mkAssignment(final Map> activeTasks, - final Map> standbyTasks, - final Map> warmupTasks) { - return new Assignment( - Collections.unmodifiableMap(Objects.requireNonNull(activeTasks)), - Collections.unmodifiableMap(Objects.requireNonNull(standbyTasks)), - Collections.unmodifiableMap(Objects.requireNonNull(warmupTasks)) - ); + public enum TaskRole { + ACTIVE, + STANDBY, + WARMUP + } + + @SafeVarargs + public static TasksTuple mkTasksTuple(TaskRole taskRole, Map.Entry>... entries) { + return switch (taskRole) { + case ACTIVE -> new TasksTuple(mkTasksPerSubtopology(entries), new HashMap<>(), new HashMap<>()); + case STANDBY -> new TasksTuple(new HashMap<>(), mkTasksPerSubtopology(entries), new HashMap<>()); + case WARMUP -> new TasksTuple(new HashMap<>(), new HashMap<>(), mkTasksPerSubtopology(entries)); + }; } public static Map.Entry> mkTasks(String subtopologyId, @@ -46,8 +49,7 @@ public static Map.Entry> mkTasks(String subtopologyId, } @SafeVarargs - public static Map> mkTasksPerSubtopology(Map.Entry>... entries) { + public static Map> mkTasksPerSubtopology(Map.Entry>... entries) { Map> assignment = new HashMap<>(); for (Map.Entry> entry : entries) { assignment.put(entry.getKey(), entry.getValue()); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/AssignmentTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java similarity index 57% rename from group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/AssignmentTest.java rename to group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java index 7c0baf273648e..73c43a6d088aa 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/AssignmentTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TasksTupleTest.java @@ -30,19 +30,21 @@ import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasks; import static org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.mkTasksPerSubtopology; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; -public class AssignmentTest { +public class TasksTupleTest { - static final String SUBTOPOLOGY_1 = "subtopology1"; - static final String SUBTOPOLOGY_2 = "subtopology2"; - static final String SUBTOPOLOGY_3 = "subtopology3"; + private static final String SUBTOPOLOGY_1 = "subtopology1"; + private static final String SUBTOPOLOGY_2 = "subtopology2"; + private static final String SUBTOPOLOGY_3 = "subtopology3"; @Test public void testTasksCannotBeNull() { - assertThrows(NullPointerException.class, () -> new Assignment(null, Collections.emptyMap(), Collections.emptyMap())); - assertThrows(NullPointerException.class, () -> new Assignment(Collections.emptyMap(), null, Collections.emptyMap())); - assertThrows(NullPointerException.class, () -> new Assignment(Collections.emptyMap(), Collections.emptyMap(), null)); + assertThrows(NullPointerException.class, () -> new TasksTuple(null, Collections.emptyMap(), Collections.emptyMap())); + assertThrows(NullPointerException.class, () -> new TasksTuple(Collections.emptyMap(), null, Collections.emptyMap())); + assertThrows(NullPointerException.class, () -> new TasksTuple(Collections.emptyMap(), Collections.emptyMap(), null)); } @Test @@ -56,14 +58,14 @@ public void testReturnUnmodifiableTaskAssignments() { Map> warmupTasks = mkTasksPerSubtopology( mkTasks(SUBTOPOLOGY_3, 4, 5, 6) ); - Assignment assignment = new Assignment(activeTasks, standbyTasks, warmupTasks); - - assertEquals(activeTasks, assignment.activeTasks()); - assertThrows(UnsupportedOperationException.class, () -> assignment.activeTasks().put("not allowed", Collections.emptySet())); - assertEquals(standbyTasks, assignment.standbyTasks()); - assertThrows(UnsupportedOperationException.class, () -> assignment.standbyTasks().put("not allowed", Collections.emptySet())); - assertEquals(warmupTasks, assignment.warmupTasks()); - assertThrows(UnsupportedOperationException.class, () -> assignment.warmupTasks().put("not allowed", Collections.emptySet())); + TasksTuple tuple = new TasksTuple(activeTasks, standbyTasks, warmupTasks); + + assertEquals(activeTasks, tuple.activeTasks()); + assertThrows(UnsupportedOperationException.class, () -> tuple.activeTasks().put("not allowed", Collections.emptySet())); + assertEquals(standbyTasks, tuple.standbyTasks()); + assertThrows(UnsupportedOperationException.class, () -> tuple.standbyTasks().put("not allowed", Collections.emptySet())); + assertEquals(warmupTasks, tuple.warmupTasks()); + assertThrows(UnsupportedOperationException.class, () -> tuple.warmupTasks().put("not allowed", Collections.emptySet())); } @Test @@ -95,28 +97,83 @@ public void testFromTargetAssignmentRecord() { .setStandbyTasks(standbyTasks) .setWarmupTasks(warmupTasks); - Assignment assignment = Assignment.fromRecord(record); + TasksTuple tuple = TasksTuple.fromTargetAssignmentRecord(record); assertEquals( mkTasksPerSubtopology( mkTasks(SUBTOPOLOGY_1, 1, 2, 3), mkTasks(SUBTOPOLOGY_2, 4, 5, 6) ), - assignment.activeTasks() + tuple.activeTasks() ); assertEquals( mkTasksPerSubtopology( mkTasks(SUBTOPOLOGY_1, 7, 8, 9), mkTasks(SUBTOPOLOGY_2, 1, 2, 3) ), - assignment.standbyTasks() + tuple.standbyTasks() ); assertEquals( mkTasksPerSubtopology( mkTasks(SUBTOPOLOGY_1, 4, 5, 6), mkTasks(SUBTOPOLOGY_2, 7, 8, 9) ), - assignment.warmupTasks() + tuple.warmupTasks() + ); + } + + @Test + public void testMerge() { + TasksTuple tuple1 = new TasksTuple( + Map.of(SUBTOPOLOGY_1, Set.of(1, 2, 3)), + Map.of(SUBTOPOLOGY_2, Set.of(4, 5, 6)), + Map.of(SUBTOPOLOGY_3, Set.of(7, 8, 9)) + ); + + TasksTuple tuple2 = new TasksTuple( + Map.of(SUBTOPOLOGY_1, Set.of(10, 11)), + Map.of(SUBTOPOLOGY_2, Set.of(12, 13)), + Map.of(SUBTOPOLOGY_3, Set.of(14, 15)) ); + + TasksTuple mergedTuple = tuple1.merge(tuple2); + + assertEquals(Map.of(SUBTOPOLOGY_1, Set.of(1, 2, 3, 10, 11)), mergedTuple.activeTasks()); + assertEquals(Map.of(SUBTOPOLOGY_2, Set.of(4, 5, 6, 12, 13)), mergedTuple.standbyTasks()); + assertEquals(Map.of(SUBTOPOLOGY_3, Set.of(7, 8, 9, 14, 15)), mergedTuple.warmupTasks()); + } + + @Test + public void testContainsAny() { + TasksTuple tuple1 = new TasksTuple( + Map.of(SUBTOPOLOGY_1, Set.of(1, 2, 3)), + Map.of(SUBTOPOLOGY_2, Set.of(4, 5, 6)), + Map.of(SUBTOPOLOGY_3, Set.of(7, 8, 9)) + ); + + TasksTuple tuple2 = new TasksTuple( + Map.of(SUBTOPOLOGY_1, Set.of(3, 10, 11)), + Map.of(SUBTOPOLOGY_2, Set.of(12, 13)), + Map.of(SUBTOPOLOGY_3, Set.of(14, 15)) + ); + + assertTrue(tuple1.containsAny(tuple2)); + + TasksTuple tuple3 = new TasksTuple( + Map.of(SUBTOPOLOGY_1, Set.of(10, 11)), + Map.of(SUBTOPOLOGY_2, Set.of(12, 13)), + Map.of(SUBTOPOLOGY_3, Set.of(14, 15)) + ); + + assertFalse(tuple1.containsAny(tuple3)); + } + + @Test + public void testIsEmpty() { + TasksTuple emptyTuple = new TasksTuple(Map.of(), Map.of(), Map.of()); + assertTrue(emptyTuple.isEmpty()); + + TasksTuple nonEmptyTuple = new TasksTuple(Map.of(SUBTOPOLOGY_1, Set.of(1)), Map.of(), Map.of()); + assertFalse(nonEmptyTuple.isEmpty()); } } From 01afba8fdbd2d24adbc6ab64bca361ed53f870ac Mon Sep 17 00:00:00 2001 From: Sushant Mahajan Date: Thu, 23 Jan 2025 22:05:33 +0530 Subject: [PATCH 33/44] MINOR: Refactor ShareConsumerTest to use ClusterTestExtensions. (#18656) Reviewers: ShivsundarR , Apoorv Mittal , Andrew Schofield --- .../kafka/test/api/ShareConsumerTest.java | 582 ++++++++++-------- .../common/test/api/ClusterInstance.java | 16 + 2 files changed, 336 insertions(+), 262 deletions(-) diff --git a/core/src/test/java/kafka/test/api/ShareConsumerTest.java b/core/src/test/java/kafka/test/api/ShareConsumerTest.java index 237fe34bbe701..c7e62d0eb70d0 100644 --- a/core/src/test/java/kafka/test/api/ShareConsumerTest.java +++ b/core/src/test/java/kafka/test/api/ShareConsumerTest.java @@ -26,12 +26,12 @@ import org.apache.kafka.clients.admin.RecordsToDelete; import org.apache.kafka.clients.consumer.AcknowledgeType; import org.apache.kafka.clients.consumer.AcknowledgementCommitCallback; +import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.clients.consumer.KafkaShareConsumer; -import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.consumer.ShareConsumer; +import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; @@ -48,22 +48,21 @@ import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.record.TimestampType; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.test.KafkaClusterTestKit; -import org.apache.kafka.common.test.TestKitNodes; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Flaky; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.coordinator.group.GroupConfig; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.ArrayList; @@ -101,67 +100,64 @@ @Timeout(1200) @Tag("integration") +@ExtendWith(ClusterTestExtensions.class) +@ClusterTestDefaults( + serverProperties = { + @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), + @ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + @ClusterConfigProperty(key = "group.share.enable", value = "true"), + @ClusterConfigProperty(key = "group.share.partition.max.record.locks", value = "10000"), + @ClusterConfigProperty(key = "group.share.record.lock.duration.ms", value = "15000"), + @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + @ClusterConfigProperty(key = "share.coordinator.state.topic.min.isr", value = "1"), + @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), + @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + @ClusterConfigProperty(key = "transaction.state.log.min.isr", value = "1"), + @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1"), + @ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + }, + types = {Type.KRAFT} +) public class ShareConsumerTest { - private KafkaClusterTestKit cluster; + private final ClusterInstance cluster; private final TopicPartition tp = new TopicPartition("topic", 0); private final TopicPartition tp2 = new TopicPartition("topic2", 0); private final TopicPartition warmupTp = new TopicPartition("warmup", 0); private List sgsTopicPartitions; - private Admin adminClient; - - @BeforeEach - public void createCluster(TestInfo testInfo) throws Exception { - cluster = new KafkaClusterTestKit.Builder( - new TestKitNodes.Builder() - .setNumBrokerNodes(1) - .setNumControllerNodes(1) - .build()) - .setConfigProp("auto.create.topics.enable", "false") - .setConfigProp("group.coordinator.rebalance.protocols", "classic,consumer,share") - .setConfigProp("group.share.enable", "true") - .setConfigProp("group.share.partition.max.record.locks", "10000") - .setConfigProp("group.share.record.lock.duration.ms", "15000") - .setConfigProp("offsets.topic.replication.factor", "1") - .setConfigProp("share.coordinator.state.topic.min.isr", "1") - .setConfigProp("share.coordinator.state.topic.num.partitions", "3") - .setConfigProp("share.coordinator.state.topic.replication.factor", "1") - .setConfigProp("transaction.state.log.min.isr", "1") - .setConfigProp("transaction.state.log.replication.factor", "1") - .setConfigProp("unstable.api.versions.enable", "true") - .build(); - cluster.format(); - cluster.startup(); - cluster.waitForActiveController(); - cluster.waitForReadyBrokers(); - createTopic("topic"); - createTopic("topic2"); - adminClient = createAdminClient(); - sgsTopicPartitions = IntStream.range(0, 3) - .mapToObj(part -> new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, part)) - .toList(); - warmup(); - } - - @AfterEach - public void destroyCluster() throws Exception { - adminClient.close(); - cluster.close(); - } - - @Test + public ShareConsumerTest(ClusterInstance cluster) { + this.cluster = cluster; + } + + private void setup() { + try { + this.cluster.waitForReadyBrokers(); + createTopic("topic"); + createTopic("topic2"); + sgsTopicPartitions = IntStream.range(0, 3) + .mapToObj(part -> new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, part)) + .toList(); + this.warmup(); + } catch (Exception e) { + fail(e); + } + } + + @ClusterTest public void testPollNoSubscribeFails() { - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + setup(); + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { assertEquals(Collections.emptySet(), shareConsumer.subscription()); // "Consumer is not subscribed to any topics." assertThrows(IllegalStateException.class, () -> shareConsumer.poll(Duration.ofMillis(500))); } } - @Test + @ClusterTest public void testSubscribeAndPollNoRecords() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); assertEquals(subscription, shareConsumer.subscription()); @@ -171,10 +167,11 @@ public void testSubscribeAndPollNoRecords() { } } - @Test + @ClusterTest public void testSubscribePollUnsubscribe() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); assertEquals(subscription, shareConsumer.subscription()); @@ -186,10 +183,11 @@ public void testSubscribePollUnsubscribe() { } } - @Test + @ClusterTest public void testSubscribePollSubscribe() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); assertEquals(subscription, shareConsumer.subscription()); @@ -203,10 +201,11 @@ public void testSubscribePollSubscribe() { } } - @Test + @ClusterTest public void testSubscribeUnsubscribePollFails() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); assertEquals(subscription, shareConsumer.subscription()); @@ -220,10 +219,11 @@ public void testSubscribeUnsubscribePollFails() { } } - @Test + @ClusterTest public void testSubscribeSubscribeEmptyPollFails() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); assertEquals(subscription, shareConsumer.subscription()); @@ -237,11 +237,12 @@ public void testSubscribeSubscribeEmptyPollFails() { } } - @Test + @ClusterTest public void testSubscriptionAndPoll() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -253,11 +254,12 @@ public void testSubscriptionAndPoll() { } } - @Test + @ClusterTest public void testSubscriptionAndPollMultiple() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -275,11 +277,12 @@ public void testSubscriptionAndPollMultiple() { } } - @Test + @ClusterTest public void testAcknowledgementSentOnSubscriptionChange() throws ExecutionException, InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { Map> partitionOffsetsMap = new HashMap<>(); Map partitionExceptionMap = new HashMap<>(); @@ -311,11 +314,12 @@ public void testAcknowledgementSentOnSubscriptionChange() throws ExecutionExcept } } - @Test + @ClusterTest public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement() throws Exception { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { Map> partitionOffsetsMap = new HashMap<>(); Map partitionExceptionMap = new HashMap<>(); @@ -341,11 +345,12 @@ public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement() throws } } - @Test + @ClusterTest public void testAcknowledgementCommitCallbackOnClose() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { Map> partitionOffsetsMap = new HashMap<>(); Map partitionExceptionMap = new HashMap<>(); @@ -372,11 +377,12 @@ public void testAcknowledgementCommitCallbackOnClose() { } @Flaky("KAFKA-18033") - @Test + @ClusterTest public void testAcknowledgementCommitCallbackInvalidRecordStateException() throws Exception { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { Map> partitionOffsetsMap = new HashMap<>(); Map partitionExceptionMap = new HashMap<>(); @@ -426,11 +432,12 @@ public void onComplete(Map> offsetsMap, Exception ex } } - @Test + @ClusterTest public void testHeaders() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { int numRecords = 1; ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -454,8 +461,16 @@ public void testHeaders() { private void testHeadersSerializeDeserialize(Serializer serializer, Deserializer deserializer) { alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), serializer); - KafkaShareConsumer shareConsumer = createShareConsumer(deserializer, new ByteArrayDeserializer(), "group1")) { + Map producerConfig = Map.of( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, serializer.getClass().getName() + ); + + Map consumerConfig = Map.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserializer.getClass().getName() + ); + + try (Producer producer = createProducer(producerConfig); + ShareConsumer shareConsumer = createShareConsumer("group1", consumerConfig)) { int numRecords = 1; ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -469,20 +484,22 @@ private void testHeadersSerializeDeserialize(Serializer serializer, Dese } } - @Test + @ClusterTest public void testHeadersSerializerDeserializer() { + setup(); testHeadersSerializeDeserialize(new BaseConsumerTest.SerializerImpl(), new BaseConsumerTest.DeserializerImpl()); verifyShareGroupStateTopicRecordsProduced(); } - @Test + @ClusterTest public void testMaxPollRecords() { + setup(); int numRecords = 10000; int maxPollRecords = 2; alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), - "group1", Collections.singletonMap(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords)))) { + try (ShareConsumer shareConsumer = createShareConsumer("group1", + Collections.singletonMap(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords)))) { long startingTimestamp = System.currentTimeMillis(); produceMessagesWithTimestamp(numRecords, startingTimestamp); @@ -508,12 +525,13 @@ public void testMaxPollRecords() { } } - @Test + @ClusterTest public void testControlRecordsSkipped() throws Exception { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer transactionalProducer = createProducer(new ByteArraySerializer(), new ByteArraySerializer(), "T1"); - KafkaProducer nonTransactionalProducer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer transactionalProducer = createProducer("T1"); + Producer nonTransactionalProducer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -553,11 +571,12 @@ public void testControlRecordsSkipped() throws Exception { } } - @Test + @ClusterTest public void testExplicitAcknowledgeSuccess() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -574,11 +593,12 @@ public void testExplicitAcknowledgeSuccess() { } } - @Test + @ClusterTest public void testExplicitAcknowledgeCommitSuccess() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -597,12 +617,13 @@ public void testExplicitAcknowledgeCommitSuccess() { } } - @Test + @ClusterTest public void testExplicitAcknowledgementCommitAsync() throws InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaShareConsumer shareConsumer2 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer1 = createShareConsumer("group1"); + ShareConsumer shareConsumer2 = createShareConsumer("group1")) { ProducerRecord record1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); ProducerRecord record2 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -653,11 +674,12 @@ public void testExplicitAcknowledgementCommitAsync() throws InterruptedException } } - @Test + @ClusterTest public void testExplicitAcknowledgementCommitAsyncPartialBatch() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer1 = createShareConsumer("group1")) { ProducerRecord record1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); ProducerRecord record2 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -716,11 +738,12 @@ public void testExplicitAcknowledgementCommitAsyncPartialBatch() { } } - @Test + @ClusterTest public void testExplicitAcknowledgeReleasePollAccept() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -739,11 +762,12 @@ public void testExplicitAcknowledgeReleasePollAccept() { } } - @Test + @ClusterTest public void testExplicitAcknowledgeReleaseAccept() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -759,11 +783,12 @@ public void testExplicitAcknowledgeReleaseAccept() { } } - @Test + @ClusterTest public void testExplicitAcknowledgeReleaseClose() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -777,11 +802,12 @@ public void testExplicitAcknowledgeReleaseClose() { } } - @Test + @ClusterTest public void testExplicitAcknowledgeThrowsNotInBatch() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -799,11 +825,12 @@ public void testExplicitAcknowledgeThrowsNotInBatch() { } } - @Test + @ClusterTest public void testImplicitAcknowledgeFailsExplicit() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -820,11 +847,12 @@ public void testImplicitAcknowledgeFailsExplicit() { } } - @Test + @ClusterTest public void testImplicitAcknowledgeCommitSync() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -843,11 +871,12 @@ public void testImplicitAcknowledgeCommitSync() { } } - @Test + @ClusterTest public void testImplicitAcknowledgementCommitAsync() throws InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); ProducerRecord record2 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -882,14 +911,19 @@ public void testImplicitAcknowledgementCommitAsync() throws InterruptedException } } - @Test + @ClusterTest public void testFetchRecordLargerThanMaxPartitionFetchBytes() throws Exception { + setup(); int maxPartitionFetchBytes = 10000; alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), - "group1", Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxPartitionFetchBytes)))) { + try ( + Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer( + "group1", + Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxPartitionFetchBytes)) + ) + ) { ProducerRecord smallRecord = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); ProducerRecord bigRecord = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), new byte[maxPartitionFetchBytes]); @@ -904,13 +938,14 @@ public void testFetchRecordLargerThanMaxPartitionFetchBytes() throws Exception { } } - @Test + @ClusterTest public void testMultipleConsumersWithDifferentGroupIds() throws InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); alterShareAutoOffsetReset("group2", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaShareConsumer shareConsumer2 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group2")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer1 = createShareConsumer("group1"); + ShareConsumer shareConsumer2 = createShareConsumer("group2")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -955,12 +990,13 @@ public void testMultipleConsumersWithDifferentGroupIds() throws InterruptedExcep } } - @Test + @ClusterTest public void testMultipleConsumersInGroupSequentialConsumption() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaShareConsumer shareConsumer2 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer1 = createShareConsumer("group1"); + ShareConsumer shareConsumer2 = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); shareConsumer1.subscribe(Collections.singleton(tp.topic())); @@ -992,9 +1028,10 @@ public void testMultipleConsumersInGroupSequentialConsumption() { } @Flaky("KAFKA-18033") - @Test + @ClusterTest public void testMultipleConsumersInGroupConcurrentConsumption() throws InterruptedException, ExecutionException, TimeoutException { + setup(); AtomicInteger totalMessagesConsumed = new AtomicInteger(0); int consumerCount = 4; @@ -1026,9 +1063,10 @@ public void testMultipleConsumersInGroupConcurrentConsumption() assertEquals(producerCount * messagesPerProducer, totalResult); } - @Test + @ClusterTest public void testMultipleConsumersInMultipleGroupsConcurrentConsumption() throws ExecutionException, InterruptedException, TimeoutException { + setup(); AtomicInteger totalMessagesConsumedGroup1 = new AtomicInteger(0); AtomicInteger totalMessagesConsumedGroup2 = new AtomicInteger(0); AtomicInteger totalMessagesConsumedGroup3 = new AtomicInteger(0); @@ -1089,12 +1127,13 @@ public void testMultipleConsumersInMultipleGroupsConcurrentConsumption() verifyShareGroupStateTopicRecordsProduced(); } - @Test + @ClusterTest public void testConsumerCloseInGroupSequential() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaShareConsumer shareConsumer2 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer1 = createShareConsumer("group1"); + ShareConsumer shareConsumer2 = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); shareConsumer1.subscribe(Collections.singleton(tp.topic())); @@ -1136,9 +1175,10 @@ public void testConsumerCloseInGroupSequential() { } } - @Test + @ClusterTest public void testMultipleConsumersInGroupFailureConcurrentConsumption() throws InterruptedException, ExecutionException, TimeoutException { + setup(); AtomicInteger totalMessagesConsumed = new AtomicInteger(0); int consumerCount = 4; @@ -1181,11 +1221,12 @@ public void testMultipleConsumersInGroupFailureConcurrentConsumption() verifyShareGroupStateTopicRecordsProduced(); } - @Test + @ClusterTest public void testAcquisitionLockTimeoutOnConsumer() throws InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord producerRecord1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key_1".getBytes(), "value_1".getBytes()); @@ -1242,14 +1283,15 @@ public void testAcquisitionLockTimeoutOnConsumer() throws InterruptedException { } /** - * Test to verify that the acknowledgement commit callback cannot invoke methods of KafkaShareConsumer. + * Test to verify that the acknowledgement commit callback cannot invoke methods of ShareConsumer. * The exception thrown is verified in {@link TestableAcknowledgementCommitCallbackWithShareConsumer} */ - @Test + @ClusterTest public void testAcknowledgementCommitCallbackCallsShareConsumerDisallowed() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -1258,7 +1300,7 @@ public void testAcknowledgementCommitCallbackCallsShareConsumerDisallowed() { shareConsumer.setAcknowledgementCommitCallback(new TestableAcknowledgementCommitCallbackWithShareConsumer<>(shareConsumer)); shareConsumer.subscribe(Collections.singleton(tp.topic())); - // The acknowledgment commit callback will try to call a method of KafkaShareConsumer + // The acknowledgment commit callback will try to call a method of ShareConsumer shareConsumer.poll(Duration.ofMillis(5000)); // The second poll sends the acknowledgements implicitly. // The acknowledgement commit callback will be called and the exception is thrown. @@ -1269,15 +1311,15 @@ public void testAcknowledgementCommitCallbackCallsShareConsumerDisallowed() { } private class TestableAcknowledgementCommitCallbackWithShareConsumer implements AcknowledgementCommitCallback { - private final KafkaShareConsumer shareConsumer; + private final ShareConsumer shareConsumer; - TestableAcknowledgementCommitCallbackWithShareConsumer(KafkaShareConsumer shareConsumer) { + TestableAcknowledgementCommitCallbackWithShareConsumer(ShareConsumer shareConsumer) { this.shareConsumer = shareConsumer; } @Override public void onComplete(Map> offsetsMap, Exception exception) { - // Accessing methods of KafkaShareConsumer should throw an exception. + // Accessing methods of ShareConsumer should throw an exception. assertThrows(IllegalStateException.class, shareConsumer::close); assertThrows(IllegalStateException.class, () -> shareConsumer.subscribe(Collections.singleton(tp.topic()))); assertThrows(IllegalStateException.class, () -> shareConsumer.poll(Duration.ofMillis(5000))); @@ -1285,20 +1327,21 @@ public void onComplete(Map> offsetsMap, Exception ex } /** - * Test to verify that the acknowledgement commit callback can invoke KafkaShareConsumer.wakeup() and it + * Test to verify that the acknowledgement commit callback can invoke ShareConsumer.wakeup() and it * wakes up the enclosing poll. */ - @Test + @ClusterTest public void testAcknowledgementCommitCallbackCallsShareConsumerWakeup() throws InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); producer.flush(); - // The acknowledgment commit callback will try to call a method of KafkaShareConsumer + // The acknowledgment commit callback will try to call a method of ShareConsumer shareConsumer.setAcknowledgementCommitCallback(new TestableAcknowledgementCommitCallbackWakeup<>(shareConsumer)); shareConsumer.subscribe(Collections.singleton(tp.topic())); @@ -1324,9 +1367,9 @@ public void testAcknowledgementCommitCallbackCallsShareConsumerWakeup() throws I } private static class TestableAcknowledgementCommitCallbackWakeup implements AcknowledgementCommitCallback { - private final KafkaShareConsumer shareConsumer; + private final ShareConsumer shareConsumer; - TestableAcknowledgementCommitCallbackWakeup(KafkaShareConsumer shareConsumer) { + TestableAcknowledgementCommitCallbackWakeup(ShareConsumer shareConsumer) { this.shareConsumer = shareConsumer; } @@ -1340,11 +1383,12 @@ public void onComplete(Map> offsetsMap, Exception ex * Test to verify that the acknowledgement commit callback can throw an exception, and it is propagated * to the caller of poll(). */ - @Test + @ClusterTest public void testAcknowledgementCommitCallbackThrowsException() throws InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -1377,13 +1421,14 @@ public void onComplete(Map> offsetsMap, Exception ex } /** - * Test to verify that calling Thread.interrupt() before KafkaShareConsumer.poll(Duration) + * Test to verify that calling Thread.interrupt() before ShareConsumer.poll(Duration) * causes it to throw InterruptException */ - @Test + @ClusterTest public void testPollThrowsInterruptExceptionIfInterrupted() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { shareConsumer.subscribe(Collections.singleton(tp.topic())); @@ -1404,10 +1449,11 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { * Test to verify that InvalidTopicException is thrown if the consumer subscribes * to an invalid topic. */ - @Test + @ClusterTest public void testSubscribeOnInvalidTopicThrowsInvalidTopicException() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (ShareConsumer shareConsumer = createShareConsumer("group1")) { shareConsumer.subscribe(Collections.singleton("topic abc")); @@ -1421,11 +1467,12 @@ public void testSubscribeOnInvalidTopicThrowsInvalidTopicException() { * Test to ensure that a wakeup when records are buffered doesn't prevent the records * being returned on the next poll. */ - @Test + @ClusterTest public void testWakeupWithFetchedRecordsAvailable() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); producer.send(record); @@ -1442,11 +1489,12 @@ public void testWakeupWithFetchedRecordsAvailable() { } } - @Test + @ClusterTest public void testSubscriptionFollowedByTopicCreation() throws InterruptedException { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { String topic = "foo"; shareConsumer.subscribe(Collections.singleton(topic)); @@ -1471,16 +1519,17 @@ public void testSubscriptionFollowedByTopicCreation() throws InterruptedExceptio } } - @Test + @ClusterTest public void testSubscriptionAndPollFollowedByTopicDeletion() throws InterruptedException, ExecutionException { + setup(); String topic1 = "bar"; String topic2 = "baz"; createTopic(topic1); createTopic(topic2); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("group1")) { ProducerRecord recordTopic1 = new ProducerRecord<>(topic1, 0, null, "key".getBytes(), "value".getBytes()); ProducerRecord recordTopic2 = new ProducerRecord<>(topic2, 0, null, "key".getBytes(), "value".getBytes()); @@ -1513,12 +1562,16 @@ public void testSubscriptionAndPollFollowedByTopicDeletion() throws InterruptedE } } - @Test + @ClusterTest public void testLsoMovementByRecordsDeletion() { + setup(); String groupId = "group1"; alterShareAutoOffsetReset(groupId, "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try ( + Producer producer = createProducer(); + Admin adminClient = createAdminClient() + ) { ProducerRecord record = new ProducerRecord<>(tp.topic(), 0, null, "key".getBytes(), "value".getBytes()); @@ -1555,10 +1608,11 @@ public void testLsoMovementByRecordsDeletion() { } } - @Test + @ClusterTest public void testShareAutoOffsetResetDefaultValue() { - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + setup(); + try (ShareConsumer shareConsumer = createShareConsumer("group1"); + Producer producer = createProducer()) { shareConsumer.subscribe(Collections.singleton(tp.topic())); ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -1581,11 +1635,12 @@ public void testShareAutoOffsetResetDefaultValue() { } } - @Test + @ClusterTest public void testShareAutoOffsetResetEarliest() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try (ShareConsumer shareConsumer = createShareConsumer("group1"); + Producer producer = createProducer()) { shareConsumer.subscribe(Collections.singleton(tp.topic())); ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -1606,11 +1661,15 @@ public void testShareAutoOffsetResetEarliest() { } } - @Test + @ClusterTest public void testShareAutoOffsetResetEarliestAfterLsoMovement() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try ( + ShareConsumer shareConsumer = createShareConsumer("group1"); + Producer producer = createProducer(); + Admin adminClient = createAdminClient() + ) { shareConsumer.subscribe(Collections.singleton(tp.topic())); @@ -1630,13 +1689,14 @@ public void testShareAutoOffsetResetEarliestAfterLsoMovement() { } } - @Test + @ClusterTest public void testShareAutoOffsetResetMultipleGroupsWithDifferentValue() { + setup(); alterShareAutoOffsetReset("group1", "earliest"); alterShareAutoOffsetReset("group2", "latest"); - try (KafkaShareConsumer shareConsumerEarliest = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaShareConsumer shareConsumerLatest = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group2"); - KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try (ShareConsumer shareConsumerEarliest = createShareConsumer("group1"); + ShareConsumer shareConsumerLatest = createShareConsumer("group2"); + Producer producer = createProducer()) { shareConsumerEarliest.subscribe(Collections.singleton(tp.topic())); @@ -1670,13 +1730,14 @@ public void testShareAutoOffsetResetMultipleGroupsWithDifferentValue() { } } - @Test + @ClusterTest public void testShareAutoOffsetResetByDuration() throws Exception { + setup(); // Set auto offset reset to 1 hour before current time alterShareAutoOffsetReset("group1", "by_duration:PT1H"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); - KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try (ShareConsumer shareConsumer = createShareConsumer("group1"); + Producer producer = createProducer()) { long currentTime = System.currentTimeMillis(); long twoHoursAgo = currentTime - TimeUnit.HOURS.toMillis(2); @@ -1711,8 +1772,8 @@ public void testShareAutoOffsetResetByDuration() throws Exception { // Set the auto offset reset to 3 hours before current time // so the consumer should consume all messages (3 records) alterShareAutoOffsetReset("group2", "by_duration:PT3H"); - try (KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group2"); - KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try (ShareConsumer shareConsumer = createShareConsumer("group2"); + Producer producer = createProducer()) { shareConsumer.subscribe(Collections.singleton(tp.topic())); List> records = consumeRecords(shareConsumer, 3); @@ -1721,29 +1782,32 @@ public void testShareAutoOffsetResetByDuration() throws Exception { } } - @Test + @ClusterTest public void testShareAutoOffsetResetByDurationInvalidFormat() throws Exception { + setup(); // Test invalid duration format ConfigResource configResource = new ConfigResource(ConfigResource.Type.GROUP, "group1"); Map> alterEntries = new HashMap<>(); // Test invalid duration format - alterEntries.put(configResource, List.of(new AlterConfigOp(new ConfigEntry( - GroupConfig.SHARE_AUTO_OFFSET_RESET_CONFIG, "by_duration:1h"), AlterConfigOp.OpType.SET))); - ExecutionException e1 = assertThrows(ExecutionException.class, () -> - adminClient.incrementalAlterConfigs(alterEntries).all().get()); - assertInstanceOf(InvalidConfigurationException.class, e1.getCause()); + try (Admin adminClient = createAdminClient()) { + alterEntries.put(configResource, List.of(new AlterConfigOp(new ConfigEntry( + GroupConfig.SHARE_AUTO_OFFSET_RESET_CONFIG, "by_duration:1h"), AlterConfigOp.OpType.SET))); + ExecutionException e1 = assertThrows(ExecutionException.class, () -> + adminClient.incrementalAlterConfigs(alterEntries).all().get()); + assertInstanceOf(InvalidConfigurationException.class, e1.getCause()); - // Test negative duration - alterEntries.put(configResource, List.of(new AlterConfigOp(new ConfigEntry( - GroupConfig.SHARE_AUTO_OFFSET_RESET_CONFIG, "by_duration:-PT1H"), AlterConfigOp.OpType.SET))); - ExecutionException e2 = assertThrows(ExecutionException.class, () -> - adminClient.incrementalAlterConfigs(alterEntries).all().get()); - assertInstanceOf(InvalidConfigurationException.class, e2.getCause()); + // Test negative duration + alterEntries.put(configResource, List.of(new AlterConfigOp(new ConfigEntry( + GroupConfig.SHARE_AUTO_OFFSET_RESET_CONFIG, "by_duration:-PT1H"), AlterConfigOp.OpType.SET))); + ExecutionException e2 = assertThrows(ExecutionException.class, () -> + adminClient.incrementalAlterConfigs(alterEntries).all().get()); + assertInstanceOf(InvalidConfigurationException.class, e2.getCause()); + } } private int produceMessages(int messageCount) { - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try (Producer producer = createProducer()) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); IntStream.range(0, messageCount).forEach(__ -> producer.send(record)); producer.flush(); @@ -1752,7 +1816,7 @@ private int produceMessages(int messageCount) { } private void produceMessagesWithTimestamp(int messageCount, long startingTimestamp) { - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { + try (Producer producer = createProducer()) { for (int i = 0; i < messageCount; i++) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), startingTimestamp + i, ("key " + i).getBytes(), ("value " + i).getBytes()); @@ -1769,8 +1833,8 @@ private int consumeMessages(AtomicInteger totalMessagesConsumed, int maxPolls, boolean commit) { return assertDoesNotThrow(() -> { - try (KafkaShareConsumer shareConsumer = createShareConsumer( - new ByteArrayDeserializer(), new ByteArrayDeserializer(), groupId)) { + try (ShareConsumer shareConsumer = createShareConsumer( + groupId)) { shareConsumer.subscribe(Collections.singleton(tp.topic())); return consumeMessages(shareConsumer, totalMessagesConsumed, totalMessages, consumerNumber, maxPolls, commit); } @@ -1785,8 +1849,8 @@ private int consumeMessages(AtomicInteger totalMessagesConsumed, boolean commit, int maxFetchBytes) { return assertDoesNotThrow(() -> { - try (KafkaShareConsumer shareConsumer = createShareConsumer( - new ByteArrayDeserializer(), new ByteArrayDeserializer(), groupId, + try (ShareConsumer shareConsumer = createShareConsumer( + groupId, Map.of(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxFetchBytes))) { shareConsumer.subscribe(Collections.singleton(tp.topic())); return consumeMessages(shareConsumer, totalMessagesConsumed, totalMessages, consumerNumber, maxPolls, commit); @@ -1794,7 +1858,7 @@ private int consumeMessages(AtomicInteger totalMessagesConsumed, }, "Consumer " + consumerNumber + " failed with exception"); } - private int consumeMessages(KafkaShareConsumer consumer, + private int consumeMessages(ShareConsumer consumer, AtomicInteger totalMessagesConsumed, int totalMessages, int consumerNumber, @@ -1827,7 +1891,7 @@ private int consumeMessages(KafkaShareConsumer consumer, }, "Consumer " + consumerNumber + " failed with exception"); } - private List> consumeRecords(KafkaShareConsumer consumer, + private List> consumeRecords(ShareConsumer consumer, int numRecords) { ArrayList> accumulatedRecords = new ArrayList<>(); long startTimeMs = System.currentTimeMillis(); @@ -1841,58 +1905,55 @@ private List> consumeRecords(KafkaShareConsumer { - try (Admin admin = Admin.create(props)) { + try (Admin admin = createAdminClient()) { admin.createTopics(Collections.singleton(new NewTopic(topicName, 1, (short) 1))).all().get(); } }, "Failed to create topic"); } private void deleteTopic(String topicName) { - Properties props = cluster.clientProperties(); assertDoesNotThrow(() -> { - try (Admin admin = Admin.create(props)) { + try (Admin admin = createAdminClient()) { admin.deleteTopics(Collections.singleton(topicName)).all().get(); } }, "Failed to delete topic"); } private Admin createAdminClient() { - Properties props = cluster.clientProperties(); - return Admin.create(props); + return cluster.admin(); } - private KafkaProducer createProducer(Serializer keySerializer, - Serializer valueSerializer) { - Properties props = cluster.clientProperties(); - return new KafkaProducer<>(props, keySerializer, valueSerializer); + private Producer createProducer() { + return createProducer(Map.of()); } - private KafkaProducer createProducer(Serializer keySerializer, - Serializer valueSerializer, - String transactionalId) { - Properties props = cluster.clientProperties(); - props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId); - return new KafkaProducer<>(props, keySerializer, valueSerializer); + private Producer createProducer(Map config) { + return cluster.producer(config); } - private KafkaShareConsumer createShareConsumer(Deserializer keyDeserializer, - Deserializer valueDeserializer, - String groupId) { - Properties props = cluster.clientProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); - return new KafkaShareConsumer<>(props, keyDeserializer, valueDeserializer); + private Producer createProducer(String transactionalId) { + return createProducer( + Map.of( + ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId + ) + ); } - private KafkaShareConsumer createShareConsumer(Deserializer keyDeserializer, - Deserializer valueDeserializer, - String groupId, - Map additionalProperties) { - Properties props = cluster.clientProperties(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + private ShareConsumer createShareConsumer(String groupId) { + return createShareConsumer(groupId, Map.of()); + } + + private ShareConsumer createShareConsumer( + String groupId, + Map additionalProperties + ) { + Properties props = new Properties(); props.putAll(additionalProperties); - return new KafkaShareConsumer<>(props, keyDeserializer, valueDeserializer); + props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + Map conf = new HashMap<>(); + props.forEach((k, v) -> conf.put((String) k, v)); + return cluster.shareConsumer(conf); } private void warmup() throws InterruptedException { @@ -1901,8 +1962,8 @@ private void warmup() throws InterruptedException { ProducerRecord record = new ProducerRecord<>(warmupTp.topic(), warmupTp.partition(), null, "key".getBytes(), "value".getBytes()); Set subscription = Collections.singleton(warmupTp.topic()); alterShareAutoOffsetReset("warmupgroup1", "earliest"); - try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); - KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "warmupgroup1")) { + try (Producer producer = createProducer(); + ShareConsumer shareConsumer = createShareConsumer("warmupgroup1")) { producer.send(record); producer.flush(); @@ -1921,12 +1982,7 @@ private void waitForMetadataCache() throws InterruptedException { private void verifyShareGroupStateTopicRecordsProduced() { try { - Map consumerConfigs = new HashMap<>(); - consumerConfigs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); - consumerConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - consumerConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - - try (KafkaConsumer consumer = new KafkaConsumer<>(consumerConfigs)) { + try (Consumer consumer = cluster.consumer()) { consumer.assign(sgsTopicPartitions); consumer.seekToBeginning(sgsTopicPartitions); Set> records = new HashSet<>(); @@ -1953,8 +2009,10 @@ private void alterShareAutoOffsetReset(String groupId, String newValue) { alterEntries.put(configResource, List.of(new AlterConfigOp(new ConfigEntry( GroupConfig.SHARE_AUTO_OFFSET_RESET_CONFIG, newValue), AlterConfigOp.OpType.SET))); AlterConfigsOptions alterOptions = new AlterConfigsOptions(); - assertDoesNotThrow(() -> adminClient.incrementalAlterConfigs(alterEntries, alterOptions) + try (Admin adminClient = createAdminClient()) { + assertDoesNotThrow(() -> adminClient.incrementalAlterConfigs(alterEntries, alterOptions) .all() .get(60, TimeUnit.SECONDS), "Failed to alter configs"); + } } } diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java b/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java index 1c8551bf9e6de..eb97c2c92a164 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java +++ b/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java @@ -32,6 +32,8 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.GroupProtocol; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.KafkaShareConsumer; +import org.apache.kafka.clients.consumer.ShareConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -188,6 +190,20 @@ default Consumer consumer() { return consumer(Map.of()); } + default ShareConsumer shareConsumer() { + return shareConsumer(Map.of()); + } + + default ShareConsumer shareConsumer(Map configs) { + Map props = new HashMap<>(configs); + props.putIfAbsent(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.putIfAbsent(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, "group_" + TestUtils.randomString(5)); + props.putIfAbsent(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + return new KafkaShareConsumer<>(setClientSaslConfig(props)); + } + default Admin admin(Map configs, boolean usingBootstrapControllers) { Map props = new HashMap<>(configs); if (usingBootstrapControllers) { From ce4eeaa379678af21376b3efa4441f420fbf0f56 Mon Sep 17 00:00:00 2001 From: TaiJuWu Date: Fri, 24 Jan 2025 01:27:18 +0800 Subject: [PATCH 34/44] MINOR: restore `testGetAllTopicMetadataShouldNotCreateTopicOrReturnUnknownTopicPartition` (#18633) Reviewers: Ismael Juma , Chia-Ping Tsai --- .../unit/kafka/server/KafkaApisTest.scala | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index 5dc5ad06f29e3..af26996c56eb2 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -3562,6 +3562,47 @@ class KafkaApisTest extends Logging { assertEquals(Set(0), response.brokers.asScala.map(_.id).toSet) } + + /** + * Metadata request to fetch all topics should not result in the followings: + * 1) Auto topic creation + * 2) UNKNOWN_TOPIC_OR_PARTITION + * + * This case is testing the case that a topic is being deleted from MetadataCache right after + * authorization but before checking in MetadataCache. + */ + @Test + def testGetAllTopicMetadataShouldNotCreateTopicOrReturnUnknownTopicPartition(): Unit = { + // Setup: authorizer authorizes 2 topics, but one got deleted in metadata cache + metadataCache = mock(classOf[KRaftMetadataCache]) + when(metadataCache.getAliveBrokerNodes(any())).thenReturn(List(new Node(brokerId,"localhost", 0))) + when(metadataCache.getRandomAliveBrokerId).thenReturn(None) + + // 2 topics returned for authorization in during handle + val topicsReturnedFromMetadataCacheForAuthorization = Set("remaining-topic", "later-deleted-topic") + when(metadataCache.getAllTopics()).thenReturn(topicsReturnedFromMetadataCacheForAuthorization) + // 1 topic is deleted from metadata right at the time between authorization and the next getTopicMetadata() call + when(metadataCache.getTopicMetadata( + ArgumentMatchers.eq(topicsReturnedFromMetadataCacheForAuthorization), + any[ListenerName], + anyBoolean, + anyBoolean + )).thenReturn(Seq( + new MetadataResponseTopic() + .setErrorCode(Errors.NONE.code) + .setName("remaining-topic") + .setIsInternal(false) + )) + + val response = sendMetadataRequestWithInconsistentListeners(new ListenerName("PLAINTEXT")) + val responseTopics = response.topicMetadata().asScala.map { metadata => metadata.topic() } + + // verify we don't create topic when getAllTopicMetadata + verify(autoTopicCreationManager, never).createTopics(any(), any(), any()) + assertEquals(List("remaining-topic"), responseTopics) + assertTrue(response.topicsByError(Errors.UNKNOWN_TOPIC_OR_PARTITION).isEmpty) + } + @Test def testUnauthorizedTopicMetadataRequest(): Unit = { // 1. Set up broker information From 40890faa1bed8296919a7e108937ec559d901348 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 24 Jan 2025 01:34:36 +0800 Subject: [PATCH 35/44] KAFKA-18592 Cleanup ReplicaManager (#18621) Reviewers: Ismael Juma , Christo Lolov , Chia-Ping Tsai --- .../builders/ReplicaManagerBuilder.java | 33 ------------------- .../main/scala/kafka/cluster/Partition.scala | 18 ---------- .../scala/kafka/server/ReplicaManager.scala | 27 +-------------- .../AbstractCoordinatorConcurrencyTest.scala | 1 - .../kafka/server/ReplicaManagerTest.scala | 6 ---- docs/zk2kraft.html | 9 +++++ 6 files changed, 10 insertions(+), 84 deletions(-) diff --git a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java index 626b53c12c4a4..b580485139ba9 100644 --- a/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java +++ b/core/src/main/java/kafka/server/builders/ReplicaManagerBuilder.java @@ -22,7 +22,6 @@ import kafka.server.AddPartitionsToTxnManager; import kafka.server.AlterPartitionManager; import kafka.server.DelayedDeleteRecords; -import kafka.server.DelayedElectLeader; import kafka.server.DelayedFetch; import kafka.server.DelayedProduce; import kafka.server.DelayedRemoteFetch; @@ -66,7 +65,6 @@ public class ReplicaManagerBuilder { private Optional> delayedProducePurgatory = Optional.empty(); private Optional> delayedFetchPurgatory = Optional.empty(); private Optional> delayedDeleteRecordsPurgatory = Optional.empty(); - private Optional> delayedElectLeaderPurgatory = Optional.empty(); private Optional> delayedRemoteFetchPurgatory = Optional.empty(); private Optional> delayedRemoteListOffsetsPurgatory = Optional.empty(); private Optional> delayedShareFetchPurgatory = Optional.empty(); @@ -130,36 +128,11 @@ public ReplicaManagerBuilder setBrokerTopicStats(BrokerTopicStats brokerTopicSta return this; } - public ReplicaManagerBuilder setIsShuttingDown(AtomicBoolean isShuttingDown) { - this.isShuttingDown = isShuttingDown; - return this; - } - - public ReplicaManagerBuilder setDelayedProducePurgatory(DelayedOperationPurgatory delayedProducePurgatory) { - this.delayedProducePurgatory = Optional.of(delayedProducePurgatory); - return this; - } - public ReplicaManagerBuilder setDelayedFetchPurgatory(DelayedOperationPurgatory delayedFetchPurgatory) { this.delayedFetchPurgatory = Optional.of(delayedFetchPurgatory); return this; } - public ReplicaManagerBuilder setDelayedRemoteFetchPurgatory(DelayedOperationPurgatory delayedRemoteFetchPurgatory) { - this.delayedRemoteFetchPurgatory = Optional.of(delayedRemoteFetchPurgatory); - return this; - } - - public ReplicaManagerBuilder setDelayedDeleteRecordsPurgatory(DelayedOperationPurgatory delayedDeleteRecordsPurgatory) { - this.delayedDeleteRecordsPurgatory = Optional.of(delayedDeleteRecordsPurgatory); - return this; - } - - public ReplicaManagerBuilder setDelayedElectLeaderPurgatoryParam(DelayedOperationPurgatory delayedElectLeaderPurgatory) { - this.delayedElectLeaderPurgatory = Optional.of(delayedElectLeaderPurgatory); - return this; - } - public ReplicaManagerBuilder setThreadNamePrefix(String threadNamePrefix) { this.threadNamePrefix = Optional.of(threadNamePrefix); return this; @@ -170,11 +143,6 @@ public ReplicaManagerBuilder setBrokerEpoch(long brokerEpoch) { return this; } - public ReplicaManagerBuilder setAddPartitionsToTransactionManager(AddPartitionsToTxnManager addPartitionsToTxnManager) { - this.addPartitionsToTxnManager = Optional.of(addPartitionsToTxnManager); - return this; - } - public ReplicaManagerBuilder setDirectoryEventHandler(DirectoryEventHandler directoryEventHandler) { this.directoryEventHandler = directoryEventHandler; return this; @@ -206,7 +174,6 @@ public ReplicaManager build() { OptionConverters.toScala(delayedProducePurgatory), OptionConverters.toScala(delayedFetchPurgatory), OptionConverters.toScala(delayedDeleteRecordsPurgatory), - OptionConverters.toScala(delayedElectLeaderPurgatory), OptionConverters.toScala(delayedRemoteFetchPurgatory), OptionConverters.toScala(delayedRemoteListOffsetsPurgatory), OptionConverters.toScala(delayedShareFetchPurgatory), diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala index 4764bba5cb7c8..5035c86aa0616 100755 --- a/core/src/main/scala/kafka/cluster/Partition.scala +++ b/core/src/main/scala/kafka/cluster/Partition.scala @@ -1635,24 +1635,6 @@ class Partition(val topicPartition: TopicPartition, localLog.fetchOffsetSnapshot } - def legacyFetchOffsetsForTimestamp(timestamp: Long, - maxNumOffsets: Int, - isFromConsumer: Boolean, - fetchOnlyFromLeader: Boolean): Seq[Long] = inReadLock(leaderIsrUpdateLock) { - val localLog = localLogWithEpochOrThrow(Optional.empty(), fetchOnlyFromLeader) - val allOffsets = localLog.legacyFetchOffsetsBefore(timestamp, maxNumOffsets) - - if (!isFromConsumer) { - allOffsets - } else { - val hw = localLog.highWatermark - if (allOffsets.exists(_ > hw)) - hw +: allOffsets.dropWhile(_ > hw) - else - allOffsets - } - } - def logStartOffset: Long = { inReadLock(leaderIsrUpdateLock) { leaderLogIfLocal.map(_.logStartOffset).getOrElse(-1) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index b76e151c2d3ed..d8e415028ea1d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -53,7 +53,7 @@ import org.apache.kafka.server.{ActionQueue, DelayedActionQueue, common} import org.apache.kafka.server.common.{DirectoryEventHandler, RequestLocal, StopPartition, TopicOptionalIdPartition} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.network.BrokerEndPoint -import org.apache.kafka.server.purgatory.{DelayedOperationKey, DelayedOperationPurgatory, TopicPartitionOperationKey} +import org.apache.kafka.server.purgatory.{DelayedOperationPurgatory, TopicPartitionOperationKey} import org.apache.kafka.server.share.fetch.{DelayedShareFetchKey, DelayedShareFetchPartitionKey} import org.apache.kafka.server.storage.log.{FetchParams, FetchPartitionData} import org.apache.kafka.server.util.{Scheduler, ShutdownableThread} @@ -274,7 +274,6 @@ class ReplicaManager(val config: KafkaConfig, delayedProducePurgatoryParam: Option[DelayedOperationPurgatory[DelayedProduce]] = None, delayedFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedFetch]] = None, delayedDeleteRecordsPurgatoryParam: Option[DelayedOperationPurgatory[DelayedDeleteRecords]] = None, - delayedElectLeaderPurgatoryParam: Option[DelayedOperationPurgatory[DelayedElectLeader]] = None, delayedRemoteFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedRemoteFetch]] = None, delayedRemoteListOffsetsPurgatoryParam: Option[DelayedOperationPurgatory[DelayedRemoteListOffsets]] = None, delayedShareFetchPurgatoryParam: Option[DelayedOperationPurgatory[DelayedShareFetch]] = None, @@ -298,9 +297,6 @@ class ReplicaManager(val config: KafkaConfig, new DelayedOperationPurgatory[DelayedDeleteRecords]( "DeleteRecords", config.brokerId, config.deleteRecordsPurgatoryPurgeIntervalRequests)) - val delayedElectLeaderPurgatory = delayedElectLeaderPurgatoryParam.getOrElse( - new DelayedOperationPurgatory[DelayedElectLeader]( - "ElectLeader", config.brokerId)) val delayedRemoteFetchPurgatory = delayedRemoteFetchPurgatoryParam.getOrElse( new DelayedOperationPurgatory[DelayedRemoteFetch]( "RemoteFetch", config.brokerId)) @@ -387,13 +383,6 @@ class ReplicaManager(val config: KafkaConfig, def getLog(topicPartition: TopicPartition): Option[UnifiedLog] = logManager.getLog(topicPartition) - def hasDelayedElectionOperations: Boolean = delayedElectLeaderPurgatory.numDelayed != 0 - - def tryCompleteElection(key: DelayedOperationKey): Unit = { - val completed = delayedElectLeaderPurgatory.checkAndComplete(key) - debug("Request key %s unblocked %d ElectLeader.".format(key.keyLabel, completed)) - } - def startup(): Unit = { // start ISR expiration thread // A follower can lag behind leader for up to config.replicaLagTimeMaxMs x 1.5 before it is removed from ISR @@ -628,10 +617,6 @@ class ReplicaManager(val config: KafkaConfig, onlinePartition(topicPartition).flatMap(_.log) } - def getLogDir(topicPartition: TopicPartition): Option[String] = { - localLog(topicPartition).map(_.parentDir) - } - def tryCompleteActions(): Unit = defaultActionQueue.tryCompleteActions() def addToActionQueue(action: Runnable): Unit = defaultActionQueue.add(action) @@ -1490,15 +1475,6 @@ class ReplicaManager(val config: KafkaConfig, partition.fetchOffsetForTimestamp(timestamp, isolationLevel, currentLeaderEpoch, fetchOnlyFromLeader, remoteLogManager) } - def legacyFetchOffsetsForTimestamp(topicPartition: TopicPartition, - timestamp: Long, - maxNumOffsets: Int, - isFromConsumer: Boolean, - fetchOnlyFromLeader: Boolean): Seq[Long] = { - val partition = getPartitionOrException(topicPartition) - partition.legacyFetchOffsetsForTimestamp(timestamp, maxNumOffsets, isFromConsumer, fetchOnlyFromLeader) - } - /** * Returns [[LogReadResult]] with error if a task for RemoteStorageFetchInfo could not be scheduled successfully * else returns [[None]]. @@ -2525,7 +2501,6 @@ class ReplicaManager(val config: KafkaConfig, delayedRemoteListOffsetsPurgatory.shutdown() delayedProducePurgatory.shutdown() delayedDeleteRecordsPurgatory.shutdown() - delayedElectLeaderPurgatory.shutdown() delayedShareFetchPurgatory.shutdown() if (checkpointHW) checkpointHighWatermarks() diff --git a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala index df8b6b6f6d441..bdc12c3051ecc 100644 --- a/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/AbstractCoordinatorConcurrencyTest.scala @@ -190,7 +190,6 @@ object AbstractCoordinatorConcurrencyTest { delayedProducePurgatoryParam = Some(producePurgatory), delayedFetchPurgatoryParam = Some(delayedFetchPurgatoryParam), delayedDeleteRecordsPurgatoryParam = Some(delayedDeleteRecordsPurgatoryParam), - delayedElectLeaderPurgatoryParam = Some(delayedElectLeaderPurgatoryParam), delayedRemoteFetchPurgatoryParam = Some(delayedRemoteFetchPurgatoryParam), delayedRemoteListOffsetsPurgatoryParam = Some(delayedRemoteListOffsetsPurgatoryParam), threadNamePrefix = Option(this.getClass.getName)) { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 6f6bc6630827c..a6aa6f4b5eeb9 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -2723,8 +2723,6 @@ class ReplicaManagerTest { "Fetch", timer, 0, false) val mockDeleteRecordsPurgatory = new DelayedOperationPurgatory[DelayedDeleteRecords]( "DeleteRecords", timer, 0, false) - val mockElectLeaderPurgatory = new DelayedOperationPurgatory[DelayedElectLeader]( - "ElectLeader", timer, 0, false) val mockRemoteFetchPurgatory = new DelayedOperationPurgatory[DelayedRemoteFetch]( "RemoteFetch", timer, 0, false) val mockRemoteListOffsetsPurgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]( @@ -2754,7 +2752,6 @@ class ReplicaManagerTest { delayedProducePurgatoryParam = Some(mockProducePurgatory), delayedFetchPurgatoryParam = Some(mockFetchPurgatory), delayedDeleteRecordsPurgatoryParam = Some(mockDeleteRecordsPurgatory), - delayedElectLeaderPurgatoryParam = Some(mockElectLeaderPurgatory), delayedRemoteFetchPurgatoryParam = Some(mockRemoteFetchPurgatory), delayedRemoteListOffsetsPurgatoryParam = Some(mockRemoteListOffsetsPurgatory), delayedShareFetchPurgatoryParam = Some(mockDelayedShareFetchPurgatory), @@ -3150,8 +3147,6 @@ class ReplicaManagerTest { "Fetch", timer, 0, false) val mockDeleteRecordsPurgatory = new DelayedOperationPurgatory[DelayedDeleteRecords]( "DeleteRecords", timer, 0, false) - val mockDelayedElectLeaderPurgatory = new DelayedOperationPurgatory[DelayedElectLeader]( - "DelayedElectLeader", timer, 0, false) val mockDelayedRemoteFetchPurgatory = new DelayedOperationPurgatory[DelayedRemoteFetch]( "DelayedRemoteFetch", timer, 0, false) val mockDelayedRemoteListOffsetsPurgatory = new DelayedOperationPurgatory[DelayedRemoteListOffsets]( @@ -3188,7 +3183,6 @@ class ReplicaManagerTest { delayedProducePurgatoryParam = Some(mockProducePurgatory), delayedFetchPurgatoryParam = Some(mockFetchPurgatory), delayedDeleteRecordsPurgatoryParam = Some(mockDeleteRecordsPurgatory), - delayedElectLeaderPurgatoryParam = Some(mockDelayedElectLeaderPurgatory), delayedRemoteFetchPurgatoryParam = Some(mockDelayedRemoteFetchPurgatory), delayedRemoteListOffsetsPurgatoryParam = Some(mockDelayedRemoteListOffsetsPurgatory), delayedShareFetchPurgatoryParam = Some(mockDelayedShareFetchPurgatory), diff --git a/docs/zk2kraft.html b/docs/zk2kraft.html index 123aaca4e18b1..2d3e8148c80ff 100644 --- a/docs/zk2kraft.html +++ b/docs/zk2kraft.html @@ -188,5 +188,14 @@

    Removal metrics
    In Kraft mode, Zookeeper is not used, so the metrics is removed.

    +
  • +

    + Remove the metrics for leader election purgatory. +

    +
      +
    • kafka.server:type=DelayedOperationPurgatory,delayedOperation=ElectLeader,name=PurgatorySize
    • +
    • kafka.server:type=DelayedOperationPurgatory,delayedOperation=ElectLeader,name=NumDelayedOperations
    • +
    +
  • \ No newline at end of file From ad79b4afa779a2a577358787096b4fe016e2d67e Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Thu, 23 Jan 2025 17:27:14 -0800 Subject: [PATCH 36/44] MINOR: cleanup ProcessorContextImplTest (#18682) Reviewers: Bill Bejeck --- .../internals/ProcessorContextImplTest.java | 168 ------------------ 1 file changed, 168 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java index 165fc35bad4d5..42c466c2120fe 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java @@ -141,8 +141,6 @@ private ProcessorContextImpl getStandbyContext() { @Test public void globalKeyValueStoreShouldBeReadOnly() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); when(stateManager.globalStore(anyString())).thenReturn(null); @@ -173,8 +171,6 @@ public void globalKeyValueStoreShouldBeReadOnly() { @Test public void globalTimestampedKeyValueStoreShouldBeReadOnly() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); when(stateManager.globalStore(anyString())).thenReturn(null); @@ -299,8 +295,6 @@ public void globalSessionStoreShouldBeReadOnly() { @Test public void localKeyValueStoreShouldNotAllowInitOrClose() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); when(stateManager.globalStore(anyString())).thenReturn(null); @@ -343,8 +337,6 @@ public void localKeyValueStoreShouldNotAllowInitOrClose() { @Test public void localTimestampedKeyValueStoreShouldNotAllowInitOrClose() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); when(stateManager.globalStore(anyString())).thenReturn(null); @@ -521,8 +513,6 @@ public void localSessionStoreShouldNotAllowInitOrClose() { @Test public void shouldNotSendRecordHeadersToChangelogTopic() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); when(stateManager.registeredChangelogPartitionFor(REGISTERED_STORE_NAME)).thenReturn(CHANGELOG_PARTITION); @@ -553,18 +543,9 @@ public void shouldNotSendRecordHeadersToChangelogTopic() { @Test public void shouldSendRecordHeadersToChangelogTopicWhenConsistencyEnabled() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); when(stateManager.registeredChangelogPartitionFor(REGISTERED_STORE_NAME)).thenReturn(CHANGELOG_PARTITION); - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - final Position position = Position.emptyPosition(); final Headers headers = new RecordHeaders(); headers.add(ChangelogRecordDeserializationHelper.CHANGELOG_VERSION_HEADER_RECORD_CONSISTENCY); @@ -593,17 +574,6 @@ public void shouldSendRecordHeadersToChangelogTopicWhenConsistencyEnabled() { @Test public void shouldThrowUnsupportedOperationExceptionOnLogChange() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -613,17 +583,6 @@ public void shouldThrowUnsupportedOperationExceptionOnLogChange() { @Test public void shouldThrowUnsupportedOperationExceptionOnGetStateStore() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -633,17 +592,6 @@ public void shouldThrowUnsupportedOperationExceptionOnGetStateStore() { @Test public void shouldThrowUnsupportedOperationExceptionOnForward() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -653,17 +601,6 @@ public void shouldThrowUnsupportedOperationExceptionOnForward() { @Test public void shouldThrowUnsupportedOperationExceptionOnForwardWithTo() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -673,17 +610,6 @@ public void shouldThrowUnsupportedOperationExceptionOnForwardWithTo() { @Test public void shouldThrowUnsupportedOperationExceptionOnCommit() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -693,17 +619,6 @@ public void shouldThrowUnsupportedOperationExceptionOnCommit() { @Test public void shouldThrowUnsupportedOperationExceptionOnSchedule() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -713,17 +628,6 @@ public void shouldThrowUnsupportedOperationExceptionOnSchedule() { @Test public void shouldThrowUnsupportedOperationExceptionOnTopic() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -733,17 +637,6 @@ public void shouldThrowUnsupportedOperationExceptionOnTopic() { @Test public void shouldThrowUnsupportedOperationExceptionOnPartition() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -753,17 +646,6 @@ public void shouldThrowUnsupportedOperationExceptionOnPartition() { @Test public void shouldThrowUnsupportedOperationExceptionOnOffset() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -773,17 +655,6 @@ public void shouldThrowUnsupportedOperationExceptionOnOffset() { @Test public void shouldThrowUnsupportedOperationExceptionOnTimestamp() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -793,17 +664,6 @@ public void shouldThrowUnsupportedOperationExceptionOnTimestamp() { @Test public void shouldThrowUnsupportedOperationExceptionOnCurrentNode() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -813,17 +673,6 @@ public void shouldThrowUnsupportedOperationExceptionOnCurrentNode() { @Test public void shouldThrowUnsupportedOperationExceptionOnSetRecordContext() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -833,17 +682,6 @@ public void shouldThrowUnsupportedOperationExceptionOnSetRecordContext() { @Test public void shouldThrowUnsupportedOperationExceptionOnRecordContext() { - foreachSetUp(); - - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); - - context = buildProcessorContextImpl(streamsConfig, stateManager); - - final StreamTask task = mock(StreamTask.class); - context.transitionToActive(task, null, null); - - mockProcessorNodeWithLocalKeyValueStore(); - context = getStandbyContext(); assertThrows( UnsupportedOperationException.class, @@ -853,8 +691,6 @@ public void shouldThrowUnsupportedOperationExceptionOnRecordContext() { @Test public void shouldMatchStreamTime() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); context = buildProcessorContextImpl(streamsConfig, stateManager); @@ -870,8 +706,6 @@ public void shouldMatchStreamTime() { @Test public void shouldAddAndGetProcessorKeyValue() { - foreachSetUp(); - when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); context = buildProcessorContextImpl(streamsConfig, stateManager); @@ -891,8 +725,6 @@ public void shouldAddAndGetProcessorKeyValue() { @Test public void shouldSetAndGetProcessorMetaData() { - foreachSetUp(); - context = buildProcessorContextImpl(streamsConfig, stateManager); mockProcessorNodeWithLocalKeyValueStore(); From 66868fc1faf0c55c2a8a2cd118ab5db7396a1a0c Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 24 Jan 2025 18:11:05 +0800 Subject: [PATCH 37/44] KAFKA-18620: Remove UnifiedLog#legacyFetchOffsetsBefore (#18686) Reviewers: Mickael Maison --- .../src/main/scala/kafka/log/UnifiedLog.scala | 45 ------------------- .../unit/kafka/server/LogOffsetTest.scala | 40 +---------------- 2 files changed, 1 insertion(+), 84 deletions(-) diff --git a/core/src/main/scala/kafka/log/UnifiedLog.scala b/core/src/main/scala/kafka/log/UnifiedLog.scala index 3253de11dfec2..3ad3b9f6139db 100644 --- a/core/src/main/scala/kafka/log/UnifiedLog.scala +++ b/core/src/main/scala/kafka/log/UnifiedLog.scala @@ -1331,51 +1331,6 @@ class UnifiedLog(@volatile var logStartOffset: Long, targetSeg.flatMap(_.findOffsetByTimestamp(targetTimestamp, startOffset).toScala) } - def legacyFetchOffsetsBefore(timestamp: Long, maxNumOffsets: Int): Seq[Long] = { - // Cache to avoid race conditions. `toBuffer` is faster than most alternatives and provides - // constant time access while being safe to use with concurrent collections unlike `toArray`. - val allSegments = logSegments.asScala.toBuffer - val lastSegmentHasSize = allSegments.last.size > 0 - - val offsetTimeArray = - if (lastSegmentHasSize) - new Array[(Long, Long)](allSegments.length + 1) - else - new Array[(Long, Long)](allSegments.length) - - for (i <- allSegments.indices) - offsetTimeArray(i) = (math.max(allSegments(i).baseOffset, logStartOffset), allSegments(i).lastModified) - if (lastSegmentHasSize) - offsetTimeArray(allSegments.length) = (logEndOffset, time.milliseconds) - - var startIndex = -1 - timestamp match { - case ListOffsetsRequest.LATEST_TIMESTAMP => - startIndex = offsetTimeArray.length - 1 - case ListOffsetsRequest.EARLIEST_TIMESTAMP => - startIndex = 0 - case _ => - var isFound = false - debug("Offset time array = " + offsetTimeArray.foreach(o => "%d, %d".format(o._1, o._2))) - startIndex = offsetTimeArray.length - 1 - while (startIndex >= 0 && !isFound) { - if (offsetTimeArray(startIndex)._2 <= timestamp) - isFound = true - else - startIndex -= 1 - } - } - - val retSize = maxNumOffsets.min(startIndex + 1) - val ret = new Array[Long](retSize) - for (j <- 0 until retSize) { - ret(j) = offsetTimeArray(startIndex)._1 - startIndex -= 1 - } - // ensure that the returned seq is in descending order of offsets - ret.toSeq.sortBy(-_) - } - /** * Given a message offset, find its corresponding offset metadata in the log. * 1. If the message offset is less than the log-start-offset (or) local-log-start-offset, then it returns the diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index efb057bd1cb3b..8d8a81c972508 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -25,19 +25,13 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.record.FileRecords import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.{IsolationLevel, TopicPartition} -import org.apache.kafka.storage.internals.log.{LogSegment, LogStartOffsetIncrementReason, OffsetResultHolder} +import org.apache.kafka.storage.internals.log.{LogStartOffsetIncrementReason, OffsetResultHolder} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Timeout import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource -import org.mockito.Mockito.{mock, when} -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import java.io.File -import java.util -import java.util.Arrays.asList -import java.util.concurrent.atomic.AtomicInteger import java.util.{Optional, Properties, Random} import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -228,38 +222,6 @@ class LogOffsetTest extends BaseRequestTest { assertEquals(0L, offsetFromResponse) } - /* We test that `fetchOffsetsBefore` works correctly if `LogSegment.size` changes after each invocation (simulating - * a race condition) */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchOffsetsBeforeWithChangingSegmentSize(quorum: String): Unit = { - val log: UnifiedLog = mock(classOf[UnifiedLog]) - val logSegment: LogSegment = mock(classOf[LogSegment]) - when(logSegment.size).thenAnswer(new Answer[Int] { - private[this] val value = new AtomicInteger(0) - override def answer(invocation: InvocationOnMock): Int = value.getAndIncrement() - }) - val logSegments = Seq(logSegment).asJava - when(log.logSegments).thenReturn(logSegments) - log.legacyFetchOffsetsBefore(System.currentTimeMillis, 100) - } - - /* We test that `fetchOffsetsBefore` works correctly if `Log.logSegments` content and size are - * different (simulating a race condition) */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchOffsetsBeforeWithChangingSegments(quorum: String): Unit = { - val log: UnifiedLog = mock(classOf[UnifiedLog]) - val logSegment: LogSegment = mock(classOf[LogSegment]) - when(log.logSegments).thenReturn( - new util.AbstractCollection[LogSegment] { - override def size = 2 - override def iterator = asList(logSegment).iterator - } - ) - log.legacyFetchOffsetsBefore(System.currentTimeMillis, 100) - } - private def broker: KafkaBroker = brokers.head private def sendListOffsetsRequest(request: ListOffsetsRequest): ListOffsetsResponse = { From 356f0d815cf99668e5acea427004bf01d8068439 Mon Sep 17 00:00:00 2001 From: Logan Zhu Date: Fri, 24 Jan 2025 18:21:34 +0800 Subject: [PATCH 38/44] KAFKA-18597 Fix max-buffer-utilization-percent is always 0 (#18627) Reviewers: Chia-Ping Tsai --- .../src/main/scala/kafka/log/LogCleaner.scala | 19 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 168 +++++++++++++++--- 2 files changed, 153 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index c07437e68d599..4f5aaa4137557 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -116,14 +116,14 @@ class LogCleaner(initialConfig: CleanerConfig, /** * @param f to compute the result - * @return the max value (int value) or 0 if there is no cleaner + * @return the max value or 0 if there is no cleaner */ - private[log] def maxOverCleanerThreads(f: CleanerThread => Double): Int = - cleaners.map(f).maxOption.getOrElse(0.0d).toInt + private[log] def maxOverCleanerThreads(f: CleanerThread => Double): Double = + cleaners.map(f).maxOption.getOrElse(0.0d) /* a metric to track the maximum utilization of any thread's buffer in the last cleaning */ metricsGroup.newGauge(MaxBufferUtilizationPercentMetricName, - () => maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100) + () => (maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100).toInt) /* a metric to track the recopy rate of each thread's last cleaning */ metricsGroup.newGauge(CleanerRecopyPercentMetricName, () => { @@ -133,12 +133,12 @@ class LogCleaner(initialConfig: CleanerConfig, }) /* a metric to track the maximum cleaning time for the last cleaning from each thread */ - metricsGroup.newGauge(MaxCleanTimeMetricName, () => maxOverCleanerThreads(_.lastStats.elapsedSecs)) + metricsGroup.newGauge(MaxCleanTimeMetricName, () => maxOverCleanerThreads(_.lastStats.elapsedSecs).toInt) // a metric to track delay between the time when a log is required to be compacted // as determined by max compaction lag and the time of last cleaner run. metricsGroup.newGauge(MaxCompactionDelayMetricsName, - () => maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000) + () => (maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000).toInt) metricsGroup.newGauge(DeadThreadCountMetricName, () => deadThreadCount) @@ -522,10 +522,11 @@ object LogCleaner { } - private val MaxBufferUtilizationPercentMetricName = "max-buffer-utilization-percent" + // Visible for test. + private[log] val MaxBufferUtilizationPercentMetricName = "max-buffer-utilization-percent" private val CleanerRecopyPercentMetricName = "cleaner-recopy-percent" - private val MaxCleanTimeMetricName = "max-clean-time-secs" - private val MaxCompactionDelayMetricsName = "max-compaction-delay-secs" + private[log] val MaxCleanTimeMetricName = "max-clean-time-secs" + private[log] val MaxCompactionDelayMetricsName = "max-compaction-delay-secs" private val DeadThreadCountMetricName = "DeadThreadCount" // package private for testing private[log] val MetricNames = Set( diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index b83a36a4b5ddd..7b22628c382bc 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -17,6 +17,7 @@ package kafka.log +import kafka.log.LogCleaner.{MaxBufferUtilizationPercentMetricName, MaxCleanTimeMetricName, MaxCompactionDelayMetricsName} import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, Logging, Pool, TestUtils} import org.apache.kafka.common.TopicPartition @@ -2046,42 +2047,159 @@ class LogCleanerTest extends Logging { } @Test - def testMaxOverCleanerThreads(): Unit = { - val logCleaner = new LogCleaner(new CleanerConfig(true), + def testMaxBufferUtilizationPercentMetric(): Unit = { + val logCleaner = new LogCleaner( + new CleanerConfig(true), logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()), logs = new Pool[TopicPartition, UnifiedLog](), logDirFailureChannel = new LogDirFailureChannel(1), - time = time) + time = time + ) + + def assertMaxBufferUtilizationPercent(expected: Int): Unit = { + val gauge = logCleaner.metricsGroup.newGauge(MaxBufferUtilizationPercentMetricName, + () => (logCleaner.maxOverCleanerThreads(_.lastStats.bufferUtilization) * 100).toInt) + assertEquals(expected, gauge.value()) + } + + try { + // No CleanerThreads + assertMaxBufferUtilizationPercent(0) + + val cleaners = logCleaner.cleaners + + val cleaner1 = new logCleaner.CleanerThread(1) + cleaner1.lastStats = new CleanerStats(time) + cleaner1.lastStats.bufferUtilization = 0.75 + cleaners += cleaner1 + + val cleaner2 = new logCleaner.CleanerThread(2) + cleaner2.lastStats = new CleanerStats(time) + cleaner2.lastStats.bufferUtilization = 0.85 + cleaners += cleaner2 + + val cleaner3 = new logCleaner.CleanerThread(3) + cleaner3.lastStats = new CleanerStats(time) + cleaner3.lastStats.bufferUtilization = 0.65 + cleaners += cleaner3 + + // expect the gauge value to reflect the maximum bufferUtilization + assertMaxBufferUtilizationPercent(85) + + // Update bufferUtilization and verify the gauge value updates + cleaner1.lastStats.bufferUtilization = 0.9 + assertMaxBufferUtilizationPercent(90) + + // All CleanerThreads have the same bufferUtilization + cleaners.foreach(_.lastStats.bufferUtilization = 0.5) + assertMaxBufferUtilizationPercent(50) + } finally { + logCleaner.shutdown() + } + } + + @Test + def testMaxCleanTimeMetric(): Unit = { + val logCleaner = new LogCleaner( + new CleanerConfig(true), + logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()), + logs = new Pool[TopicPartition, UnifiedLog](), + logDirFailureChannel = new LogDirFailureChannel(1), + time = time + ) + + def assertMaxCleanTime(expected: Int): Unit = { + val gauge = logCleaner.metricsGroup.newGauge(MaxCleanTimeMetricName, + () => logCleaner.maxOverCleanerThreads(_.lastStats.elapsedSecs).toInt) + assertEquals(expected, gauge.value()) + } - val cleaners = logCleaner.cleaners + try { + // No CleanerThreads + assertMaxCleanTime(0) - val cleaner1 = new logCleaner.CleanerThread(1) - cleaner1.lastStats = new CleanerStats(time) - cleaner1.lastStats.bufferUtilization = 0.75 - cleaners += cleaner1 + val cleaners = logCleaner.cleaners - val cleaner2 = new logCleaner.CleanerThread(2) - cleaner2.lastStats = new CleanerStats(time) - cleaner2.lastStats.bufferUtilization = 0.85 - cleaners += cleaner2 + val cleaner1 = new logCleaner.CleanerThread(1) + cleaner1.lastStats = new CleanerStats(time) + cleaner1.lastStats.endTime = cleaner1.lastStats.startTime + 1_000L + cleaners += cleaner1 - val cleaner3 = new logCleaner.CleanerThread(3) - cleaner3.lastStats = new CleanerStats(time) - cleaner3.lastStats.bufferUtilization = 0.65 - cleaners += cleaner3 + val cleaner2 = new logCleaner.CleanerThread(2) + cleaner2.lastStats = new CleanerStats(time) + cleaner2.lastStats.endTime = cleaner2.lastStats.startTime + 2_000L + cleaners += cleaner2 - assertEquals(0, logCleaner.maxOverCleanerThreads(_.lastStats.bufferUtilization)) + val cleaner3 = new logCleaner.CleanerThread(3) + cleaner3.lastStats = new CleanerStats(time) + cleaner3.lastStats.endTime = cleaner3.lastStats.startTime + 3_000L + cleaners += cleaner3 - cleaners.clear() + // expect the gauge value to reflect the maximum cleanTime + assertMaxCleanTime(3) - cleaner1.lastStats.bufferUtilization = 5d - cleaners += cleaner1 - cleaner2.lastStats.bufferUtilization = 6d - cleaners += cleaner2 - cleaner3.lastStats.bufferUtilization = 7d - cleaners += cleaner3 + // Update cleanTime and verify the gauge value updates + cleaner1.lastStats.endTime = cleaner1.lastStats.startTime + 4_000L + assertMaxCleanTime(4) - assertEquals(7, logCleaner.maxOverCleanerThreads(_.lastStats.bufferUtilization)) + // All CleanerThreads have the same cleanTime + cleaners.foreach(cleaner => cleaner.lastStats.endTime = cleaner.lastStats.startTime + 1_500L) + assertMaxCleanTime(1) + } finally { + logCleaner.shutdown() + } + } + + @Test + def testMaxCompactionDelayMetrics(): Unit = { + val logCleaner = new LogCleaner( + new CleanerConfig(true), + logDirs = Array(TestUtils.tempDir(), TestUtils.tempDir()), + logs = new Pool[TopicPartition, UnifiedLog](), + logDirFailureChannel = new LogDirFailureChannel(1), + time = time + ) + + def assertMaxCompactionDelay(expected: Int): Unit = { + val gauge = logCleaner.metricsGroup.newGauge(MaxCompactionDelayMetricsName, + () => (logCleaner.maxOverCleanerThreads(_.lastPreCleanStats.maxCompactionDelayMs.toDouble) / 1000).toInt) + assertEquals(expected, gauge.value()) + } + + try { + // No CleanerThreads + assertMaxCompactionDelay(0) + + val cleaners = logCleaner.cleaners + + val cleaner1 = new logCleaner.CleanerThread(1) + cleaner1.lastStats = new CleanerStats(time) + cleaner1.lastPreCleanStats.maxCompactionDelayMs = 1_000L + cleaners += cleaner1 + + val cleaner2 = new logCleaner.CleanerThread(2) + cleaner2.lastStats = new CleanerStats(time) + cleaner2.lastPreCleanStats.maxCompactionDelayMs = 2_000L + cleaners += cleaner2 + + val cleaner3 = new logCleaner.CleanerThread(3) + cleaner3.lastStats = new CleanerStats(time) + cleaner3.lastPreCleanStats.maxCompactionDelayMs = 3_000L + cleaners += cleaner3 + + // expect the gauge value to reflect the maximum CompactionDelay + assertMaxCompactionDelay(3) + + // Update CompactionDelay and verify the gauge value updates + cleaner1.lastPreCleanStats.maxCompactionDelayMs = 4_000L + assertMaxCompactionDelay(4) + + // All CleanerThreads have the same CompactionDelay + cleaners.foreach(_.lastPreCleanStats.maxCompactionDelayMs = 1_500L) + assertMaxCompactionDelay(1) + } finally { + logCleaner.shutdown() + } } private def writeToLog(log: UnifiedLog, keysAndValues: Iterable[(Int, Int)], offsetSeq: Iterable[Long]): Iterable[Long] = { From fa2df3bca75fdd72a94890bbef34aaa8a48e76c7 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 24 Jan 2025 19:39:01 +0800 Subject: [PATCH 39/44] KAFKA-18559 Cleanup FinalizedFeatures (#18593) Reviewers: Ismael Juma , Chia-Ping Tsai --- .../server/metadata/KRaftMetadataCache.scala | 3 +-- .../TransactionCoordinatorConcurrencyTest.scala | 4 +--- .../transaction/TransactionStateManagerTest.scala | 8 ++------ .../scala/unit/kafka/network/ProcessorTest.scala | 6 +++--- .../unit/kafka/network/SocketServerTest.scala | 2 +- .../scala/unit/kafka/server/KafkaApisTest.scala | 6 ++---- .../metadata/publisher/FeaturesPublisher.java | 4 ++-- .../kafka/server/common/FinalizedFeatures.java | 15 ++++----------- .../server/common/FinalizedFeaturesTest.java | 13 +------------ 9 files changed, 17 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala index 51c05d68c3215..be13635c1ab97 100644 --- a/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala +++ b/core/src/main/scala/kafka/server/metadata/KRaftMetadataCache.scala @@ -540,8 +540,7 @@ class KRaftMetadataCache( } new FinalizedFeatures(image.features().metadataVersion(), finalizedFeatures, - image.highestOffsetAndEpoch().offset, - true) + image.highestOffsetAndEpoch().offset) } } diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala index 24000894fe9bb..621730bc65eb9 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala @@ -85,9 +85,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren new FinalizedFeatures( MetadataVersion.latestTesting(), Collections.singletonMap(TransactionVersion.FEATURE_NAME, TransactionVersion.TV_2.featureLevel()), - 0, - true - ) + 0) } when(metadataCache.metadataVersion()) diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala index 41e6b1a954a5f..522461e5485e0 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala @@ -70,9 +70,7 @@ class TransactionStateManagerTest { new FinalizedFeatures( MetadataVersion.latestTesting(), Collections.singletonMap(TransactionVersion.FEATURE_NAME, TransactionVersion.TV_2.featureLevel()), - 0, - true - ) + 0) } val metrics = new Metrics() @@ -1332,9 +1330,7 @@ class TransactionStateManagerTest { new FinalizedFeatures( MetadataVersion.latestTesting(), Collections.singletonMap(TransactionVersion.FEATURE_NAME, transactionVersion.featureLevel()), - 0, - true - ) + 0) } val transactionManager = new TransactionStateManager(0, scheduler, replicaManager, metadataCache, txnConfig, time, metrics) diff --git a/core/src/test/scala/unit/kafka/network/ProcessorTest.scala b/core/src/test/scala/unit/kafka/network/ProcessorTest.scala index 94e93d4d2a801..3a862678ca79b 100644 --- a/core/src/test/scala/unit/kafka/network/ProcessorTest.scala +++ b/core/src/test/scala/unit/kafka/network/ProcessorTest.scala @@ -37,7 +37,7 @@ class ProcessorTest { val requestHeader = RequestTestUtils.serializeRequestHeader( new RequestHeader(ApiKeys.INIT_PRODUCER_ID, 0, "clientid", 0)) val apiVersionManager = new SimpleApiVersionManager(ListenerType.CONTROLLER, true, - () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0)) val e = assertThrows(classOf[InvalidRequestException], (() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable, "INIT_PRODUCER_ID with listener type CONTROLLER should throw InvalidRequestException exception") @@ -55,7 +55,7 @@ class ProcessorTest { .setCorrelationId(0); val requestHeader = RequestTestUtils.serializeRequestHeader(new RequestHeader(requestHeaderData, headerVersion)) val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true, - () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0)) val e = assertThrows(classOf[InvalidRequestException], (() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable, "LEADER_AND_ISR should throw InvalidRequestException exception") @@ -67,7 +67,7 @@ class ProcessorTest { val requestHeader = RequestTestUtils.serializeRequestHeader( new RequestHeader(ApiKeys.PRODUCE, 0, "clientid", 0)) val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true, - () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0)) val e = assertThrows(classOf[UnsupportedVersionException], (() => Processor.parseRequestHeader(apiVersionManager, requestHeader)): Executable, "PRODUCE v0 should throw UnsupportedVersionException exception") diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 5ebcfd65ccec2..5b2196018e557 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -84,7 +84,7 @@ class SocketServerTest { TestUtils.clearYammerMetrics() private val apiVersionManager = new SimpleApiVersionManager(ListenerType.BROKER, true, - () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0)) var server: SocketServer = _ val sockets = new ArrayBuffer[Socket] diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index af26996c56eb2..901309b289ded 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -179,7 +179,7 @@ class KafkaApisTest extends Logging { enabledApis, BrokerFeatures.defaultSupportedFeatures(true), true, - () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0, true)) + () => new FinalizedFeatures(MetadataVersion.latestTesting(), Collections.emptyMap[String, java.lang.Short], 0)) when(groupCoordinator.isNewGroupCoordinator).thenReturn(config.isNewGroupCoordinatorEnabled) setupFeatures(featureVersions) @@ -220,9 +220,7 @@ class KafkaApisTest extends Logging { featureVersions.map { featureVersion => featureVersion.featureName -> featureVersion.featureLevel.asInstanceOf[java.lang.Short] }.toMap.asJava, - 0, - true - ) + 0) } case _ => throw new IllegalStateException("Test must set an instance of KRaftMetadataCache") diff --git a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java index a03f08291b5d8..01572dd941143 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/publisher/FeaturesPublisher.java @@ -57,8 +57,8 @@ public void onMetadataUpdate( if (delta.featuresDelta() != null) { FinalizedFeatures newFinalizedFeatures = new FinalizedFeatures(newImage.features().metadataVersion(), newImage.features().finalizedVersions(), - newImage.provenance().lastContainedOffset(), - true); + newImage.provenance().lastContainedOffset() + ); if (!newFinalizedFeatures.equals(finalizedFeatures)) { log.info("Loaded new metadata {}.", newFinalizedFeatures); finalizedFeatures = newFinalizedFeatures; diff --git a/server-common/src/main/java/org/apache/kafka/server/common/FinalizedFeatures.java b/server-common/src/main/java/org/apache/kafka/server/common/FinalizedFeatures.java index de78a3a72a883..1eb394664094d 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/FinalizedFeatures.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/FinalizedFeatures.java @@ -27,25 +27,18 @@ public final class FinalizedFeatures { private final long finalizedFeaturesEpoch; public static FinalizedFeatures fromKRaftVersion(MetadataVersion version) { - return new FinalizedFeatures(version, Collections.emptyMap(), -1, true); + return new FinalizedFeatures(version, Collections.emptyMap(), -1); } public FinalizedFeatures( MetadataVersion metadataVersion, Map finalizedFeatures, - long finalizedFeaturesEpoch, - boolean kraftMode + long finalizedFeaturesEpoch ) { - this.metadataVersion = metadataVersion; + this.metadataVersion = Objects.requireNonNull(metadataVersion); this.finalizedFeatures = new HashMap<>(finalizedFeatures); this.finalizedFeaturesEpoch = finalizedFeaturesEpoch; - // In KRaft mode, we always include the metadata version in the features map. - // In ZK mode, we never include it. - if (kraftMode) { - this.finalizedFeatures.put(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel()); - } else { - this.finalizedFeatures.remove(MetadataVersion.FEATURE_NAME); - } + this.finalizedFeatures.put(MetadataVersion.FEATURE_NAME, metadataVersion.featureLevel()); } public MetadataVersion metadataVersion() { diff --git a/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java b/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java index ae6ca998df2da..31d57bedfe585 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/FinalizedFeaturesTest.java @@ -24,27 +24,16 @@ import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_KRAFT_VERSION; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; class FinalizedFeaturesTest { @Test public void testKRaftModeFeatures() { FinalizedFeatures finalizedFeatures = new FinalizedFeatures(MINIMUM_KRAFT_VERSION, - Collections.singletonMap("foo", (short) 2), 123, true); + Collections.singletonMap("foo", (short) 2), 123); assertEquals(MINIMUM_KRAFT_VERSION.featureLevel(), finalizedFeatures.finalizedFeatures().get(FEATURE_NAME)); assertEquals((short) 2, finalizedFeatures.finalizedFeatures().get("foo")); assertEquals(2, finalizedFeatures.finalizedFeatures().size()); } - - @Test - public void testZkModeFeatures() { - FinalizedFeatures finalizedFeatures = new FinalizedFeatures(MINIMUM_KRAFT_VERSION, - Collections.singletonMap("foo", (short) 2), 123, false); - assertNull(finalizedFeatures.finalizedFeatures().get(FEATURE_NAME)); - assertEquals((short) 2, - finalizedFeatures.finalizedFeatures().get("foo")); - assertEquals(1, finalizedFeatures.finalizedFeatures().size()); - } } From 5d81fe20c836670cbb763f6976ab3e59957feb34 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Fri, 24 Jan 2025 20:12:03 +0800 Subject: [PATCH 40/44] KAFKA-18590 Cleanup DelegationTokenManager (#18618) Reviewers: Ismael Juma , Chia-Ping Tsai --- .../scala/kafka/server/BrokerServer.scala | 4 - .../kafka/server/DelegationTokenManager.scala | 153 +----------------- .../main/scala/kafka/server/KafkaApis.scala | 10 +- 3 files changed, 12 insertions(+), 155 deletions(-) diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 3c735be0004c4..b229ce3409d76 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -355,7 +355,6 @@ class BrokerServer( /* start token manager */ tokenManager = new DelegationTokenManager(config, tokenCache, time) - tokenManager.startup() /* initializing the groupConfigManager */ groupConfigManager = new GroupConfigManager(config.groupCoordinatorConfig.extractGroupConfigMap(config.shareGroupConfig)) @@ -787,9 +786,6 @@ class BrokerServer( if (shareCoordinator.isDefined) CoreUtils.swallow(shareCoordinator.get.shutdown(), this) - if (tokenManager != null) - CoreUtils.swallow(tokenManager.shutdown(), this) - if (assignmentsManager != null) CoreUtils.swallow(assignmentsManager.close(), this) diff --git a/core/src/main/scala/kafka/server/DelegationTokenManager.scala b/core/src/main/scala/kafka/server/DelegationTokenManager.scala index 4fa4cae209f11..b74e57999fb18 100644 --- a/core/src/main/scala/kafka/server/DelegationTokenManager.scala +++ b/core/src/main/scala/kafka/server/DelegationTokenManager.scala @@ -17,14 +17,11 @@ package kafka.server -import java.nio.ByteBuffer import java.nio.charset.StandardCharsets import java.security.InvalidKeyException - import javax.crypto.spec.SecretKeySpec import javax.crypto.{Mac, SecretKey} import kafka.utils.Logging -import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.scram.internals.{ScramFormatter, ScramMechanism} import org.apache.kafka.common.security.scram.ScramCredential @@ -32,24 +29,15 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.Time +import java.util +import java.util.stream.Collectors import scala.jdk.CollectionConverters._ import scala.collection.mutable object DelegationTokenManager { private val DefaultHmacAlgorithm = "HmacSHA512" - val CurrentVersion = 3 val ErrorTimestamp = -1 - /** - * - * @param tokenId - * @param secretKey - * @return - */ - def createHmac(tokenId: String, secretKey: String) : Array[Byte] = { - createHmac(tokenId, createSecretKey(secretKey.getBytes(StandardCharsets.UTF_8))) - } - /** * Convert the byte[] to a secret key * @param keybytes the byte[] to create the secret key from @@ -102,14 +90,8 @@ class DelegationTokenManager(val config: KafkaConfig, val time: Time) extends Logging { this.logIdent = s"[Token Manager on Node ${config.brokerId}]: " - protected val lock = new Object() - import DelegationTokenManager._ - type CreateResponseCallback = CreateTokenResult => Unit - type RenewResponseCallback = (Errors, Long) => Unit - type ExpireResponseCallback = (Errors, Long) => Unit - val secretKey: SecretKey = { val keyBytes = if (config.tokenAuthEnabled) config.delegationTokenSecretKey.value.getBytes(StandardCharsets.UTF_8) else null if (keyBytes == null || keyBytes.isEmpty) null @@ -117,26 +99,6 @@ class DelegationTokenManager(val config: KafkaConfig, createSecretKey(keyBytes) } - val tokenMaxLifetime: Long = config.delegationTokenMaxLifeMs - val defaultTokenRenewTime: Long = config.delegationTokenExpiryTimeMs - - def startup(): Unit = { - // Nothing to do. Overridden for Zk case - } - - def shutdown(): Unit = { - // Nothing to do. Overridden for Zk case - } - - /** - * - * @param token - */ - protected def updateCache(token: DelegationToken): Unit = { - val hmacString = token.hmacAsBase64String - val scramCredentialMap = prepareScramCredentials(hmacString) - tokenCache.updateCache(token, scramCredentialMap.asJava) - } /** * @param hmacString */ @@ -157,38 +119,9 @@ class DelegationTokenManager(val config: KafkaConfig, * @param token */ def updateToken(token: DelegationToken): Unit = { - updateCache(token) - } - - /** - * - * @param owner - * @param renewers - * @param maxLifeTimeMs - * @param responseCallback - */ - def createToken(owner: KafkaPrincipal, - tokenRequester: KafkaPrincipal, - renewers: List[KafkaPrincipal], - maxLifeTimeMs: Long, - responseCallback: CreateResponseCallback): Unit = { - // Must be forwarded to KRaft Controller or handled in DelegationTokenManagerZk - throw new IllegalStateException("API createToken was not forwarded to a handler.") - } - - /** - * - * @param principal - * @param hmac - * @param renewLifeTimeMs - * @param renewCallback - */ - def renewToken(principal: KafkaPrincipal, - hmac: ByteBuffer, - renewLifeTimeMs: Long, - renewCallback: RenewResponseCallback): Unit = { - // Must be forwarded to KRaft Controller or handled in DelegationTokenManagerZk - throw new IllegalStateException("API renewToken was not forwarded to a handler.") + val hmacString = token.hmacAsBase64String + val scramCredentialMap = prepareScramCredentials(hmacString) + tokenCache.updateCache(token, scramCredentialMap.asJava) } def getDelegationToken(tokenInfo: TokenInformation): DelegationToken = { @@ -196,87 +129,15 @@ class DelegationTokenManager(val config: KafkaConfig, new DelegationToken(tokenInfo, hmac) } - /** - * - * @param principal - * @param hmac - * @param expireLifeTimeMs - * @param expireResponseCallback - */ - def expireToken(principal: KafkaPrincipal, - hmac: ByteBuffer, - expireLifeTimeMs: Long, - expireResponseCallback: ExpireResponseCallback): Unit = { - // Must be forwarded to KRaft Controller or handled in DelegationTokenManagerZk - throw new IllegalStateException("API expireToken was not forwarded to a handler.") - } - /** * * @param tokenId */ def removeToken(tokenId: String): Unit = { - removeCache(tokenId) - } - - /** - * - * @param tokenId - */ - protected def removeCache(tokenId: String): Unit = { tokenCache.removeCache(tokenId) } - /** - * - * @return - */ - def expireTokens(): Unit = { - lock.synchronized { - for (tokenInfo <- getAllTokenInformation) { - val now = time.milliseconds - if (tokenInfo.maxTimestamp < now || tokenInfo.expiryTimestamp < now) { - info(s"Delegation token expired for token: ${tokenInfo.tokenId} for owner: ${tokenInfo.owner}") - removeToken(tokenInfo.tokenId) - } - } - } - } - - def getAllTokenInformation: List[TokenInformation] = tokenCache.tokens.asScala.toList - - def getTokens(filterToken: TokenInformation => Boolean): List[DelegationToken] = { - getAllTokenInformation.filter(filterToken).map(token => getDelegationToken(token)) - } - -} - -case class CreateTokenResult(owner: KafkaPrincipal, - tokenRequester: KafkaPrincipal, - issueTimestamp: Long, - expiryTimestamp: Long, - maxTimestamp: Long, - tokenId: String, - hmac: Array[Byte], - error: Errors) { - - override def equals(other: Any): Boolean = { - other match { - case that: CreateTokenResult => - error.equals(that.error) && - owner.equals(that.owner) && - tokenRequester.equals(that.tokenRequester) && - tokenId.equals(that.tokenId) && - issueTimestamp.equals(that.issueTimestamp) && - expiryTimestamp.equals(that.expiryTimestamp) && - maxTimestamp.equals(that.maxTimestamp) && - (hmac sameElements that.hmac) - case _ => false - } - } - - override def hashCode(): Int = { - val fields = Seq(owner, tokenRequester, issueTimestamp, expiryTimestamp, maxTimestamp, tokenId, hmac, error) - fields.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) + def getTokens(filterToken: TokenInformation => Boolean): util.List[DelegationToken] = { + tokenCache.tokens.stream().filter(token => filterToken(token)).map(getDelegationToken).collect(Collectors.toList[DelegationToken]) } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6e25db6b05886..8d0d62667f285 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -2247,22 +2247,22 @@ class KafkaApis(val requestChannel: RequestChannel, val describeTokenRequest = request.body[DescribeDelegationTokenRequest] // the callback for sending a describe token response - def sendResponseCallback(error: Errors, tokenDetails: List[DelegationToken]): Unit = { + def sendResponseCallback(error: Errors, tokenDetails: util.List[DelegationToken]): Unit = { requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs => - new DescribeDelegationTokenResponse(request.context.requestVersion(), requestThrottleMs, error, tokenDetails.asJava)) + new DescribeDelegationTokenResponse(request.context.requestVersion(), requestThrottleMs, error, tokenDetails)) trace("Sending describe token response for correlation id %d to client %s." .format(request.header.correlationId, request.header.clientId)) } if (!allowTokenRequests(request)) - sendResponseCallback(Errors.DELEGATION_TOKEN_REQUEST_NOT_ALLOWED, List.empty) + sendResponseCallback(Errors.DELEGATION_TOKEN_REQUEST_NOT_ALLOWED, Collections.emptyList) else if (!config.tokenAuthEnabled) - sendResponseCallback(Errors.DELEGATION_TOKEN_AUTH_DISABLED, List.empty) + sendResponseCallback(Errors.DELEGATION_TOKEN_AUTH_DISABLED, Collections.emptyList) else { val requestPrincipal = request.context.principal if (describeTokenRequest.ownersListEmpty()) { - sendResponseCallback(Errors.NONE, List()) + sendResponseCallback(Errors.NONE, Collections.emptyList) } else { val owners = if (describeTokenRequest.data.owners == null) From 17846fe743aabedd2b556a0331731a5e899e1d65 Mon Sep 17 00:00:00 2001 From: Okada Haruki Date: Fri, 24 Jan 2025 21:23:43 +0900 Subject: [PATCH 41/44] KAFKA-16372 Fix producer doc discrepancy with the exception behavior (#15574) Currently, Producer.send doc is inconsistent with actual exception behavior - TimeoutException: This won't be thrown from send on buffer-full or metadata-missing actually. Instead, it will returned as failed future. - AuthenticationException/AuthorizationException: These exceptions are also won't be thrown. Returned with failed future actually. Fixed Callback javadoc and ProducerConfig doc as well. Reviewers: Luke Chen , Andrew Schofield --- .../java/org/apache/kafka/clients/producer/Callback.java | 3 +++ .../org/apache/kafka/clients/producer/KafkaProducer.java | 8 ++------ .../org/apache/kafka/clients/producer/ProducerConfig.java | 6 +++--- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java index 29acb88044be6..2eb7ebc882bc3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java @@ -42,6 +42,8 @@ public interface Callback { *
  • {@link org.apache.kafka.common.errors.UnknownServerException UnknownServerException} *
  • {@link org.apache.kafka.common.errors.UnknownProducerIdException UnknownProducerIdException} *
  • {@link org.apache.kafka.common.errors.InvalidProducerEpochException InvalidProducerEpochException} + *
  • {@link org.apache.kafka.common.errors.AuthenticationException AuthenticationException} + *
  • {@link org.apache.kafka.common.errors.AuthorizationException AuthorizationException} * * Retriable exceptions (transient, may be covered by increasing #.retries): *
      @@ -52,6 +54,7 @@ public interface Callback { *
    • {@link org.apache.kafka.common.errors.OffsetOutOfRangeException OffsetOutOfRangeException} *
    • {@link org.apache.kafka.common.errors.TimeoutException TimeoutException} *
    • {@link org.apache.kafka.common.errors.UnknownTopicOrPartitionException UnknownTopicOrPartitionException} + *
    • {@link org.apache.kafka.clients.producer.BufferExhaustedException BufferExhaustedException} *
    */ void onCompletion(RecordMetadata metadata, Exception exception); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 64930b36e27e4..130cfcb7bfb4e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -149,8 +149,8 @@ *

    * The buffer.memory controls the total amount of memory available to the producer for buffering. If records * are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is - * exhausted additional send calls will block. The threshold for time to block is determined by max.block.ms after which it throws - * a TimeoutException. + * exhausted additional send calls will block. The threshold for time to block is determined by max.block.ms after which it returns + * a failed future with BufferExhaustedException. *

    * The key.serializer and value.serializer instruct how to turn the key and value objects the user provides with * their ProducerRecord into bytes. You can use the included {@link org.apache.kafka.common.serialization.ByteArraySerializer} or @@ -926,14 +926,10 @@ public Future send(ProducerRecord record) { * @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null * indicates no callback) * - * @throws AuthenticationException if authentication fails. See the exception for more details - * @throws AuthorizationException fatal error indicating that the producer is not allowed to write * @throws IllegalStateException if a transactional.id has been configured and no transaction has been started, or * when send is invoked after producer has been closed. * @throws InterruptException If the thread is interrupted while blocked * @throws SerializationException If the key or value are not valid objects given the configured serializers - * @throws TimeoutException If the record could not be appended to the send buffer due to memory unavailable - * or missing metadata within {@code max.block.ms}. * @throws KafkaException If a Kafka related error occurs that does not belong to the public API exceptions. */ @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 23dd02bda98f3..295feee824009 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -209,7 +209,7 @@ public class ProducerConfig extends AbstractConfig { /** buffer.memory */ public static final String BUFFER_MEMORY_CONFIG = "buffer.memory"; private static final String BUFFER_MEMORY_DOC = "The total bytes of memory the producer can use to buffer records waiting to be sent to the server. If records are " - + "sent faster than they can be delivered to the server the producer will block for " + MAX_BLOCK_MS_CONFIG + " after which it will throw an exception." + + "sent faster than they can be delivered to the server the producer will block for " + MAX_BLOCK_MS_CONFIG + " after which it will fail with an exception." + "

    " + "This setting should correspond roughly to the total memory the producer will use, but is not a hard bound since " + "not all memory the producer uses is used for buffering. Some additional memory will be used for compression (if " @@ -309,8 +309,8 @@ public class ProducerConfig extends AbstractConfig { public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; private static final String PARTITIONER_CLASS_DOC = "Determines which partition to send a record to when records are produced. Available options are:" + "

      " + - "
    • If not set, the default partitioning logic is used. " + - "This strategy send records to a partition until at least " + BATCH_SIZE_CONFIG + " bytes is produced to the partition. It works with the strategy:" + + "
    • If not set, the default partitioning logic is used. " + + "This strategy send records to a partition until at least " + BATCH_SIZE_CONFIG + " bytes is produced to the partition. It works with the strategy:" + "
        " + "
      1. If no partition is specified but a key is present, choose a partition based on a hash of the key.
      2. " + "
      3. If no partition or key is present, choose the sticky partition that changes when at least " + BATCH_SIZE_CONFIG + " bytes are produced to the partition.
      4. " + From 80d2a8a42d0976ff859fccc76fd10b7cb4411491 Mon Sep 17 00:00:00 2001 From: David Jacot Date: Fri, 24 Jan 2025 13:59:30 +0100 Subject: [PATCH 42/44] KAFKA-18616; Refactor DumpLogSegments's MessageParsers (#18688) All the work that we have done to automate and to simplify the coordinator records allows us to simplify the related MessageParsers in DumpLogSegments. They can all share the same based implementation. This is nice because it ensures that we handle all those records similarly. Reviewers: Chia-Ping Tsai --- .../scala/kafka/tools/DumpLogSegments.scala | 187 ++++++------------ .../kafka/tools/DumpLogSegmentsTest.scala | 6 +- 2 files changed, 67 insertions(+), 126 deletions(-) diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index e63be08bfb49d..dfa357943c5b2 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -36,8 +36,9 @@ import org.apache.kafka.common.metadata.{MetadataJsonConverters, MetadataRecordT import org.apache.kafka.common.protocol.{ApiMessage, ByteBufferAccessor} import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils -import org.apache.kafka.coordinator.group.generated.{CoordinatorRecordJsonConverters => GroupCoordinatorRecordJsonConverters, CoordinatorRecordType => GroupCoordinatorRecordType, GroupMetadataValue, GroupMetadataValueJsonConverter} +import org.apache.kafka.coordinator.group.generated.{GroupMetadataValue, GroupMetadataValueJsonConverter, CoordinatorRecordJsonConverters => GroupCoordinatorRecordJsonConverters, CoordinatorRecordType => GroupCoordinatorRecordType} import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader.UnknownRecordTypeException +import org.apache.kafka.coordinator.common.runtime.CoordinatorRecordSerde import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde import org.apache.kafka.coordinator.share.ShareCoordinatorRecordSerde import org.apache.kafka.coordinator.share.generated.{CoordinatorRecordJsonConverters => ShareCoordinatorRecordJsonConverters} @@ -419,17 +420,65 @@ object DumpLogSegments { } } - // Package private for testing. - class OffsetsMessageParser extends MessageParser[String, String] { - private val serde = new GroupCoordinatorRecordSerde() + abstract class CoordinatorRecordMessageParser(serde: CoordinatorRecordSerde) extends MessageParser[String, String] { + override def parse(record: Record): (Option[String], Option[String]) = { + if (!record.hasKey) + throw new RuntimeException(s"Failed to decode message at offset ${record.offset} using the " + + "specified decoder (message had a missing key)") + + try { + val r = serde.deserialize(record.key, record.value) + ( + Some(prepareKey(r.key)), + Option(r.value).map(v => prepareValue(v.message, v.version)).orElse(Some("")) + ) + } catch { + case e: UnknownRecordTypeException => + ( + Some(s"Unknown record type ${e.unknownType} at offset ${record.offset}, skipping."), + None + ) + + case e: Throwable => + ( + Some(s"Error at offset ${record.offset}, skipping. ${e.getMessage}"), + None + ) + } + } private def prepareKey(message: ApiMessage): String = { val json = new ObjectNode(JsonNodeFactory.instance) json.set("type", new TextNode(message.apiKey.toString)) - json.set("data", GroupCoordinatorRecordJsonConverters.writeRecordKeyAsJson(message)) + json.set("data", keyAsJson(message)) + json.toString + } + + private def prepareValue(message: ApiMessage, version: Short): String = { + val json = new ObjectNode(JsonNodeFactory.instance) + json.set("version", new TextNode(version.toString)) + json.set("data", valueAsJson(message, version)) json.toString } + protected def keyAsJson(message: ApiMessage): JsonNode + protected def valueAsJson(message: ApiMessage, version: Short): JsonNode + } + + // Package private for testing. + class OffsetsMessageParser extends CoordinatorRecordMessageParser(new GroupCoordinatorRecordSerde()) { + protected def keyAsJson(message: ApiMessage): JsonNode = { + GroupCoordinatorRecordJsonConverters.writeRecordKeyAsJson(message) + } + + protected def valueAsJson(message: ApiMessage, version: Short): JsonNode = { + if (message.apiKey == GroupCoordinatorRecordType.GROUP_METADATA.id) { + prepareGroupMetadataValue(message.asInstanceOf[GroupMetadataValue], version) + } else { + GroupCoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version) + } + } + private def prepareGroupMetadataValue(message: GroupMetadataValue, version: Short): JsonNode = { val json = GroupMetadataValueJsonConverter.write(message, version) @@ -482,90 +531,16 @@ object DumpLogSegments { json } - - private def prepareValue(message: ApiMessage, version: Short): String = { - val messageAsJson = if (message.apiKey == GroupCoordinatorRecordType.GROUP_METADATA.id) { - prepareGroupMetadataValue(message.asInstanceOf[GroupMetadataValue], version) - } else { - GroupCoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version) - } - - val json = new ObjectNode(JsonNodeFactory.instance) - json.set("version", new TextNode(version.toString)) - json.set("data", messageAsJson) - json.toString - } - - override def parse(record: Record): (Option[String], Option[String]) = { - if (!record.hasKey) - throw new RuntimeException(s"Failed to decode message at offset ${record.offset} using offset " + - "topic decoder (message had a missing key)") - - try { - val r = serde.deserialize(record.key, record.value) - ( - Some(prepareKey(r.key)), - Option(r.value).map(v => prepareValue(v.message, v.version)).orElse(Some("")) - ) - } catch { - case e: UnknownRecordTypeException => - ( - Some(s"Unknown record type ${e.unknownType} at offset ${record.offset}, skipping."), - None - ) - - case e: Throwable => - ( - Some(s"Error at offset ${record.offset}, skipping. ${e.getMessage}"), - None - ) - } - } } // Package private for testing. - class TransactionLogMessageParser extends MessageParser[String, String] { - private val serde = new TransactionCoordinatorRecordSerde() - - private def prepareKey(message: ApiMessage): String = { - val json = new ObjectNode(JsonNodeFactory.instance) - json.set("type", new TextNode(message.apiKey.toString)) - json.set("data", TransactionCoordinatorRecordJsonConverters.writeRecordKeyAsJson(message)) - json.toString + class TransactionLogMessageParser extends CoordinatorRecordMessageParser(new TransactionCoordinatorRecordSerde()) { + override protected def keyAsJson(message: ApiMessage): JsonNode = { + TransactionCoordinatorRecordJsonConverters.writeRecordKeyAsJson(message) } - private def prepareValue(message: ApiMessage, version: Short): String = { - val json = new ObjectNode(JsonNodeFactory.instance) - json.set("version", new TextNode(version.toString)) - json.set("data", TransactionCoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version)) - json.toString - } - - override def parse(record: Record): (Option[String], Option[String]) = { - if (!record.hasKey) - throw new RuntimeException(s"Failed to decode message at offset ${record.offset} using offset " + - "transaction-log decoder (message had a missing key)") - - try { - val r = serde.deserialize(record.key, record.value) - ( - Some(prepareKey(r.key)), - Option(r.value).map(v => prepareValue(v.message, v.version)).orElse(Some("")) - ) - } catch { - case e: UnknownRecordTypeException => - ( - Some(s"Unknown record type ${e.unknownType} at offset ${record.offset}, skipping."), - None - ) - - case e: Throwable => - e.printStackTrace() - ( - Some(s"Error at offset ${record.offset}, skipping. ${e.getMessage}"), - None - ) - } + override protected def valueAsJson(message: ApiMessage, version: Short): JsonNode = { + TransactionCoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version) } } @@ -610,47 +585,13 @@ object DumpLogSegments { } // for test visibility - class ShareGroupStateMessageParser extends MessageParser[String, String] { - private val serde = new ShareCoordinatorRecordSerde() - - private def prepareKey(message: ApiMessage): String = { - val json = new ObjectNode(JsonNodeFactory.instance) - json.set("type", new TextNode(message.apiKey.toString)) - json.set("data", ShareCoordinatorRecordJsonConverters.writeRecordKeyAsJson(message)) - json.toString - } - - private def prepareValue(message: ApiMessage, version: Short): String = { - val json = new ObjectNode(JsonNodeFactory.instance) - json.set("version", new TextNode(version.toString)) - json.set("data", ShareCoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version)) - json.toString + class ShareGroupStateMessageParser extends CoordinatorRecordMessageParser(new ShareCoordinatorRecordSerde()) { + override protected def keyAsJson(message: ApiMessage): JsonNode = { + ShareCoordinatorRecordJsonConverters.writeRecordKeyAsJson(message) } - override def parse(record: Record): (Option[String], Option[String]) = { - if (!record.hasKey) - throw new RuntimeException(s"Failed to decode message at offset ${record.offset} using share group state " + - "topic decoder (message had a missing key)") - - try { - val r = serde.deserialize(record.key, record.value) - ( - Some(prepareKey(r.key)), - Option(r.value).map(v => prepareValue(v.message, v.version)).orElse(Some("")) - ) - } catch { - case e: UnknownRecordTypeException => - ( - Some(s"Unknown record type ${e.unknownType} at offset ${record.offset}, skipping."), - None - ) - - case e: Throwable => - ( - Some(s"Error at offset ${record.offset}, skipping. ${e.getMessage}"), - None - ) - } + override protected def valueAsJson(message: ApiMessage, version: Short): JsonNode = { + ShareCoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version) } } diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index 2e6c1abe11416..53d153ea27307 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -655,7 +655,7 @@ class DumpLogSegmentsTest { // The key is mandatory. assertEquals( - "Failed to decode message at offset 0 using offset topic decoder (message had a missing key)", + "Failed to decode message at offset 0 using the specified decoder (message had a missing key)", assertThrows( classOf[RuntimeException], () => parser.parse(TestUtils.singletonRecords(key = null, value = null).records.iterator.next) @@ -813,7 +813,7 @@ class DumpLogSegmentsTest { // The key is mandatory. assertEquals( - "Failed to decode message at offset 0 using offset transaction-log decoder (message had a missing key)", + "Failed to decode message at offset 0 using the specified decoder (message had a missing key)", assertThrows( classOf[RuntimeException], () => parser.parse(TestUtils.singletonRecords(key = null, value = null).records.iterator.next) @@ -1044,7 +1044,7 @@ class DumpLogSegmentsTest { // The key is mandatory. assertEquals( - "Failed to decode message at offset 0 using share group state topic decoder (message had a missing key)", + "Failed to decode message at offset 0 using the specified decoder (message had a missing key)", assertThrows( classOf[RuntimeException], () => parser.parse(TestUtils.singletonRecords(key = null, value = null).records.iterator.next) From 0c9df75295fd448e89e1d2751a129b89597e2731 Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Fri, 24 Jan 2025 21:53:32 +0800 Subject: [PATCH 43/44] KAFKA-18474: Remove zkBroker listener (#18477) Reviewers: Ismael Juma , Chia-Ping Tsai , PoAn Yang --- .../apache/kafka/common/protocol/ApiKeys.java | 8 +--- .../message/AddOffsetsToTxnRequest.json | 2 +- .../message/AddPartitionsToTxnRequest.json | 2 +- .../message/AllocateProducerIdsRequest.json | 2 +- .../message/AlterClientQuotasRequest.json | 2 +- .../common/message/AlterConfigsRequest.json | 2 +- .../AlterPartitionReassignmentsRequest.json | 2 +- .../common/message/AlterPartitionRequest.json | 2 +- .../message/AlterReplicaLogDirsRequest.json | 2 +- .../AlterUserScramCredentialsRequest.json | 2 +- .../common/message/ApiVersionsRequest.json | 2 +- .../common/message/CreateAclsRequest.json | 2 +- .../message/CreateDelegationTokenRequest.json | 2 +- .../message/CreatePartitionsRequest.json | 2 +- .../common/message/CreateTopicsRequest.json | 2 +- .../common/message/DeleteAclsRequest.json | 2 +- .../common/message/DeleteGroupsRequest.json | 2 +- .../common/message/DeleteRecordsRequest.json | 2 +- .../common/message/DeleteTopicsRequest.json | 2 +- .../common/message/DescribeAclsRequest.json | 2 +- .../message/DescribeClientQuotasRequest.json | 2 +- .../message/DescribeClusterRequest.json | 2 +- .../message/DescribeConfigsRequest.json | 2 +- .../DescribeDelegationTokenRequest.json | 2 +- .../common/message/DescribeGroupsRequest.json | 2 +- .../message/DescribeLogDirsRequest.json | 2 +- .../message/DescribeProducersRequest.json | 2 +- .../message/DescribeTransactionsRequest.json | 2 +- .../DescribeUserScramCredentialsRequest.json | 2 +- .../common/message/ElectLeadersRequest.json | 2 +- .../common/message/EndTxnRequest.json | 2 +- .../common/message/EnvelopeRequest.json | 2 +- .../message/ExpireDelegationTokenRequest.json | 2 +- .../common/message/FetchRequest.json | 2 +- .../message/FindCoordinatorRequest.json | 2 +- .../common/message/HeartbeatRequest.json | 2 +- .../IncrementalAlterConfigsRequest.json | 2 +- .../common/message/InitProducerIdRequest.json | 2 +- .../common/message/JoinGroupRequest.json | 2 +- .../common/message/LeaveGroupRequest.json | 2 +- .../common/message/ListGroupsRequest.json | 2 +- .../common/message/ListOffsetsRequest.json | 2 +- .../ListPartitionReassignmentsRequest.json | 2 +- .../message/ListTransactionsRequest.json | 2 +- .../common/message/MetadataRequest.json | 2 +- .../common/message/OffsetCommitRequest.json | 2 +- .../common/message/OffsetDeleteRequest.json | 2 +- .../common/message/OffsetFetchRequest.json | 2 +- .../message/OffsetForLeaderEpochRequest.json | 2 +- .../common/message/ProduceRequest.json | 2 +- .../message/RenewDelegationTokenRequest.json | 2 +- .../message/SaslAuthenticateRequest.json | 2 +- .../common/message/SaslHandshakeRequest.json | 2 +- .../common/message/SyncGroupRequest.json | 2 +- .../message/TxnOffsetCommitRequest.json | 2 +- .../common/message/UpdateFeaturesRequest.json | 2 +- .../message/WriteTxnMarkersRequest.json | 2 +- .../kafka/clients/NetworkClientTest.java | 4 +- .../clients/admin/KafkaAdminClientTest.java | 2 +- .../internals/FetchRequestManagerTest.java | 2 +- .../consumer/internals/FetcherTest.java | 2 +- .../producer/internals/SenderTest.java | 2 +- .../kafka/common/network/NioEchoServer.java | 2 +- .../network/SaslChannelBuilderTest.java | 2 +- .../common/network/SslTransportLayerTest.java | 2 +- .../requests/ApiVersionsResponseTest.java | 42 +++++++++---------- .../common/requests/RequestResponseTest.java | 4 +- .../authenticator/SaslAuthenticatorTest.java | 6 +-- .../SaslServerAuthenticatorTest.java | 2 +- .../server/GssapiAuthenticationTest.scala | 2 +- .../kafka/server/ApiVersionManagerTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 6 +-- .../kafka/message/RequestListenerType.java | 2 - 73 files changed, 99 insertions(+), 105 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index e33dd2589ae99..1f8a98554c243 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -332,11 +332,7 @@ public void visit(Type field) { return hasBuffer.get(); } - public static EnumSet zkBrokerApis() { - return apisForListener(ApiMessageType.ListenerType.ZK_BROKER); - } - - public static EnumSet kraftBrokerApis() { + public static EnumSet brokerApis() { return apisForListener(ApiMessageType.ListenerType.BROKER); } @@ -346,7 +342,7 @@ public static EnumSet controllerApis() { public static EnumSet clientApis() { List apis = Arrays.stream(ApiKeys.values()) - .filter(apiKey -> apiKey.inScope(ApiMessageType.ListenerType.ZK_BROKER) || apiKey.inScope(ApiMessageType.ListenerType.BROKER)) + .filter(apiKey -> apiKey.inScope(ApiMessageType.ListenerType.BROKER)) .collect(Collectors.toList()); return EnumSet.copyOf(apis); } diff --git a/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json b/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json index 157ae20c0a41d..9bebc8366cf1d 100644 --- a/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json +++ b/clients/src/main/resources/common/message/AddOffsetsToTxnRequest.json @@ -16,7 +16,7 @@ { "apiKey": 25, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "AddOffsetsToTxnRequest", // Version 1 is the same as version 0. // diff --git a/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json b/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json index 2ed84be2180ba..68a45cdd0aca2 100644 --- a/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json +++ b/clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json @@ -16,7 +16,7 @@ { "apiKey": 24, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "AddPartitionsToTxnRequest", // Version 1 is the same as version 0. // diff --git a/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json b/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json index e8271c60321a3..9447b080f84f4 100644 --- a/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json +++ b/clients/src/main/resources/common/message/AllocateProducerIdsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 67, "type": "request", - "listeners": ["zkBroker", "controller"], + "listeners": ["controller"], "name": "AllocateProducerIdsRequest", "validVersions": "0", "flexibleVersions": "0+", diff --git a/clients/src/main/resources/common/message/AlterClientQuotasRequest.json b/clients/src/main/resources/common/message/AlterClientQuotasRequest.json index 6bfdc925c2919..93524a80e2137 100644 --- a/clients/src/main/resources/common/message/AlterClientQuotasRequest.json +++ b/clients/src/main/resources/common/message/AlterClientQuotasRequest.json @@ -16,7 +16,7 @@ { "apiKey": 49, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "AlterClientQuotasRequest", "validVersions": "0-1", // Version 1 enables flexible versions. diff --git a/clients/src/main/resources/common/message/AlterConfigsRequest.json b/clients/src/main/resources/common/message/AlterConfigsRequest.json index 31057e3410aaf..b87091f2e5c65 100644 --- a/clients/src/main/resources/common/message/AlterConfigsRequest.json +++ b/clients/src/main/resources/common/message/AlterConfigsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 33, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "AlterConfigsRequest", // Version 1 is the same as version 0. // Version 2 enables flexible versions. diff --git a/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json b/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json index 47043ff02d0a4..f3047feb0a3ad 100644 --- a/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json +++ b/clients/src/main/resources/common/message/AlterPartitionReassignmentsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 45, "type": "request", - "listeners": ["broker", "controller", "zkBroker"], + "listeners": ["broker", "controller"], "name": "AlterPartitionReassignmentsRequest", "validVersions": "0", "flexibleVersions": "0+", diff --git a/clients/src/main/resources/common/message/AlterPartitionRequest.json b/clients/src/main/resources/common/message/AlterPartitionRequest.json index 954b98578af70..d22f3eb13ad9b 100644 --- a/clients/src/main/resources/common/message/AlterPartitionRequest.json +++ b/clients/src/main/resources/common/message/AlterPartitionRequest.json @@ -16,7 +16,7 @@ { "apiKey": 56, "type": "request", - "listeners": ["zkBroker", "controller"], + "listeners": ["controller"], "name": "AlterPartitionRequest", // Version 1 adds LeaderRecoveryState field (KIP-704). // diff --git a/clients/src/main/resources/common/message/AlterReplicaLogDirsRequest.json b/clients/src/main/resources/common/message/AlterReplicaLogDirsRequest.json index b309243fb62d9..42ef669332526 100644 --- a/clients/src/main/resources/common/message/AlterReplicaLogDirsRequest.json +++ b/clients/src/main/resources/common/message/AlterReplicaLogDirsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 34, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "AlterReplicaLogDirsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // diff --git a/clients/src/main/resources/common/message/AlterUserScramCredentialsRequest.json b/clients/src/main/resources/common/message/AlterUserScramCredentialsRequest.json index 8937394ef6e51..ea687072f1686 100644 --- a/clients/src/main/resources/common/message/AlterUserScramCredentialsRequest.json +++ b/clients/src/main/resources/common/message/AlterUserScramCredentialsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 51, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "AlterUserScramCredentialsRequest", "validVersions": "0", "flexibleVersions": "0+", diff --git a/clients/src/main/resources/common/message/ApiVersionsRequest.json b/clients/src/main/resources/common/message/ApiVersionsRequest.json index 050dbcfd3f2c9..56170c9667350 100644 --- a/clients/src/main/resources/common/message/ApiVersionsRequest.json +++ b/clients/src/main/resources/common/message/ApiVersionsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 18, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "ApiVersionsRequest", // Versions 0 through 2 of ApiVersionsRequest are the same. // diff --git a/clients/src/main/resources/common/message/CreateAclsRequest.json b/clients/src/main/resources/common/message/CreateAclsRequest.json index 0f4582030747f..d3a028b0536d5 100644 --- a/clients/src/main/resources/common/message/CreateAclsRequest.json +++ b/clients/src/main/resources/common/message/CreateAclsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 30, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "CreateAclsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 adds resource pattern type. diff --git a/clients/src/main/resources/common/message/CreateDelegationTokenRequest.json b/clients/src/main/resources/common/message/CreateDelegationTokenRequest.json index 276cbe57901f5..f4c586cbcadd9 100644 --- a/clients/src/main/resources/common/message/CreateDelegationTokenRequest.json +++ b/clients/src/main/resources/common/message/CreateDelegationTokenRequest.json @@ -16,7 +16,7 @@ { "apiKey": 38, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "CreateDelegationTokenRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // diff --git a/clients/src/main/resources/common/message/CreatePartitionsRequest.json b/clients/src/main/resources/common/message/CreatePartitionsRequest.json index 6e249498659fa..95552a080b980 100644 --- a/clients/src/main/resources/common/message/CreatePartitionsRequest.json +++ b/clients/src/main/resources/common/message/CreatePartitionsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 37, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "CreatePartitionsRequest", // Version 1 is the same as version 0. // diff --git a/clients/src/main/resources/common/message/CreateTopicsRequest.json b/clients/src/main/resources/common/message/CreateTopicsRequest.json index 9aed4d236dbf0..3ee03933572c6 100644 --- a/clients/src/main/resources/common/message/CreateTopicsRequest.json +++ b/clients/src/main/resources/common/message/CreateTopicsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 19, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "CreateTopicsRequest", // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. // diff --git a/clients/src/main/resources/common/message/DeleteAclsRequest.json b/clients/src/main/resources/common/message/DeleteAclsRequest.json index b430364d8611b..db605305ae28b 100644 --- a/clients/src/main/resources/common/message/DeleteAclsRequest.json +++ b/clients/src/main/resources/common/message/DeleteAclsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 31, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "DeleteAclsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 adds the pattern type. diff --git a/clients/src/main/resources/common/message/DeleteGroupsRequest.json b/clients/src/main/resources/common/message/DeleteGroupsRequest.json index 1ac6a053e63b3..7d7c437178997 100644 --- a/clients/src/main/resources/common/message/DeleteGroupsRequest.json +++ b/clients/src/main/resources/common/message/DeleteGroupsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 42, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "DeleteGroupsRequest", // Version 1 is the same as version 0. // diff --git a/clients/src/main/resources/common/message/DeleteRecordsRequest.json b/clients/src/main/resources/common/message/DeleteRecordsRequest.json index 06a12d85c8bb4..fc697944a02bb 100644 --- a/clients/src/main/resources/common/message/DeleteRecordsRequest.json +++ b/clients/src/main/resources/common/message/DeleteRecordsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 21, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "DeleteRecordsRequest", // Version 1 is the same as version 0. diff --git a/clients/src/main/resources/common/message/DeleteTopicsRequest.json b/clients/src/main/resources/common/message/DeleteTopicsRequest.json index 465d9e0b31f4e..35c9eb28f9c47 100644 --- a/clients/src/main/resources/common/message/DeleteTopicsRequest.json +++ b/clients/src/main/resources/common/message/DeleteTopicsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 20, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "DeleteTopicsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Versions 0, 1, 2, and 3 are the same. diff --git a/clients/src/main/resources/common/message/DescribeAclsRequest.json b/clients/src/main/resources/common/message/DescribeAclsRequest.json index a9bdfba40e7fb..23883c154fe63 100644 --- a/clients/src/main/resources/common/message/DescribeAclsRequest.json +++ b/clients/src/main/resources/common/message/DescribeAclsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 29, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "DescribeAclsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 adds resource pattern type. diff --git a/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json b/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json index d14cfc95733d3..6644e53343bc8 100644 --- a/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json +++ b/clients/src/main/resources/common/message/DescribeClientQuotasRequest.json @@ -16,7 +16,7 @@ { "apiKey": 48, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "DescribeClientQuotasRequest", // Version 1 enables flexible versions. "validVersions": "0-1", diff --git a/clients/src/main/resources/common/message/DescribeClusterRequest.json b/clients/src/main/resources/common/message/DescribeClusterRequest.json index 71e00df09b2f9..9c17c6b1ba51b 100644 --- a/clients/src/main/resources/common/message/DescribeClusterRequest.json +++ b/clients/src/main/resources/common/message/DescribeClusterRequest.json @@ -16,7 +16,7 @@ { "apiKey": 60, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "DescribeClusterRequest", // // Version 1 adds EndpointType for KIP-919 support. diff --git a/clients/src/main/resources/common/message/DescribeConfigsRequest.json b/clients/src/main/resources/common/message/DescribeConfigsRequest.json index a382d9fecaf44..d1a85a67fea5a 100644 --- a/clients/src/main/resources/common/message/DescribeConfigsRequest.json +++ b/clients/src/main/resources/common/message/DescribeConfigsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 32, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "DescribeConfigsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 adds IncludeSynonyms and removes IsDefault. diff --git a/clients/src/main/resources/common/message/DescribeDelegationTokenRequest.json b/clients/src/main/resources/common/message/DescribeDelegationTokenRequest.json index d62eb28a29fb4..bc29789b6891f 100644 --- a/clients/src/main/resources/common/message/DescribeDelegationTokenRequest.json +++ b/clients/src/main/resources/common/message/DescribeDelegationTokenRequest.json @@ -16,7 +16,7 @@ { "apiKey": 41, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "DescribeDelegationTokenRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 is the same as version 0. diff --git a/clients/src/main/resources/common/message/DescribeGroupsRequest.json b/clients/src/main/resources/common/message/DescribeGroupsRequest.json index 8dabf71bd5255..cec56852cad9d 100644 --- a/clients/src/main/resources/common/message/DescribeGroupsRequest.json +++ b/clients/src/main/resources/common/message/DescribeGroupsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 15, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "DescribeGroupsRequest", // Versions 1 and 2 are the same as version 0. // diff --git a/clients/src/main/resources/common/message/DescribeLogDirsRequest.json b/clients/src/main/resources/common/message/DescribeLogDirsRequest.json index 115947ff394b2..4f3bfa2c58ce0 100644 --- a/clients/src/main/resources/common/message/DescribeLogDirsRequest.json +++ b/clients/src/main/resources/common/message/DescribeLogDirsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 35, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "DescribeLogDirsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 is the same as version 0. diff --git a/clients/src/main/resources/common/message/DescribeProducersRequest.json b/clients/src/main/resources/common/message/DescribeProducersRequest.json index 7a54c65622da6..b7889ef1f1e1c 100644 --- a/clients/src/main/resources/common/message/DescribeProducersRequest.json +++ b/clients/src/main/resources/common/message/DescribeProducersRequest.json @@ -16,7 +16,7 @@ { "apiKey": 61, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "DescribeProducersRequest", "validVersions": "0", "flexibleVersions": "0+", diff --git a/clients/src/main/resources/common/message/DescribeTransactionsRequest.json b/clients/src/main/resources/common/message/DescribeTransactionsRequest.json index 442f11f8b0b47..f7349d60cd18a 100644 --- a/clients/src/main/resources/common/message/DescribeTransactionsRequest.json +++ b/clients/src/main/resources/common/message/DescribeTransactionsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 65, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "DescribeTransactionsRequest", "validVersions": "0", "flexibleVersions": "0+", diff --git a/clients/src/main/resources/common/message/DescribeUserScramCredentialsRequest.json b/clients/src/main/resources/common/message/DescribeUserScramCredentialsRequest.json index 2f7a1112c4800..cde4b7cc84468 100644 --- a/clients/src/main/resources/common/message/DescribeUserScramCredentialsRequest.json +++ b/clients/src/main/resources/common/message/DescribeUserScramCredentialsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 50, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "DescribeUserScramCredentialsRequest", "validVersions": "0", "flexibleVersions": "0+", diff --git a/clients/src/main/resources/common/message/ElectLeadersRequest.json b/clients/src/main/resources/common/message/ElectLeadersRequest.json index dd9fa21641585..bce04585a70e2 100644 --- a/clients/src/main/resources/common/message/ElectLeadersRequest.json +++ b/clients/src/main/resources/common/message/ElectLeadersRequest.json @@ -16,7 +16,7 @@ { "apiKey": 43, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "ElectLeadersRequest", // Version 1 implements multiple leader election types, as described by KIP-460. // diff --git a/clients/src/main/resources/common/message/EndTxnRequest.json b/clients/src/main/resources/common/message/EndTxnRequest.json index 80ac5c5d2551c..f11c7a3268f13 100644 --- a/clients/src/main/resources/common/message/EndTxnRequest.json +++ b/clients/src/main/resources/common/message/EndTxnRequest.json @@ -16,7 +16,7 @@ { "apiKey": 26, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "EndTxnRequest", // Version 1 is the same as version 0. // diff --git a/clients/src/main/resources/common/message/EnvelopeRequest.json b/clients/src/main/resources/common/message/EnvelopeRequest.json index a30a50ba68460..1f6ff62de8d7f 100644 --- a/clients/src/main/resources/common/message/EnvelopeRequest.json +++ b/clients/src/main/resources/common/message/EnvelopeRequest.json @@ -16,7 +16,7 @@ { "apiKey": 58, "type": "request", - "listeners": ["controller", "zkBroker"], + "listeners": ["controller"], "name": "EnvelopeRequest", // Request struct for forwarding. "validVersions": "0", diff --git a/clients/src/main/resources/common/message/ExpireDelegationTokenRequest.json b/clients/src/main/resources/common/message/ExpireDelegationTokenRequest.json index 2694243f1f3c9..92a9e4e947f39 100644 --- a/clients/src/main/resources/common/message/ExpireDelegationTokenRequest.json +++ b/clients/src/main/resources/common/message/ExpireDelegationTokenRequest.json @@ -16,7 +16,7 @@ { "apiKey": 40, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "ExpireDelegationTokenRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 is the same as version 0. diff --git a/clients/src/main/resources/common/message/FetchRequest.json b/clients/src/main/resources/common/message/FetchRequest.json index c49dd1a9b0a4c..b7ad185f60b39 100644 --- a/clients/src/main/resources/common/message/FetchRequest.json +++ b/clients/src/main/resources/common/message/FetchRequest.json @@ -16,7 +16,7 @@ { "apiKey": 1, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "FetchRequest", // Versions 0-3 were removed in Apache Kafka 4.0, Version 4 is the new baseline. // diff --git a/clients/src/main/resources/common/message/FindCoordinatorRequest.json b/clients/src/main/resources/common/message/FindCoordinatorRequest.json index 7a926501f7bb3..2807f40c857af 100644 --- a/clients/src/main/resources/common/message/FindCoordinatorRequest.json +++ b/clients/src/main/resources/common/message/FindCoordinatorRequest.json @@ -16,7 +16,7 @@ { "apiKey": 10, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "FindCoordinatorRequest", // Version 1 adds KeyType. // diff --git a/clients/src/main/resources/common/message/HeartbeatRequest.json b/clients/src/main/resources/common/message/HeartbeatRequest.json index dcf776d8ec4e3..57ef18e922471 100644 --- a/clients/src/main/resources/common/message/HeartbeatRequest.json +++ b/clients/src/main/resources/common/message/HeartbeatRequest.json @@ -16,7 +16,7 @@ { "apiKey": 12, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "HeartbeatRequest", // Version 1 and version 2 are the same as version 0. // diff --git a/clients/src/main/resources/common/message/IncrementalAlterConfigsRequest.json b/clients/src/main/resources/common/message/IncrementalAlterConfigsRequest.json index d4955c91b85a4..d908c28012faa 100644 --- a/clients/src/main/resources/common/message/IncrementalAlterConfigsRequest.json +++ b/clients/src/main/resources/common/message/IncrementalAlterConfigsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 44, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "IncrementalAlterConfigsRequest", // Version 1 is the first flexible version. "validVersions": "0-1", diff --git a/clients/src/main/resources/common/message/InitProducerIdRequest.json b/clients/src/main/resources/common/message/InitProducerIdRequest.json index 9b9d247fc6237..5a056db520fbc 100644 --- a/clients/src/main/resources/common/message/InitProducerIdRequest.json +++ b/clients/src/main/resources/common/message/InitProducerIdRequest.json @@ -16,7 +16,7 @@ { "apiKey": 22, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "InitProducerIdRequest", // Version 1 is the same as version 0. // diff --git a/clients/src/main/resources/common/message/JoinGroupRequest.json b/clients/src/main/resources/common/message/JoinGroupRequest.json index 2c4c9fdfd62db..41d7c1acbaefe 100644 --- a/clients/src/main/resources/common/message/JoinGroupRequest.json +++ b/clients/src/main/resources/common/message/JoinGroupRequest.json @@ -16,7 +16,7 @@ { "apiKey": 11, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "JoinGroupRequest", // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. // diff --git a/clients/src/main/resources/common/message/LeaveGroupRequest.json b/clients/src/main/resources/common/message/LeaveGroupRequest.json index fb16f72eb8274..929f4fb468c95 100644 --- a/clients/src/main/resources/common/message/LeaveGroupRequest.json +++ b/clients/src/main/resources/common/message/LeaveGroupRequest.json @@ -16,7 +16,7 @@ { "apiKey": 13, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "LeaveGroupRequest", // Version 1 and 2 are the same as version 0. // diff --git a/clients/src/main/resources/common/message/ListGroupsRequest.json b/clients/src/main/resources/common/message/ListGroupsRequest.json index 32defaa203382..cbc791e0a5a26 100644 --- a/clients/src/main/resources/common/message/ListGroupsRequest.json +++ b/clients/src/main/resources/common/message/ListGroupsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 16, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "ListGroupsRequest", // Version 1 and 2 are the same as version 0. // diff --git a/clients/src/main/resources/common/message/ListOffsetsRequest.json b/clients/src/main/resources/common/message/ListOffsetsRequest.json index 5a864d8ddc1f8..6f8ff7d6cf935 100644 --- a/clients/src/main/resources/common/message/ListOffsetsRequest.json +++ b/clients/src/main/resources/common/message/ListOffsetsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 2, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "ListOffsetsRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // diff --git a/clients/src/main/resources/common/message/ListPartitionReassignmentsRequest.json b/clients/src/main/resources/common/message/ListPartitionReassignmentsRequest.json index 952a3db0d23ef..428a256ac3070 100644 --- a/clients/src/main/resources/common/message/ListPartitionReassignmentsRequest.json +++ b/clients/src/main/resources/common/message/ListPartitionReassignmentsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 46, "type": "request", - "listeners": ["broker", "controller", "zkBroker"], + "listeners": ["broker", "controller"], "name": "ListPartitionReassignmentsRequest", "validVersions": "0", "flexibleVersions": "0+", diff --git a/clients/src/main/resources/common/message/ListTransactionsRequest.json b/clients/src/main/resources/common/message/ListTransactionsRequest.json index 4879c4d5f957e..5d7c688da2213 100644 --- a/clients/src/main/resources/common/message/ListTransactionsRequest.json +++ b/clients/src/main/resources/common/message/ListTransactionsRequest.json @@ -16,7 +16,7 @@ { "apiKey": 66, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "ListTransactionsRequest", // Version 1: adds DurationFilter to list transactions older than specified duration "validVersions": "0-1", diff --git a/clients/src/main/resources/common/message/MetadataRequest.json b/clients/src/main/resources/common/message/MetadataRequest.json index eaee4a3453d41..c29093239eda1 100644 --- a/clients/src/main/resources/common/message/MetadataRequest.json +++ b/clients/src/main/resources/common/message/MetadataRequest.json @@ -16,7 +16,7 @@ { "apiKey": 3, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "MetadataRequest", "validVersions": "4-13", "flexibleVersions": "9+", diff --git a/clients/src/main/resources/common/message/OffsetCommitRequest.json b/clients/src/main/resources/common/message/OffsetCommitRequest.json index 8f9e1d74d96d6..348ed2b90c5c8 100644 --- a/clients/src/main/resources/common/message/OffsetCommitRequest.json +++ b/clients/src/main/resources/common/message/OffsetCommitRequest.json @@ -16,7 +16,7 @@ { "apiKey": 8, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "OffsetCommitRequest", // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. // diff --git a/clients/src/main/resources/common/message/OffsetDeleteRequest.json b/clients/src/main/resources/common/message/OffsetDeleteRequest.json index 4583030060add..1974b67f69fc3 100644 --- a/clients/src/main/resources/common/message/OffsetDeleteRequest.json +++ b/clients/src/main/resources/common/message/OffsetDeleteRequest.json @@ -16,7 +16,7 @@ { "apiKey": 47, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "OffsetDeleteRequest", "validVersions": "0", "flexibleVersions": "none", diff --git a/clients/src/main/resources/common/message/OffsetFetchRequest.json b/clients/src/main/resources/common/message/OffsetFetchRequest.json index d9d97da384b62..88f5b568d724c 100644 --- a/clients/src/main/resources/common/message/OffsetFetchRequest.json +++ b/clients/src/main/resources/common/message/OffsetFetchRequest.json @@ -16,7 +16,7 @@ { "apiKey": 9, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "OffsetFetchRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // diff --git a/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json b/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json index b2126a4001449..dd559bc8777f8 100644 --- a/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json +++ b/clients/src/main/resources/common/message/OffsetForLeaderEpochRequest.json @@ -16,7 +16,7 @@ { "apiKey": 23, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "OffsetForLeaderEpochRequest", // Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline. // diff --git a/clients/src/main/resources/common/message/ProduceRequest.json b/clients/src/main/resources/common/message/ProduceRequest.json index 90e46a3041b22..db7d961f1373f 100644 --- a/clients/src/main/resources/common/message/ProduceRequest.json +++ b/clients/src/main/resources/common/message/ProduceRequest.json @@ -16,7 +16,7 @@ { "apiKey": 0, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "ProduceRequest", // Versions 0-2 were removed in Apache Kafka 4.0, Version 3 is the new baseline. // diff --git a/clients/src/main/resources/common/message/RenewDelegationTokenRequest.json b/clients/src/main/resources/common/message/RenewDelegationTokenRequest.json index 302e5d3e2ba1c..5ce0a4775db88 100644 --- a/clients/src/main/resources/common/message/RenewDelegationTokenRequest.json +++ b/clients/src/main/resources/common/message/RenewDelegationTokenRequest.json @@ -16,7 +16,7 @@ { "apiKey": 39, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "RenewDelegationTokenRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // Version 1 is the same as version 0. diff --git a/clients/src/main/resources/common/message/SaslAuthenticateRequest.json b/clients/src/main/resources/common/message/SaslAuthenticateRequest.json index 3f5558b812042..cdb4247b8a95a 100644 --- a/clients/src/main/resources/common/message/SaslAuthenticateRequest.json +++ b/clients/src/main/resources/common/message/SaslAuthenticateRequest.json @@ -16,7 +16,7 @@ { "apiKey": 36, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "SaslAuthenticateRequest", // Version 1 is the same as version 0. // Version 2 adds flexible version support diff --git a/clients/src/main/resources/common/message/SaslHandshakeRequest.json b/clients/src/main/resources/common/message/SaslHandshakeRequest.json index a370a80df3949..d2189d826ead1 100644 --- a/clients/src/main/resources/common/message/SaslHandshakeRequest.json +++ b/clients/src/main/resources/common/message/SaslHandshakeRequest.json @@ -16,7 +16,7 @@ { "apiKey": 17, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "SaslHandshakeRequest", // Version 1 supports SASL_AUTHENTICATE. // NOTE: Version cannot be easily bumped due to incorrect diff --git a/clients/src/main/resources/common/message/SyncGroupRequest.json b/clients/src/main/resources/common/message/SyncGroupRequest.json index 5525844138366..1b53df27757f2 100644 --- a/clients/src/main/resources/common/message/SyncGroupRequest.json +++ b/clients/src/main/resources/common/message/SyncGroupRequest.json @@ -16,7 +16,7 @@ { "apiKey": 14, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "SyncGroupRequest", // Versions 1 and 2 are the same as version 0. // diff --git a/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json b/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json index fd2c34b74906c..59a1f05e0972f 100644 --- a/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json +++ b/clients/src/main/resources/common/message/TxnOffsetCommitRequest.json @@ -16,7 +16,7 @@ { "apiKey": 28, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "TxnOffsetCommitRequest", // Version 1 is the same as version 0. // diff --git a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json index 8de1eeedd908d..e2f9b45d4cb8f 100644 --- a/clients/src/main/resources/common/message/UpdateFeaturesRequest.json +++ b/clients/src/main/resources/common/message/UpdateFeaturesRequest.json @@ -16,7 +16,7 @@ { "apiKey": 57, "type": "request", - "listeners": ["zkBroker", "broker", "controller"], + "listeners": ["broker", "controller"], "name": "UpdateFeaturesRequest", // Version 1 adds validate only field. // diff --git a/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json b/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json index 933d009b5582a..cacda4198e416 100644 --- a/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json +++ b/clients/src/main/resources/common/message/WriteTxnMarkersRequest.json @@ -16,7 +16,7 @@ { "apiKey": 27, "type": "request", - "listeners": ["zkBroker", "broker"], + "listeners": ["broker"], "name": "WriteTxnMarkersRequest", // Version 0 was removed in Apache Kafka 4.0, Version 1 is the new baseline. // diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java index ce7d4d83506d1..cef48b65bb69f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -385,7 +385,7 @@ private void setExpectedApiVersionsResponse(ApiVersionsResponse response) { private void awaitReady(NetworkClient client, Node node) { if (client.discoverBrokerVersions()) { setExpectedApiVersionsResponse(TestUtils.defaultApiVersionsResponse( - ApiMessageType.ListenerType.ZK_BROKER)); + ApiMessageType.ListenerType.BROKER)); } while (!client.ready(node, time.milliseconds())) client.poll(1, time.milliseconds()); @@ -1455,7 +1455,7 @@ private void awaitInFlightApiVersionRequest() throws Exception { } private ApiVersionsResponse defaultApiVersionsResponse() { - return TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER); + return TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.BROKER); } private static class TestCallbackHandler implements RequestCompletionHandler { diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java index 6e521c65898da..7c7b02fc9cdf4 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/KafkaAdminClientTest.java @@ -776,7 +776,7 @@ private static ApiVersionsResponse prepareApiVersionsResponseForDescribeFeatures if (error == Errors.NONE) { return new ApiVersionsResponse.Builder(). setApiVersions(ApiVersionsResponse.filterApis( - ApiMessageType.ListenerType.ZK_BROKER, false, false)). + ApiMessageType.ListenerType.BROKER, false, false)). setSupportedFeatures( convertSupportedFeaturesMap(defaultFeatureMetadata().supportedFeatures())). setFinalizedFeatures( diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 8657dcfc1e99d..6505a167a3353 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -1902,7 +1902,7 @@ time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogConte MetadataRecoveryStrategy.NONE); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( - 400, ApiMessageType.ListenerType.ZK_BROKER); + 400, ApiMessageType.ListenerType.BROKER); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(apiVersionsResponse, ApiKeys.API_VERSIONS.latestVersion(), 0); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index ede973c5f9b4f..729668f807626 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -1888,7 +1888,7 @@ time, true, new ApiVersions(), metricsManager.throttleTimeSensor(), new LogConte MetadataRecoveryStrategy.NONE); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( - 400, ApiMessageType.ListenerType.ZK_BROKER); + 400, ApiMessageType.ListenerType.BROKER); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(apiVersionsResponse, ApiKeys.API_VERSIONS.latestVersion(), 0); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index 3dd612b0d7982..f66ce1f906076 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -221,7 +221,7 @@ time, true, new ApiVersions(), throttleTimeSensor, logContext, MetadataRecoveryStrategy.NONE); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( - 400, ApiMessageType.ListenerType.ZK_BROKER); + 400, ApiMessageType.ListenerType.BROKER); ByteBuffer buffer = RequestTestUtils.serializeResponseWithHeader(apiVersionsResponse, ApiKeys.API_VERSIONS.latestVersion(), 0); selector.delayedReceive(new DelayedReceive(node.idString(), new NetworkReceive(node.idString(), buffer))); diff --git a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java index 9bdc7c38c4477..90dd34bb07835 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java +++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java @@ -129,7 +129,7 @@ public NioEchoServer(ListenerName listenerName, SecurityProtocol securityProtoco if (channelBuilder == null) channelBuilder = ChannelBuilders.serverChannelBuilder(listenerName, false, securityProtocol, config, credentialCache, tokenCache, time, logContext, - version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER)); + version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.BROKER)); this.metrics = new Metrics(); this.selector = new Selector(10000, failedAuthenticationDelayMs, metrics, time, "MetricGroup", channelBuilder, logContext); diff --git a/clients/src/test/java/org/apache/kafka/common/network/SaslChannelBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/network/SaslChannelBuilderTest.java index eae6fb0b0a036..3366a46e06941 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SaslChannelBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SaslChannelBuilderTest.java @@ -179,7 +179,7 @@ protected GSSManager gssManager() { } private Function defaultApiVersionsSupplier() { - return version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER); + return version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.BROKER); } private SaslChannelBuilder createChannelBuilder(SecurityProtocol securityProtocol, String saslMechanism) { diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index 8a5bf6fdad4e4..9208171d1a926 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -1356,7 +1356,7 @@ private interface FailureAction { } private Function defaultApiVersionsSupplier() { - return version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER); + return version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.BROKER); } static class TestSslChannelBuilder extends SslChannelBuilder { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java index f87ad0fbf54ba..dd8b8144a29e9 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/ApiVersionsResponseTest.java @@ -33,9 +33,11 @@ import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; +import java.util.Arrays; import java.util.Collections; import java.util.HashSet; import java.util.Map; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -99,7 +101,7 @@ public void shouldHaveCommonlyAgreedApiVersionResponseWithControllerOnForwardabl ); ApiVersionCollection commonResponse = ApiVersionsResponse.intersectForwardableApis( - ApiMessageType.ListenerType.ZK_BROKER, + ApiMessageType.ListenerType.BROKER, activeControllerApiVersions, true, false @@ -111,20 +113,19 @@ public void shouldHaveCommonlyAgreedApiVersionResponseWithControllerOnForwardabl ApiKeys.JOIN_GROUP.latestVersion(), commonResponse); } - @ParameterizedTest - @EnumSource(names = {"ZK_BROKER", "BROKER"}) - public void shouldReturnAllKeysWhenThrottleMsIsDefaultThrottle(ListenerType listenerType) { + @Test + public void shouldReturnAllKeysWhenThrottleMsIsDefaultThrottle() { ApiVersionsResponse response = new ApiVersionsResponse.Builder(). setThrottleTimeMs(AbstractResponse.DEFAULT_THROTTLE_TIME). setApiVersions(ApiVersionsResponse.filterApis( - listenerType, + ListenerType.BROKER, true, true)). setSupportedFeatures(Features.emptySupportedFeatures()). setFinalizedFeatures(Collections.emptyMap()). setFinalizedFeaturesEpoch(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH). build(); - assertEquals(new HashSet<>(ApiKeys.apisForListener(listenerType)), apiKeysInResponse(response)); + assertEquals(new HashSet<>(ApiKeys.apisForListener(ListenerType.BROKER)), apiKeysInResponse(response)); assertEquals(AbstractResponse.DEFAULT_THROTTLE_TIME, response.throttleTimeMs()); assertTrue(response.data().supportedFeatures().isEmpty()); assertTrue(response.data().finalizedFeatures().isEmpty()); @@ -160,25 +161,30 @@ public void shouldNotCreateApiResponseWithTelemetryWhenDisabled() { build(); verifyApiKeysForTelemetry(response, 0); } - + @Test - public void testMetadataQuorumApisAreDisabled() { + public void testBrokerApisAreEnabled() { ApiVersionsResponse response = new ApiVersionsResponse.Builder(). setThrottleTimeMs(AbstractResponse.DEFAULT_THROTTLE_TIME). setApiVersions(ApiVersionsResponse.filterApis( - ListenerType.ZK_BROKER, + ListenerType.BROKER, true, true)). setSupportedFeatures(Features.emptySupportedFeatures()). setFinalizedFeatures(Collections.emptyMap()). setFinalizedFeaturesEpoch(ApiVersionsResponse.UNKNOWN_FINALIZED_FEATURES_EPOCH). build(); - // Ensure that APIs needed for the KRaft mode are not exposed through ApiVersions until we are ready for them - HashSet exposedApis = apiKeysInResponse(response); - assertFalse(exposedApis.contains(ApiKeys.VOTE)); - assertFalse(exposedApis.contains(ApiKeys.BEGIN_QUORUM_EPOCH)); - assertFalse(exposedApis.contains(ApiKeys.END_QUORUM_EPOCH)); - assertFalse(exposedApis.contains(ApiKeys.DESCRIBE_QUORUM)); + + Set exposed = apiKeysInResponse(response); + + + Arrays.stream(ApiKeys.values()) + .filter(key -> key.messageType.listeners().contains(ListenerType.BROKER)) + .forEach(key -> assertTrue(exposed.contains(key))); + Arrays.stream(ApiKeys.values()) + .filter(key -> key.messageType.listeners() + .stream().noneMatch(listener -> listener == ListenerType.BROKER)) + .forEach(key -> assertFalse(exposed.contains(key))); } @Test @@ -251,12 +257,6 @@ private void verifyVersions(short forwardableAPIKey, assertEquals(expectedVersionsForForwardableAPI, commonResponse.find(forwardableAPIKey)); } - private void verifyApiKeysForMagic(ApiVersionsResponse response, Byte maxMagic) { - for (ApiVersion version : response.data().apiKeys()) { - assertTrue(ApiKeys.forId(version.apiKey()).minRequiredInterBrokerMagic <= maxMagic); - } - } - private void verifyApiKeysForTelemetry(ApiVersionsResponse response, int expectedCount) { int count = 0; for (ApiVersion version : response.data().apiKeys()) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index e8bdd21d33de6..0acaf9fc7d051 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -407,7 +407,7 @@ public void testSerializationSpecialCases() { public void testApiVersionsSerialization() { for (short version : API_VERSIONS.allVersions()) { checkErrorResponse(createApiVersionRequest(version), new UnsupportedVersionException("Not Supported")); - checkResponse(TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER), version); + checkResponse(TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.BROKER), version); } } @@ -840,7 +840,7 @@ public void testApiVersionResponseWithNotUnsupportedError() { } private ApiVersionsResponse defaultApiVersionsResponse() { - return TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER); + return TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.BROKER); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 3b1e54dee2c62..8261c90014cf3 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -1107,7 +1107,7 @@ public void testServerAuthenticateCallbackHandler() throws Exception { /** * Test that callback handlers are only applied to connections for the mechanisms - * configured for the handler. Test enables two mechanisms 'PLAIN` and `DIGEST-MD5` + * configured for the handler. Test enables two mechanisms `PLAIN` and `DIGEST-MD5` * on the servers with different callback handlers for the two mechanisms. Verifies * that clients using both mechanisms authenticate successfully. */ @@ -1980,7 +1980,7 @@ private NioEchoServer startServerWithoutSaslAuthenticateHeader(final SecurityPro Function apiVersionSupplier = version -> { ApiVersionsResponse defaultApiVersionResponse = TestUtils.defaultApiVersionsResponse( - ApiMessageType.ListenerType.ZK_BROKER); + ApiMessageType.ListenerType.BROKER); ApiVersionCollection apiVersions = new ApiVersionCollection(); for (ApiVersion apiVersion : defaultApiVersionResponse.data().apiKeys()) { if (apiVersion.apiKey() != ApiKeys.SASL_AUTHENTICATE.id) { @@ -2574,7 +2574,7 @@ public AlternateSaslChannelBuilder(ConnectionMode connectionMode, Map TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.ZK_BROKER)); + version -> TestUtils.defaultApiVersionsResponse(ApiMessageType.ListenerType.BROKER)); } @Override diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java index 81df34f85f4b9..0cc6f6f94b46d 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java @@ -401,7 +401,7 @@ private SaslServerAuthenticator setupAuthenticator(Map configs, Trans Map callbackHandlers = Collections.singletonMap( mechanism, new SaslServerCallbackHandler()); ApiVersionsResponse apiVersionsResponse = TestUtils.defaultApiVersionsResponse( - ApiMessageType.ListenerType.ZK_BROKER); + ApiMessageType.ListenerType.BROKER); Map connectionsMaxReauthMsByMechanism = maxReauth != null ? Collections.singletonMap(mechanism, maxReauth) : Collections.emptyMap(); diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index 667b552333670..aa202c0df10f4 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -270,7 +270,7 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { val jaasContexts = Collections.singletonMap("GSSAPI", JaasContext.loadClientContext(config.values())) val channelBuilder = new SaslChannelBuilder(ConnectionMode.CLIENT, jaasContexts, securityProtocol, null, false, kafkaClientSaslMechanism, null, null, null, time, new LogContext(), - _ => org.apache.kafka.test.TestUtils.defaultApiVersionsResponse(ListenerType.ZK_BROKER)) { + _ => org.apache.kafka.test.TestUtils.defaultApiVersionsResponse(ListenerType.BROKER)) { override protected def defaultLoginClass(): Class[_ <: Login] = classOf[TestableKerberosLogin] } channelBuilder.configure(config.values()) diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala index 341c859bf32da..1d506b3b7c229 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionManagerTest.scala @@ -86,7 +86,7 @@ class ApiVersionManagerTest { ))) val versionManager = new DefaultApiVersionManager( - listenerType = ListenerType.ZK_BROKER, + listenerType = ListenerType.BROKER, forwardingManager = forwardingManager, brokerFeatures = brokerFeatures, metadataCache = metadataCache, diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 470fb20304d73..ae910956011f2 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -183,7 +183,7 @@ class RequestQuotaTest extends BaseRequestTest { def testUnauthorizedThrottle(quorum: String): Unit = { RequestQuotaTest.principal = RequestQuotaTest.UnauthorizedPrincipal - val apiKeys = ApiKeys.kraftBrokerApis + val apiKeys = ApiKeys.brokerApis for (apiKey <- apiKeys.asScala.toSet -- RequestQuotaTest.Envelope) { submitTest(apiKey, () => checkUnauthorizedRequestThrottle(apiKey)) } @@ -192,11 +192,11 @@ class RequestQuotaTest extends BaseRequestTest { } private def clientActions: Set[ApiKeys] = { - ApiKeys.kraftBrokerApis.asScala.toSet -- clusterActions -- RequestQuotaTest.SaslActions -- RequestQuotaTest.Envelope + ApiKeys.brokerApis.asScala.toSet -- clusterActions -- RequestQuotaTest.SaslActions -- RequestQuotaTest.Envelope } private def clusterActions: Set[ApiKeys] = { - ApiKeys.kraftBrokerApis.asScala.filter(_.clusterAction).toSet + ApiKeys.brokerApis.asScala.filter(_.clusterAction).toSet } private def clusterActionsWithThrottleForBroker: Set[ApiKeys] = { diff --git a/generator/src/main/java/org/apache/kafka/message/RequestListenerType.java b/generator/src/main/java/org/apache/kafka/message/RequestListenerType.java index 729da21f4c40c..ac3bfca480ba2 100644 --- a/generator/src/main/java/org/apache/kafka/message/RequestListenerType.java +++ b/generator/src/main/java/org/apache/kafka/message/RequestListenerType.java @@ -19,8 +19,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; public enum RequestListenerType { - @JsonProperty("zkBroker") - ZK_BROKER, @JsonProperty("broker") BROKER, From 8c0a0e07ced062419cf51c31307e5d168ad9efbc Mon Sep 17 00:00:00 2001 From: David Arthur Date: Fri, 24 Jan 2025 09:03:43 -0500 Subject: [PATCH 44/44] KAFKA-17587 Refactor test infrastructure (#18602) This patch reorganizes our test infrastructure into three Gradle modules: ":test-common:test-common-internal-api" is now a minimal dependency which exposes interfaces and annotations only. It has one project dependency on server-common to expose commonly used data classes (MetadataVersion, Feature, etc). Since this pulls in server-common, this module is Java 17+. It cannot be used by ":clients" or other Java 11 modules. ":test-common:test-common-util" includes the auto-quarantined JUnit extension. The @Flaky annotation has been moved here. Since this module has no project dependencies, we can add it to the Java 11 list so that ":clients" and others can utilize the @Flaky annotation ":test-common:test-common-runtime" now includes all of the test infrastructure code (TestKitNodes, etc). This module carries heavy dependencies (core, etc) and so it should not normally be included as a compile-time dependency. In addition to this reorganization, this patch leverages JUnit SPI service discovery so that modules can utilize the integration test framework without depending on ":core". This will allow us to start moving integration tests out of core and into the appropriate sub-module. This is done by adding ":test-common:test-common-runtime" as a testRuntimeOnly dependency rather than as a testImplementation dependency. A trivial example was added to QuorumControllerTest to illustrate this. Reviewers: Ismael Juma , Chia-Ping Tsai --- .github/scripts/junit.py | 5 +- build.gradle | 107 +++++++++--------- .../import-control-coordinator-common.xml | 1 + checkstyle/import-control-core.xml | 1 + .../import-control-group-coordinator.xml | 1 + checkstyle/import-control-metadata.xml | 1 + checkstyle/import-control-server-common.xml | 1 + checkstyle/import-control-server.xml | 1 + .../import-control-share-coordinator.xml | 1 + checkstyle/import-control-storage.xml | 1 + ...mport-control-test-common-internal-api.xml | 43 +++++++ ...=> import-control-test-common-runtime.xml} | 8 +- ...ml => import-control-test-common-util.xml} | 6 +- ...import-control-transaction-coordinator.xml | 1 + checkstyle/import-control.xml | 3 + .../internals/AbstractCoordinatorTest.java | 8 +- .../kafka/admin/AdminFenceProducersTest.java | 6 +- .../java/kafka/admin/ClientTelemetryTest.java | 6 +- .../admin/ConfigCommandIntegrationTest.java | 5 +- .../java/kafka/admin/DeleteTopicTest.java | 6 +- .../UserScramCredentialsCommandTest.java | 6 +- .../consumer/ConsumerIntegrationTest.java | 6 +- .../BootstrapControllersIntegrationTest.java | 5 +- .../server/LogManagerIntegrationTest.java | 6 +- .../kafka/test/api/ShareConsumerTest.java | 5 +- .../api/GroupCoordinatorIntegrationTest.scala | 5 +- .../transaction/ProducerIntegrationTest.scala | 6 +- .../kafka/server/KRaftClusterTest.scala | 2 +- .../MetadataVersionIntegrationTest.scala | 5 +- .../server/RaftClusterSnapshotTest.scala | 3 +- .../AbstractApiVersionsRequestTest.scala | 2 +- .../AllocateProducerIdsRequestTest.scala | 5 +- .../kafka/server/ApiVersionsRequestTest.scala | 5 +- .../kafka/server/BrokerMetricNamesTest.scala | 5 +- .../BrokerRegistrationRequestTest.scala | 5 +- .../server/ClientQuotasRequestTest.scala | 5 +- .../ConsumerGroupDescribeRequestTest.scala | 5 +- .../ConsumerGroupHeartbeatRequestTest.scala | 5 +- .../ConsumerProtocolMigrationTest.scala | 5 +- .../kafka/server/ControllerApisTest.scala | 2 +- .../server/DeleteGroupsRequestTest.scala | 5 +- .../server/DescribeGroupsRequestTest.scala | 5 +- .../server/DescribeQuorumRequestTest.scala | 5 +- .../GroupCoordinatorBaseRequestTest.scala | 2 +- .../kafka/server/HeartbeatRequestTest.scala | 5 +- .../kafka/server/JoinGroupRequestTest.scala | 5 +- .../kafka/server/LeaveGroupRequestTest.scala | 5 +- .../kafka/server/ListGroupsRequestTest.scala | 5 +- .../server/OffsetCommitRequestTest.scala | 5 +- .../server/OffsetDeleteRequestTest.scala | 5 +- .../kafka/server/OffsetFetchRequestTest.scala | 6 +- .../server/SaslApiVersionsRequestTest.scala | 5 +- .../ShareFetchAcknowledgeRequestTest.scala | 5 +- .../ShareGroupDescribeRequestTest.scala | 5 +- .../ShareGroupHeartbeatRequestTest.scala | 5 +- .../kafka/server/SyncGroupRequestTest.scala | 5 +- .../server/TxnOffsetCommitRequestTest.scala | 5 +- .../BrokerMetadataPublisherTest.scala | 2 +- settings.gradle | 4 +- .../RemoteLogSegmentLifecycleTest.java | 6 +- ...adataManagerMultipleSubscriptionsTest.java | 6 +- ...edRemoteLogMetadataManagerRestartTest.java | 6 +- ...opicBasedRemoteLogMetadataManagerTest.java | 5 +- .../storage/RemoteLogMetadataManagerTest.java | 6 +- .../kafka/common/test/api/AutoStart.java | 0 .../kafka/common/test/api/ClusterConfig.java | 8 +- .../test/api/ClusterConfigProperty.java | 0 .../kafka/common/test/api/ClusterFeature.java | 0 .../common/test/api/ClusterGenerator.java | 0 .../common/test/api/ClusterTemplate.java | 0 .../kafka/common/test/api/ClusterTest.java | 4 +- .../common/test/api/ClusterTestDefaults.java | 0 .../kafka/common/test/api/ClusterTests.java | 0 .../common/test/api/DetectThreadLeak.java | 0 .../apache/kafka/common/test/api/README.md | 2 +- .../common/test/api/TestKitDefaults.java | 35 ++++++ .../apache/kafka/common/test/api/Type.java | 18 +-- .../common/test/api/ClusterConfigTest.java | 13 ++- .../common/test/api/DetectThreadLeakTest.java | 0 .../src/test}/resources/log4j2.yaml | 0 .../kafka/common/test}/ClusterInstance.java | 7 +- .../apache/kafka/common/test/JaasModule.java | 0 .../apache/kafka/common/test/JaasUtils.java | 0 .../common/test/KafkaClusterTestKit.java | 0 .../test/KafkaClusterThreadFactory.java | 0 .../kafka/common/test/MockController.java | 0 .../kafka/common/test/MockFaultHandler.java | 0 .../test/PreboundSocketFactoryManager.java | 0 .../apache/kafka/common/test/TestKitNode.java | 0 .../kafka/common/test/TestKitNodes.java | 20 ++-- .../apache/kafka/common/test/TestUtils.java | 0 .../ClusterInstanceParameterResolver.java | 8 +- .../test/junit}/ClusterTestExtensions.java | 63 ++++++++--- .../junit}/RaftClusterInvocationContext.java | 5 +- .../org.junit.jupiter.api.extension.Extension | 16 +++ .../src/main}/resources/log4j2.yaml | 0 .../common/test/KafkaClusterTestKitTest.java | 0 .../test/KafkaClusterThreadFactoryTest.java | 0 .../kafka/common/test/TestKitNodeTest.java | 0 .../junit}/ClusterTestExtensionsTest.java | 15 ++- .../junit}/ClusterTestExtensionsUnitTest.java | 5 +- .../apache/kafka/common/test/api/Flaky.java | 0 .../test/junit/AutoQuarantinedTestFilter.java | 0 .../junit/QuarantinedPostDiscoveryFilter.java | 0 ...unit.platform.launcher.PostDiscoveryFilter | 0 .../main/resources/junit-platform.properties | 0 .../junit/AutoQuarantinedTestFilterTest.java | 0 .../QuarantinedPostDiscoveryFilterTest.java | 0 .../apache/kafka/tools/AclCommandTest.java | 5 +- .../tools/BrokerApiVersionsCommandTest.java | 6 +- .../apache/kafka/tools/ClusterToolTest.java | 5 +- .../kafka/tools/DeleteRecordsCommandTest.java | 5 +- .../kafka/tools/FeatureCommandTest.java | 5 +- .../kafka/tools/GetOffsetShellTest.java | 6 +- .../apache/kafka/tools/GroupsCommandTest.java | 5 +- .../tools/LeaderElectionCommandTest.java | 5 +- .../kafka/tools/LogDirsCommandTest.java | 5 +- .../tools/MetadataQuorumCommandTest.java | 5 +- .../apache/kafka/tools/TopicCommandTest.java | 5 +- .../tools/consumer/ConsoleConsumerTest.java | 5 +- .../group/DeleteConsumerGroupsTest.java | 6 +- ...tsConsumerGroupCommandIntegrationTest.java | 5 +- .../group/DescribeConsumerGroupTest.java | 5 +- .../consumer/group/ListConsumerGroupTest.java | 5 +- .../group/ResetConsumerGroupOffsetTest.java | 6 +- .../ReassignPartitionsCommandTest.java | 5 +- 126 files changed, 349 insertions(+), 387 deletions(-) create mode 100644 checkstyle/import-control-test-common-internal-api.xml rename checkstyle/{import-control-test-common.xml => import-control-test-common-runtime.xml} (86%) rename checkstyle/{import-control-test-common-api.xml => import-control-test-common-util.xml} (90%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/AutoStart.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java (97%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java (92%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java (100%) rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/README.md (98%) create mode 100644 test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/TestKitDefaults.java rename test-common/{test-common-api => test-common-internal-api}/src/main/java/org/apache/kafka/common/test/api/Type.java (59%) rename test-common/{test-common-api => test-common-internal-api}/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java (91%) rename test-common/{test-common-api => test-common-internal-api}/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java (100%) rename test-common/{src/main => test-common-internal-api/src/test}/resources/log4j2.yaml (100%) rename test-common/{test-common-api/src/main/java/org/apache/kafka/common/test/api => test-common-runtime/src/main/java/org/apache/kafka/common/test}/ClusterInstance.java (98%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/JaasModule.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/JaasUtils.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/MockController.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/TestKitNode.java (100%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/TestKitNodes.java (94%) rename test-common/{ => test-common-runtime}/src/main/java/org/apache/kafka/common/test/TestUtils.java (100%) rename test-common/{test-common-api/src/main/java/org/apache/kafka/common/test/api => test-common-runtime/src/main/java/org/apache/kafka/common/test/junit}/ClusterInstanceParameterResolver.java (92%) rename test-common/{test-common-api/src/main/java/org/apache/kafka/common/test/api => test-common-runtime/src/main/java/org/apache/kafka/common/test/junit}/ClusterTestExtensions.java (84%) rename test-common/{test-common-api/src/main/java/org/apache/kafka/common/test/api => test-common-runtime/src/main/java/org/apache/kafka/common/test/junit}/RaftClusterInvocationContext.java (98%) create mode 100644 test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.jupiter.api.extension.Extension rename test-common/{test-common-api/src/test => test-common-runtime/src/main}/resources/log4j2.yaml (100%) rename test-common/{ => test-common-runtime}/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java (100%) rename test-common/{ => test-common-runtime}/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java (100%) rename test-common/{ => test-common-runtime}/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java (100%) rename test-common/{test-common-api/src/test/java/org/apache/kafka/common/test/api => test-common-runtime/src/test/java/org/apache/kafka/common/test/junit}/ClusterTestExtensionsTest.java (97%) rename test-common/{test-common-api/src/test/java/org/apache/kafka/common/test/api => test-common-runtime/src/test/java/org/apache/kafka/common/test/junit}/ClusterTestExtensionsUnitTest.java (95%) rename test-common/{test-common-api => test-common-util}/src/main/java/org/apache/kafka/common/test/api/Flaky.java (100%) rename test-common/{test-common-runtime => test-common-util}/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java (100%) rename test-common/{test-common-runtime => test-common-util}/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java (100%) rename test-common/{test-common-runtime => test-common-util}/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter (100%) rename test-common/{test-common-runtime => test-common-util}/src/main/resources/junit-platform.properties (100%) rename test-common/{test-common-runtime => test-common-util}/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java (100%) rename test-common/{test-common-runtime => test-common-util}/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java (100%) diff --git a/.github/scripts/junit.py b/.github/scripts/junit.py index b23f444720ba3..5a2088f8ea196 100644 --- a/.github/scripts/junit.py +++ b/.github/scripts/junit.py @@ -91,7 +91,10 @@ class TestSuite: def clean_test_name(test_name: str) -> str: cleaned = test_name.strip("\"").rstrip("()") m = method_matcher.match(cleaned) - return m.group(1) + if m is None: + raise ValueError(f"Could not parse test name '{test_name}'. Expected a valid Java method name.") + else: + return m.group(1) class TestCatalogExporter: diff --git a/build.gradle b/build.gradle index 882d4a2f8aa12..eb96a0912405f 100644 --- a/build.gradle +++ b/build.gradle @@ -49,7 +49,7 @@ ext { gradleVersion = versions.gradle minClientJavaVersion = 11 minNonClientJavaVersion = 17 - modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams-scala", ":test-common:test-common-runtime"] + modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams-scala", ":test-common:test-common-util"] buildVersionFileName = "kafka-version.properties" @@ -139,10 +139,11 @@ ext { runtimeTestLibs = [ libs.slf4jLog4j2, libs.junitPlatformLanucher, - project(":test-common:test-common-runtime") + libs.jacksonDatabindYaml, + project(":test-common:test-common-util") ] - log4jRuntimeLibs = [ + log4jReleaseLibs = [ libs.slf4jLog4j2, libs.log4j1Bridge2Api, libs.jacksonDatabindYaml @@ -1059,7 +1060,7 @@ project(':core') { } dependencies { - releaseOnly log4jRuntimeLibs + releaseOnly log4jReleaseLibs // `core` is often used in users' tests, define the following dependencies as `api` for backwards compatibility // even though the `core` module doesn't expose any public API api project(':clients') @@ -1102,8 +1103,9 @@ project(':core') { testImplementation project(':server-common').sourceSets.test.output testImplementation project(':storage:storage-api').sourceSets.test.output testImplementation project(':server').sourceSets.test.output - testImplementation project(':test-common') - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-runtime') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-util') testImplementation libs.bcpkix testImplementation libs.mockitoCore testImplementation(libs.apacheda) { @@ -1535,21 +1537,17 @@ project(':group-coordinator') { srcJar.dependsOn 'processMessages' } -project(':test-common') { - // Test framework stuff. Implementations that support test-common-api + +project(':test-common:test-common-internal-api') { + // Interfaces, config classes, and other test APIs. Java 17 only base { - archivesName = "kafka-test-common" + archivesName = "kafka-test-common-internal-api" } dependencies { - implementation project(':core') - implementation project(':metadata') - implementation project(':server') - implementation project(':raft') - implementation project(':storage') - implementation project(':server-common') - implementation libs.jacksonDatabindYaml - implementation libs.slf4jApi + implementation project(':server-common') // Only project dependency allowed + + implementation libs.junitJupiterApi testImplementation libs.junitJupiter testImplementation libs.mockitoCore @@ -1559,7 +1557,7 @@ project(':test-common') { } checkstyle { - configProperties = checkstyleConfigProperties("import-control-test-common.xml") + configProperties = checkstyleConfigProperties("import-control-test-common-internal-api.xml") } javadoc { @@ -1567,33 +1565,22 @@ project(':test-common') { } } -project(':test-common:test-common-api') { - // Interfaces, config classes, and other test APIs +project(':test-common:test-common-util') { + // Runtime-only JUnit extensions for entire project. Java 11 only base { - archivesName = "kafka-test-common-api" + archivesName = "kafka-test-common-util" } dependencies { - implementation project(':clients') - implementation project(':core') - implementation project(':group-coordinator') - implementation project(':metadata') - implementation project(':raft') - implementation project(':server') - implementation project(':server-common') - implementation project(':storage') - implementation project(':test-common') + implementation libs.junitPlatformLanucher implementation libs.junitJupiterApi - - testImplementation libs.junitJupiter - testImplementation libs.mockitoCore + implementation libs.junitJupiter + implementation libs.slf4jApi testImplementation testLog4j2Libs - - testRuntimeOnly runtimeTestLibs } checkstyle { - configProperties = checkstyleConfigProperties("import-control-test-common-api.xml") + configProperties = checkstyleConfigProperties("import-control-test-common-util.xml") } javadoc { @@ -1602,21 +1589,36 @@ project(':test-common:test-common-api') { } project(':test-common:test-common-runtime') { - // Runtime-only test code including JUnit extentions + // Runtime-only JUnit extensions for integration tests. Java 17 only base { archivesName = "kafka-test-common-runtime" } dependencies { + implementation project(':test-common:test-common-internal-api') + implementation project(':clients') + implementation project(':core') + implementation project(':group-coordinator') + implementation project(':metadata') + implementation project(':raft') + implementation project(':server') + implementation project(':server-common') + implementation project(':storage') + implementation libs.junitPlatformLanucher - implementation libs.junitJupiterApi implementation libs.junitJupiter + implementation libs.jacksonDatabindYaml implementation libs.slf4jApi + + testImplementation libs.junitJupiter + testImplementation libs.mockitoCore testImplementation testLog4j2Libs + + testRuntimeOnly runtimeTestLibs } checkstyle { - configProperties = checkstyleConfigProperties("import-control-test-common-api.xml") + configProperties = checkstyleConfigProperties("import-control-test-common-runtime.xml") } javadoc { @@ -1644,8 +1646,8 @@ project(':transaction-coordinator') { testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation project(':clients').sourceSets.test.output - testImplementation project(':test-common') - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-runtime') + testImplementation project(':test-common:test-common-internal-api') testRuntimeOnly runtimeTestLibs @@ -1866,6 +1868,7 @@ project(':clients') { compileOnly libs.jose4j // for SASL/OAUTHBEARER JWT validation; only used by broker + testImplementation project(':test-common:test-common-util') testImplementation libs.bcpkix testImplementation libs.jacksonJakartarsJsonProvider testImplementation libs.jose4j @@ -1880,7 +1883,6 @@ project(':clients') { testRuntimeOnly libs.jacksonDatabind testRuntimeOnly libs.jacksonJDK8Datatypes testRuntimeOnly runtimeTestLibs - testRuntimeOnly log4jRuntimeLibs generator project(':generator') } @@ -2267,7 +2269,8 @@ project(':storage') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':core') testImplementation project(':core').sourceSets.test.output - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server') testImplementation project(':server-common') testImplementation project(':server-common').sourceSets.test.output @@ -2424,7 +2427,7 @@ project(':tools') { } dependencies { - releaseOnly log4jRuntimeLibs + releaseOnly log4jReleaseLibs implementation project(':clients') implementation project(':metadata') @@ -2456,7 +2459,8 @@ project(':tools') { testImplementation project(':server').sourceSets.test.output testImplementation project(':core') testImplementation project(':core').sourceSets.test.output - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server-common') testImplementation project(':server-common').sourceSets.test.output testImplementation project(':connect:api') @@ -2467,7 +2471,6 @@ project(':tools') { testImplementation project(':streams') testImplementation project(':streams').sourceSets.test.output testImplementation project(':streams:integration-tests').sourceSets.test.output - testImplementation project(':test-common') testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension @@ -2648,7 +2651,6 @@ project(':streams') { testRuntimeOnly project(':streams:test-utils') testRuntimeOnly runtimeTestLibs - testRuntimeOnly log4jRuntimeLibs generator project(':generator') } @@ -2839,7 +2841,7 @@ project(':streams:integration-tests') { testImplementation project(':storage') testImplementation project(':streams').sourceSets.test.output testImplementation project(':streams:streams-scala') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':tools') testImplementation project(':transaction-coordinator') testImplementation libs.bcpkix @@ -3515,14 +3517,15 @@ project(':connect:runtime') { testImplementation project(':server') testImplementation project(':metadata') testImplementation project(':server-common') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-util') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server-common') testImplementation project(':server') testImplementation project(':group-coordinator') testImplementation project(':storage') testImplementation project(':connect:test-plugins') testImplementation project(':server-common').sourceSets.test.output - testImplementation project(':test-common:test-common-api') testImplementation libs.jacksonDatabindYaml testImplementation libs.junitJupiter @@ -3636,7 +3639,7 @@ project(':connect:file') { testImplementation project(':connect:runtime') testImplementation project(':connect:runtime').sourceSets.test.output testImplementation project(':core') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server-common').sourceSets.test.output testRuntimeOnly runtimeTestLibs @@ -3740,7 +3743,7 @@ project(':connect:mirror') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':connect:runtime').sourceSets.test.output testImplementation project(':core') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server') testImplementation project(':server-common').sourceSets.test.output diff --git a/checkstyle/import-control-coordinator-common.xml b/checkstyle/import-control-coordinator-common.xml index eb107a4b17a51..c08955fd4228a 100644 --- a/checkstyle/import-control-coordinator-common.xml +++ b/checkstyle/import-control-coordinator-common.xml @@ -37,6 +37,7 @@ + diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 3cfd0ce663cc1..ce3ae6ea7c544 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -36,6 +36,7 @@ + diff --git a/checkstyle/import-control-group-coordinator.xml b/checkstyle/import-control-group-coordinator.xml index 0619ea444d5f6..19d9d5616135d 100644 --- a/checkstyle/import-control-group-coordinator.xml +++ b/checkstyle/import-control-group-coordinator.xml @@ -45,6 +45,7 @@ + diff --git a/checkstyle/import-control-metadata.xml b/checkstyle/import-control-metadata.xml index d45fbc7d16014..484d6866db682 100644 --- a/checkstyle/import-control-metadata.xml +++ b/checkstyle/import-control-metadata.xml @@ -44,6 +44,7 @@ + diff --git a/checkstyle/import-control-server-common.xml b/checkstyle/import-control-server-common.xml index ac8706686617c..7d7ddb70d87e4 100644 --- a/checkstyle/import-control-server-common.xml +++ b/checkstyle/import-control-server-common.xml @@ -44,6 +44,7 @@ + diff --git a/checkstyle/import-control-server.xml b/checkstyle/import-control-server.xml index f046ceb74a64f..6c3332b1cfd4f 100644 --- a/checkstyle/import-control-server.xml +++ b/checkstyle/import-control-server.xml @@ -47,6 +47,7 @@ + diff --git a/checkstyle/import-control-share-coordinator.xml b/checkstyle/import-control-share-coordinator.xml index d574588ac65c1..aaea93d32e648 100644 --- a/checkstyle/import-control-share-coordinator.xml +++ b/checkstyle/import-control-share-coordinator.xml @@ -40,6 +40,7 @@ + diff --git a/checkstyle/import-control-storage.xml b/checkstyle/import-control-storage.xml index fecc55f98ea11..d0b6524d9e128 100644 --- a/checkstyle/import-control-storage.xml +++ b/checkstyle/import-control-storage.xml @@ -45,6 +45,7 @@ + diff --git a/checkstyle/import-control-test-common-internal-api.xml b/checkstyle/import-control-test-common-internal-api.xml new file mode 100644 index 0000000000000..c86fc3ab831dc --- /dev/null +++ b/checkstyle/import-control-test-common-internal-api.xml @@ -0,0 +1,43 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/checkstyle/import-control-test-common.xml b/checkstyle/import-control-test-common-runtime.xml similarity index 86% rename from checkstyle/import-control-test-common.xml rename to checkstyle/import-control-test-common-runtime.xml index 9520c0b21b86e..f85c117edea32 100644 --- a/checkstyle/import-control-test-common.xml +++ b/checkstyle/import-control-test-common-runtime.xml @@ -19,10 +19,14 @@ --> + + + + + + - - diff --git a/checkstyle/import-control-test-common-api.xml b/checkstyle/import-control-test-common-util.xml similarity index 90% rename from checkstyle/import-control-test-common-api.xml rename to checkstyle/import-control-test-common-util.xml index 7331cc2d35e98..05e0cc0662e3d 100644 --- a/checkstyle/import-control-test-common-api.xml +++ b/checkstyle/import-control-test-common-util.xml @@ -19,8 +19,8 @@ --> + - - - + + diff --git a/checkstyle/import-control-transaction-coordinator.xml b/checkstyle/import-control-transaction-coordinator.xml index f4045907257a3..bf2157750c3a6 100644 --- a/checkstyle/import-control-transaction-coordinator.xml +++ b/checkstyle/import-control-transaction-coordinator.xml @@ -37,6 +37,7 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 921db8162e33a..34f5220a86853 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -46,6 +46,9 @@ + + + diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 969de83c328ff..f144ccf5061c5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -51,6 +51,7 @@ import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; +import org.apache.kafka.common.test.api.Flaky; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -59,7 +60,6 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import java.nio.ByteBuffer; @@ -1435,7 +1435,7 @@ public void testWakeupAfterJoinGroupReceivedExternalCompletion() throws Exceptio awaitFirstHeartbeat(heartbeatReceived); } - @Tag("flaky") // "KAFKA-18310" + @Flaky("KAFKA-18310") @Test public void testWakeupAfterSyncGroupSentExternalCompletion() throws Exception { setupCoordinator(); @@ -1472,7 +1472,7 @@ public boolean matches(AbstractRequest body) { awaitFirstHeartbeat(heartbeatReceived); } - @Tag("flaky") // "KAFKA-18310" + @Flaky("KAFKA-18310") @Test public void testWakeupAfterSyncGroupReceived() throws Exception { setupCoordinator(); @@ -1506,7 +1506,7 @@ public void testWakeupAfterSyncGroupReceived() throws Exception { awaitFirstHeartbeat(heartbeatReceived); } - @Tag("flaky") // KAFKA-15474 and KAFKA-18310 + @Flaky("KAFKA-15474,KAFKA-18310") @Test public void testWakeupAfterSyncGroupReceivedExternalCompletion() throws Exception { setupCoordinator(); diff --git a/core/src/test/java/kafka/admin/AdminFenceProducersTest.java b/core/src/test/java/kafka/admin/AdminFenceProducersTest.java index 7b0ac1445d421..05a42c10f58e9 100644 --- a/core/src/test/java/kafka/admin/AdminFenceProducersTest.java +++ b/core/src/test/java/kafka/admin/AdminFenceProducersTest.java @@ -27,17 +27,14 @@ import org.apache.kafka.common.errors.InvalidProducerEpochException; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.coordinator.transaction.TransactionLogConfig; import org.apache.kafka.coordinator.transaction.TransactionStateManagerConfig; import org.apache.kafka.server.config.ServerLogConfigs; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -54,7 +51,6 @@ @ClusterConfigProperty(key = TransactionLogConfig.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, value = "1"), @ClusterConfigProperty(key = TransactionStateManagerConfig.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_CONFIG, value = "2000") }) -@ExtendWith(ClusterTestExtensions.class) public class AdminFenceProducersTest { private static final String TOPIC_NAME = "mytopic"; private static final String TXN_ID = "mytxnid"; diff --git a/core/src/test/java/kafka/admin/ClientTelemetryTest.java b/core/src/test/java/kafka/admin/ClientTelemetryTest.java index f701d3f38450e..e43f52f7271a7 100644 --- a/core/src/test/java/kafka/admin/ClientTelemetryTest.java +++ b/core/src/test/java/kafka/admin/ClientTelemetryTest.java @@ -37,16 +37,13 @@ import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.telemetry.ClientTelemetry; import org.apache.kafka.server.telemetry.ClientTelemetryReceiver; -import org.junit.jupiter.api.extension.ExtendWith; - import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -68,7 +65,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class ClientTelemetryTest { @ClusterTest( diff --git a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java index 5731692f98e1d..31d0de824f176 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java @@ -25,16 +25,14 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mockito; import java.io.ByteArrayOutputStream; @@ -71,7 +69,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; -@ExtendWith(value = ClusterTestExtensions.class) public class ConfigCommandIntegrationTest { private final String defaultBrokerId = "0"; private final String defaultGroupName = "group"; diff --git a/core/src/test/java/kafka/admin/DeleteTopicTest.java b/core/src/test/java/kafka/admin/DeleteTopicTest.java index be87e086f7f5a..522462f4b3be5 100644 --- a/core/src/test/java/kafka/admin/DeleteTopicTest.java +++ b/core/src/test/java/kafka/admin/DeleteTopicTest.java @@ -30,12 +30,11 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.TestUtils; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.metadata.BrokerState; @@ -44,8 +43,6 @@ import org.apache.kafka.storage.internals.log.AppendOrigin; import org.apache.kafka.storage.internals.log.VerificationGuard; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -64,7 +61,6 @@ import static org.junit.jupiter.api.Assertions.fail; -@ExtendWith(value = ClusterTestExtensions.class) @ClusterTestDefaults(types = {Type.KRAFT}, brokers = 3, serverProperties = { diff --git a/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java b/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java index 697dda07363a7..8ca269f4b6866 100644 --- a/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java +++ b/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java @@ -16,15 +16,12 @@ */ package kafka.admin; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.test.NoRetryException; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -37,7 +34,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @SuppressWarnings("dontUseSystemExit") -@ExtendWith(value = ClusterTestExtensions.class) public class UserScramCredentialsCommandTest { private static final String USER1 = "user1"; private static final String USER2 = "user2"; diff --git a/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java b/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java index 542b9c48d0d7b..98cd4d0449bc8 100644 --- a/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java +++ b/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java @@ -31,16 +31,13 @@ import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.TestUtils; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.ClusterTests; import org.apache.kafka.common.test.api.Type; -import org.junit.jupiter.api.extension.ExtendWith; - import java.time.Duration; import java.util.Collection; import java.util.Collections; @@ -50,7 +47,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(ClusterTestExtensions.class) public class ConsumerIntegrationTest { @ClusterTests({ diff --git a/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java b/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java index 75cd070b93d30..dcee16b8ec62c 100644 --- a/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java +++ b/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java @@ -51,18 +51,16 @@ import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.metadata.authorizer.StandardAuthorizer; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Timeout; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; @@ -88,7 +86,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(120) -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(types = {Type.KRAFT}) public class BootstrapControllersIntegrationTest { private Map adminConfig(ClusterInstance clusterInstance, boolean usingBootstrapControllers) { diff --git a/core/src/test/java/kafka/server/LogManagerIntegrationTest.java b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java index 05b5e8423bab4..3d386283943f5 100644 --- a/core/src/test/java/kafka/server/LogManagerIntegrationTest.java +++ b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java @@ -31,15 +31,12 @@ import org.apache.kafka.common.TopicPartitionInfo; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -55,7 +52,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class LogManagerIntegrationTest { private final ClusterInstance cluster; diff --git a/core/src/test/java/kafka/test/api/ShareConsumerTest.java b/core/src/test/java/kafka/test/api/ShareConsumerTest.java index c7e62d0eb70d0..9018761429a03 100644 --- a/core/src/test/java/kafka/test/api/ShareConsumerTest.java +++ b/core/src/test/java/kafka/test/api/ShareConsumerTest.java @@ -50,11 +50,10 @@ import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Flaky; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.coordinator.group.GroupConfig; @@ -62,7 +61,6 @@ import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Timeout; -import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.ArrayList; @@ -100,7 +98,6 @@ @Timeout(1200) @Tag("integration") -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults( serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala index 84214a79ed91f..cbd69baedc75c 100644 --- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala @@ -13,9 +13,7 @@ package kafka.api import kafka.log.UnifiedLog -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, ConsumerGroupDescription} import org.apache.kafka.clients.consumer.{Consumer, GroupProtocol, OffsetAndMetadata} @@ -26,17 +24,16 @@ import org.junit.jupiter.api.Assertions._ import scala.jdk.CollectionConverters._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record.CompressionType +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Timeout -import org.junit.jupiter.api.extension.ExtendWith import java.time.Duration import java.util.Collections import java.util.concurrent.TimeUnit @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) { @ClusterTest( diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala index 5fe06748631cf..e1fa03403b3e4 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala @@ -25,18 +25,17 @@ import org.apache.kafka.clients.producer.{Producer, ProducerConfig, ProducerReco import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.RecordTooLargeException import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, ClusterTests, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, ClusterTests, Type} import org.apache.kafka.common.message.InitProducerIdRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{InitProducerIdRequest, InitProducerIdResponse} -import org.apache.kafka.common.test.TestUtils +import org.apache.kafka.common.test.{ClusterInstance, TestUtils} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.server.common.{Feature, MetadataVersion} import org.junit.jupiter.api.Assertions.{assertEquals, assertInstanceOf, assertThrows, assertTrue} -import org.junit.jupiter.api.extension.ExtendWith import java.time.Duration import java.util @@ -53,7 +52,6 @@ import scala.jdk.CollectionConverters._ new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), )) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ProducerIntegrationTest { @ClusterTests(Array( diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 14169a5c9b9a0..7e0b10d1d6a63 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -20,7 +20,6 @@ package kafka.server import kafka.log.UnifiedLog import kafka.network.SocketServer import kafka.server.IntegrationTestUtils.connectAndReceive -import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin._ @@ -36,6 +35,7 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration.Op import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.common.requests.{ApiError, DescribeClusterRequest, DescribeClusterResponse} import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.{Cluster, Endpoint, Reconfigurable, TopicPartition, TopicPartitionInfo} import org.apache.kafka.controller.{QuorumController, QuorumControllerIntegrationTestUtils} import org.apache.kafka.image.ClusterImage diff --git a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala index 522e70732f4fa..a0637a5a4bf2a 100644 --- a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala @@ -17,19 +17,16 @@ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterTest, ClusterTests, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType import org.apache.kafka.clients.admin.{FeatureUpdate, UpdateFeaturesOptions} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.common.MetadataVersion import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class MetadataVersionIntegrationTest { @ClusterTests(value = Array( new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV0), diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala index ad47da549ff41..7196f6ed7eea5 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala @@ -17,9 +17,8 @@ package kafka.server -import org.apache.kafka.common.test.KafkaClusterTestKit -import org.apache.kafka.common.test.TestKitNodes import kafka.utils.TestUtils +import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.utils.BufferSupplier import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.server.config.KRaftConfigs diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 15fb0ac2a2f72..f71fc37bb2ff7 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -16,7 +16,6 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.clients.NodeApiVersions import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion @@ -24,6 +23,7 @@ import org.apache.kafka.common.message.ApiMessageType import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, RequestUtils} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.common.{EligibleLeaderReplicasVersion, GroupVersion, MetadataVersion, TransactionVersion} import org.apache.kafka.test.TestUtils diff --git a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala index 6c882d3877d83..d54e3227f80c4 100644 --- a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala @@ -18,17 +18,14 @@ package unit.kafka.server import kafka.network.SocketServer import kafka.server.{BrokerServer, ControllerServer, IntegrationTestUtils} -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.AllocateProducerIdsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.common.ProducerIdsBlock import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class AllocateProducerIdsRequestTest(cluster: ClusterInstance) { diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index eea2d7cc46b22..6e32cfc01f8be 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -17,17 +17,14 @@ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.message.ApiVersionsRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.ApiVersionsRequest +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.server.common.MetadataVersion import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala index 8ff20fedcabcd..4f48dedbf9c40 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala @@ -17,18 +17,15 @@ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.ClusterTest -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.AfterEach import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class BrokerMetricNamesTest(cluster: ClusterInstance) { @AfterEach def tearDown(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index 7d40f34fd9dd9..1bec9f7f4309c 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -17,7 +17,7 @@ package kafka.server -import org.apache.kafka.common.test.api.{ClusterInstance, ClusterTest, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterTest, Type} import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreateTopicsRequestData} @@ -26,11 +26,11 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, Feature, MetadataVersion, NodeToControllerChannelManager} import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import java.util import java.util.Collections @@ -39,7 +39,6 @@ import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException} /** * This test simulates a broker registering with the KRaft quorum under different configurations. */ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class BrokerRegistrationRequestTest { def brokerToControllerChannelManager(clusterInstance: ClusterInstance): NodeToControllerChannelManager = { diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index a56914c724859..cec9289e69140 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -20,23 +20,20 @@ package kafka.server import java.net.InetAddress import java.util import java.util.concurrent.{ExecutionException, TimeUnit} -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.ClusterTest -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{ScramCredentialInfo, ScramMechanism, UserScramCredentialUpsertion} import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException} import org.apache.kafka.common.internals.KafkaFutureImpl import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.common.requests.{AlterClientQuotasRequest, AlterClientQuotasResponse, DescribeClientQuotasRequest, DescribeClientQuotasResponse} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Disabled -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ClientQuotasRequestTest(cluster: ClusterInstance) { @ClusterTest def testAlterClientQuotasRequest(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala index 8753ceb78dc81..f6831ca8e3dd1 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala @@ -16,9 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api._ -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol @@ -28,18 +26,17 @@ import org.apache.kafka.common.message.{ConsumerGroupDescribeRequestData, Consum import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse} import org.apache.kafka.common.resource.ResourceType +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} -import org.junit.jupiter.api.extension.ExtendWith import java.lang.{Byte => JByte} import java.util.Collections import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1) class ConsumerGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala index 23b5589225dd7..6b42c4f566a94 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} @@ -25,15 +25,14 @@ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.message.{ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig} import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull} -import org.junit.jupiter.api.extension.ExtendWith import scala.collection.Map import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults( types = Array(Type.KRAFT), serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala index b5ef943df844e..0007f32714626 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala @@ -20,23 +20,20 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.message.{JoinGroupResponseData, ListGroupsResponseData, OffsetFetchResponseData, SyncGroupResponseData} -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.{Group, GroupCoordinatorConfig} import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Timeout -import org.junit.jupiter.api.extension.ExtendWith import java.nio.ByteBuffer import java.util.Collections import scala.jdk.CollectionConverters._ @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index b7e42c2399727..8a51652a7bbd8 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -21,7 +21,6 @@ import kafka.network.RequestChannel import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.KRaftMetadataCache -import org.apache.kafka.common.test.MockController import org.apache.kafka.clients.admin.AlterConfigOp import org.apache.kafka.common.Uuid.ZERO_UUID import org.apache.kafka.common.acl.AclOperation @@ -47,6 +46,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors} import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} +import org.apache.kafka.common.test.MockController import org.apache.kafka.common.utils.MockTime import org.apache.kafka.common.{ElectionType, Uuid} import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala index 0fab872363bd8..f9b9e9c946aa2 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala @@ -16,17 +16,14 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.junit.jupiter.api.Assertions.{assertEquals, fail} -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala index de8044ce2c113..4f1ba4b9b2ca3 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala @@ -16,19 +16,16 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.DescribeGroupsResponseData.{DescribedGroup, DescribedGroupMember} import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala index b564bfd739121..2aa8f5a9e2ca0 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala @@ -16,19 +16,16 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.DescribeQuorumRequest.singletonRequest import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, DescribeQuorumRequest, DescribeQuorumResponse} +import org.apache.kafka.common.test.ClusterInstance import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class DescribeQuorumRequestTest(cluster: ClusterInstance) { diff --git a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala index 916c244c2954d..eaec283ced757 100644 --- a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala @@ -17,7 +17,6 @@ package kafka.server import kafka.network.SocketServer -import org.apache.kafka.common.test.api.ClusterInstance import kafka.utils.TestUtils import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} @@ -29,6 +28,7 @@ import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsTo import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse} import org.apache.kafka.common.serialization.StringSerializer +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.ProducerIdAndEpoch import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.junit.jupiter.api.Assertions.{assertEquals, fail} diff --git a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala index 3d1a06e8b3906..332c01aeeb53b 100644 --- a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala @@ -16,23 +16,20 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.SyncGroupRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState -import org.junit.jupiter.api.extension.ExtendWith import java.util.Collections import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class HeartbeatRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala index 4117080a85a8a..f77c2fc1bfadd 100644 --- a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala @@ -16,19 +16,17 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember import org.apache.kafka.common.message.{JoinGroupResponseData, SyncGroupRequestData} import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import java.util.Collections import scala.concurrent.ExecutionContext.Implicits.global @@ -36,7 +34,6 @@ import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class JoinGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), diff --git a/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala index 4cc3f968d2769..4ff454d1d2de2 100644 --- a/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala @@ -18,20 +18,17 @@ package kafka.server import org.apache.kafka.common.Uuid import org.apache.kafka.common.message.LeaveGroupResponseData -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.JoinGroupRequest +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), serverProperties = Array( new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1") diff --git a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala index ea22f5c12167a..3961c725ed445 100644 --- a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala @@ -16,18 +16,15 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.ListGroupsResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState import org.apache.kafka.coordinator.group.{Group, GroupCoordinatorConfig} import org.junit.jupiter.api.Assertions.{assertEquals, fail} -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala index 6b0c5c6d0a5bb..577007123792b 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala @@ -16,15 +16,12 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.junit.jupiter.api.Assertions.fail -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { diff --git a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala index 7b0672916d5e2..0a808f6c868f4 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala @@ -16,15 +16,12 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.junit.jupiter.api.Assertions.fail -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class OffsetDeleteRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index 5bcc256d0c042..a504ecdeea0f8 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -16,20 +16,16 @@ */ package kafka.server - -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message.OffsetFetchResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.junit.jupiter.api.Assertions.{assertEquals, fail} -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 70791a4cef05f..3a8042e2590b3 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -19,16 +19,16 @@ package kafka.server import kafka.api.SaslSetup import kafka.security.JaasTestUtils import kafka.server.SaslApiVersionsRequestTest.{kafkaClientSaslMechanism, kafkaServerSaslMechanisms} -import org.apache.kafka.common.test.api.{ClusterTemplate, Type, ClusterTestExtensions, ClusterConfig, ClusterInstance} +import org.apache.kafka.common.test.api.{ClusterConfig, ClusterTemplate, Type} import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.message.SaslHandshakeRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, SaslHandshakeRequest, SaslHandshakeResponse} import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.network.SocketServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled} import java.net.Socket @@ -66,7 +66,6 @@ object SaslApiVersionsRequestTest { } @Disabled("TODO: KAFKA-17631 - Convert SaslApiVersionsRequestTest to kraft") -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class SaslApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { private var sasl: SaslSetup = _ diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index f60fc12f47b4b..843a2b4c3ba64 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -17,22 +17,21 @@ package kafka.server import kafka.utils.TestUtils -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, ClusterTests, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, ClusterTests, Type} import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords import org.apache.kafka.common.message.{ShareAcknowledgeRequestData, ShareAcknowledgeResponseData, ShareFetchRequestData, ShareFetchResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.requests.{ShareAcknowledgeRequest, ShareAcknowledgeResponse, ShareFetchRequest, ShareFetchResponse, ShareRequestMetadata} +import org.apache.kafka.common.test.ClusterInstance import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, Tag, Timeout} -import org.junit.jupiter.api.extension.ExtendWith import java.util import java.util.Collections import scala.jdk.CollectionConverters._ @Timeout(1200) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( new ClusterConfigProperty(key = "group.share.persister.class.name", value = "") )) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala index a6a1129d084bf..cf297198e39e2 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala @@ -16,9 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api._ -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.common.GroupState import org.apache.kafka.common.message.ShareGroupDescribeResponseData.DescribedGroup @@ -26,6 +24,7 @@ import org.apache.kafka.common.message.{ShareGroupDescribeRequestData, ShareGrou import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ShareGroupDescribeRequest, ShareGroupDescribeResponse} import org.apache.kafka.common.resource.ResourceType +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig @@ -33,13 +32,11 @@ import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{Tag, Timeout} -import org.junit.jupiter.api.extension.ExtendWith import java.lang.{Byte => JByte} import scala.jdk.CollectionConverters._ @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( new ClusterConfigProperty(key = ShareGroupConfig.SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG, value = "") )) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index c471b0f279372..07c7b959ab81c 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import kafka.utils.TestUtils.waitForAllPartitionsMetadata import org.apache.kafka.clients.admin.{Admin, NewPartitions} @@ -24,14 +24,13 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.message.{ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse} +import org.apache.kafka.common.test.ClusterInstance import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNotNull, assertNull, assertTrue} import org.junit.jupiter.api.{Tag, Timeout} -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( new ClusterConfigProperty(key = "group.share.persister.class.name", value = "") )) diff --git a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala index a52947fdfe4a8..3a53fbf144aba 100644 --- a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala @@ -16,24 +16,21 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.SyncGroupRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState -import org.junit.jupiter.api.extension.ExtendWith import java.util.Collections import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class SyncGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala index 57277f3dbed49..b2cd44bbd9222 100644 --- a/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala @@ -16,21 +16,20 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.JoinGroupRequest +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.ProducerIdAndEpoch import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.junit.jupiter.api.Assertions.{assertThrows, assertTrue, fail} -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters.IterableHasAsScala -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), serverProperties = Array( new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index 08b6bbe7f21d2..df383b25bf6ca 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -24,12 +24,12 @@ import java.util.Properties import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import kafka.log.LogManager import kafka.server.{BrokerServer, KafkaConfig, ReplicaManager} -import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.BROKER +import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.utils.Exit import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator diff --git a/settings.gradle b/settings.gradle index e640ba906256f..2414593c2e58d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -110,8 +110,8 @@ include 'clients', 'tools:tools-api', 'transaction-coordinator', 'trogdor', - 'test-common', - 'test-common:test-common-api', + 'test-common:test-common-internal-api', + 'test-common:test-common-util', 'test-common:test-common-runtime' project(":storage:api").name = "storage-api" diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java index 45838959eddd2..3c7a2a9397f92 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; @@ -33,8 +32,6 @@ import org.apache.kafka.storage.internals.log.EpochEntry; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -54,7 +51,6 @@ import static org.mockito.Mockito.verify; @ClusterTestDefaults(brokers = 3) -@ExtendWith(value = ClusterTestExtensions.class) public class RemoteLogSegmentLifecycleTest { private final int segSize = 1048576; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java index 6016491a09524..23af9c50fc331 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java @@ -23,17 +23,14 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; import org.apache.kafka.server.log.remote.storage.RemoteStorageException; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -52,7 +49,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3) public class TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest { private final ClusterInstance clusterInstance; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java index 0026f64581702..3c0f60650e441 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java @@ -21,23 +21,19 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class TopicBasedRemoteLogMetadataManagerRestartTest { private final Time time = Time.SYSTEM; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java index 20d3d78e37f7e..e36e64560f2a0 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java @@ -21,10 +21,9 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; @@ -32,7 +31,6 @@ import org.apache.kafka.server.log.remote.storage.RemoteStorageException; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; import java.util.Arrays; @@ -50,7 +48,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3) public class TopicBasedRemoteLogMetadataManagerTest { private static final int SEG_SIZE = 1048576; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java index 17f80b63af1f3..840417b5b3e3b 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java @@ -19,17 +19,14 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataManagerTestUtils; import org.apache.kafka.server.log.remote.metadata.storage.RemotePartitionMetadataStore; import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -44,7 +41,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3) public class RemoteLogMetadataManagerTest { private final ClusterInstance clusterInstance; diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java similarity index 97% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java index 6708297c85b9f..e1db72f1106bb 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java @@ -33,10 +33,10 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_SECURITY_PROTOCOL; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; /** * Represents an immutable requested configuration of a Kafka cluster for integration testing. diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java similarity index 92% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java index 86aba1030d878..5d4a36f834904 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java @@ -30,8 +30,8 @@ import static java.lang.annotation.ElementType.METHOD; import static java.lang.annotation.RetentionPolicy.RUNTIME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME; @Documented @Target({METHOD}) diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/README.md b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/README.md similarity index 98% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/README.md rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/README.md index 7a3ea14dc6663..10a7b6dcf9a7f 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/README.md +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/README.md @@ -79,7 +79,7 @@ a JUnit extension called `ClusterTestExtensions` which knows how to process thes invocations. Test classes that wish to make use of these annotations need to explicitly register this extension: ```scala -import org.apache.kafka.common.test.api.ClusterTestExtensions +import org.apache.kafka.common.test.junit.ClusterTestExtensions @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ApiVersionsRequestTest { diff --git a/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/TestKitDefaults.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/TestKitDefaults.java new file mode 100644 index 0000000000000..3374458222970 --- /dev/null +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/TestKitDefaults.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.test.api; + +import org.apache.kafka.common.security.auth.SecurityProtocol; + +/** + * Constants used by TestKitNodes and ClusterTest annotation defaults + */ +public class TestKitDefaults { + public static final int CONTROLLER_ID_OFFSET = 3000; + public static final int BROKER_ID_OFFSET = 0; + public static final SecurityProtocol DEFAULT_BROKER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; + public static final String DEFAULT_BROKER_LISTENER_NAME = "EXTERNAL"; + public static final SecurityProtocol DEFAULT_CONTROLLER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; + public static final String DEFAULT_CONTROLLER_LISTENER_NAME = "CONTROLLER"; + + private TestKitDefaults() { + + } +} diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Type.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/Type.java similarity index 59% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Type.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/Type.java index 807d4f3354a7a..9e028acf18673 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Type.java +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/Type.java @@ -16,25 +16,9 @@ */ package org.apache.kafka.common.test.api; -import org.junit.jupiter.api.extension.TestTemplateInvocationContext; - - /** * The type of cluster config being requested. Used by {@link org.apache.kafka.common.test.api.ClusterConfig} and the test annotations. */ public enum Type { - KRAFT { - @Override - public TestTemplateInvocationContext invocationContexts(String baseDisplayName, ClusterConfig config) { - return new RaftClusterInvocationContext(baseDisplayName, config, false); - } - }, - CO_KRAFT { - @Override - public TestTemplateInvocationContext invocationContexts(String baseDisplayName, ClusterConfig config) { - return new RaftClusterInvocationContext(baseDisplayName, config, true); - } - }; - - public abstract TestTemplateInvocationContext invocationContexts(String baseDisplayName, ClusterConfig config); + KRAFT, CO_KRAFT; } diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java similarity index 91% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java rename to test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java index 555514774793a..0eeb8ede394de 100644 --- a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java +++ b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.test.TestUtils; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Assertions; @@ -28,6 +27,7 @@ import java.io.File; import java.io.IOException; import java.lang.reflect.Field; +import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -35,10 +35,10 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_SECURITY_PROTOCOL; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; public class ClusterConfigTest { @@ -51,7 +51,8 @@ private static Map fields(ClusterConfig config) { @Test public void testCopy() throws IOException { - File trustStoreFile = TestUtils.tempFile(); + File trustStoreFile = Files.createTempFile("kafka", ".tmp").toFile(); + trustStoreFile.deleteOnExit(); ClusterConfig clusterConfig = ClusterConfig.builder() .setTypes(Collections.singleton(Type.KRAFT)) diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java similarity index 100% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java rename to test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java diff --git a/test-common/src/main/resources/log4j2.yaml b/test-common/test-common-internal-api/src/test/resources/log4j2.yaml similarity index 100% rename from test-common/src/main/resources/log4j2.yaml rename to test-common/test-common-internal-api/src/test/resources/log4j2.yaml diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java similarity index 98% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java index eb97c2c92a164..2bedb966adb24 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test; import kafka.log.UnifiedLog; import kafka.network.SocketServer; @@ -45,8 +45,9 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.test.JaasUtils; -import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.fault.FaultHandlerException; import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile; diff --git a/test-common/src/main/java/org/apache/kafka/common/test/JaasModule.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasModule.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/JaasModule.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasModule.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/JaasUtils.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasUtils.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/JaasUtils.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasUtils.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/MockController.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockController.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/MockController.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockController.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/TestKitNode.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNode.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/TestKitNode.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNode.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/TestKitNodes.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java similarity index 94% rename from test-common/src/main/java/org/apache/kafka/common/test/TestKitNodes.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java index b6d6f9f69fc6c..c78462c213a23 100644 --- a/test-common/src/main/java/org/apache/kafka/common/test/TestKitNodes.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.test.api.TestKitDefaults; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.properties.MetaProperties; import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble; @@ -43,13 +44,6 @@ @SuppressWarnings("NPathComplexity") public class TestKitNodes { - public static final int CONTROLLER_ID_OFFSET = 3000; - public static final int BROKER_ID_OFFSET = 0; - public static final SecurityProtocol DEFAULT_BROKER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; - public static final String DEFAULT_BROKER_LISTENER_NAME = "EXTERNAL"; - public static final SecurityProtocol DEFAULT_CONTROLLER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; - public static final String DEFAULT_CONTROLLER_LISTENER_NAME = "CONTROLLER"; - public static class Builder { private boolean combined; private String clusterId; @@ -69,10 +63,10 @@ public Builder(BootstrapMetadata bootstrapMetadata) { } // The broker and controller listener name and SecurityProtocol configurations must // be kept in sync with the default values in ClusterTest. - private ListenerName brokerListenerName = ListenerName.normalised(DEFAULT_BROKER_LISTENER_NAME); - private SecurityProtocol brokerSecurityProtocol = DEFAULT_BROKER_SECURITY_PROTOCOL; - private ListenerName controllerListenerName = ListenerName.normalised(DEFAULT_CONTROLLER_LISTENER_NAME); - private SecurityProtocol controllerSecurityProtocol = DEFAULT_CONTROLLER_SECURITY_PROTOCOL; + private ListenerName brokerListenerName = ListenerName.normalised(TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME); + private SecurityProtocol brokerSecurityProtocol = TestKitDefaults.DEFAULT_BROKER_SECURITY_PROTOCOL; + private ListenerName controllerListenerName = ListenerName.normalised(TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME); + private SecurityProtocol controllerSecurityProtocol = TestKitDefaults.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; public Builder setClusterId(String clusterId) { this.clusterId = clusterId; @@ -169,11 +163,11 @@ public TestKitNodes build() { clusterId = Uuid.randomUuid().toString(); } - int controllerId = combined ? BROKER_ID_OFFSET : BROKER_ID_OFFSET + CONTROLLER_ID_OFFSET; + int controllerId = combined ? TestKitDefaults.BROKER_ID_OFFSET : TestKitDefaults.BROKER_ID_OFFSET + TestKitDefaults.CONTROLLER_ID_OFFSET; List controllerNodeIds = IntStream.range(controllerId, controllerId + numControllerNodes) .boxed() .collect(Collectors.toList()); - List brokerNodeIds = IntStream.range(BROKER_ID_OFFSET, BROKER_ID_OFFSET + numBrokerNodes) + List brokerNodeIds = IntStream.range(TestKitDefaults.BROKER_ID_OFFSET, TestKitDefaults.BROKER_ID_OFFSET + numBrokerNodes) .boxed() .collect(Collectors.toList()); diff --git a/test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestUtils.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestUtils.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstanceParameterResolver.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterInstanceParameterResolver.java similarity index 92% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstanceParameterResolver.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterInstanceParameterResolver.java index 5582eb379ba1e..5276ff1ad1ab5 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstanceParameterResolver.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterInstanceParameterResolver.java @@ -15,17 +15,19 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; +import org.apache.kafka.common.test.ClusterInstance; + import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.ParameterContext; import org.junit.jupiter.api.extension.ParameterResolver; +import org.junit.platform.commons.util.AnnotationUtils; import java.lang.reflect.Executable; -import static org.junit.platform.commons.util.AnnotationUtils.isAnnotated; /** * This resolver provides an instance of {@link ClusterInstance} to a test invocation. The instance represents the @@ -57,7 +59,7 @@ public boolean supportsParameter(ParameterContext parameterContext, ExtensionCon } else { // If we're injecting into a method, make sure it's a test method and not a lifecycle method Executable parameterizedMethod = parameterContext.getParameter().getDeclaringExecutable(); - return isAnnotated(parameterizedMethod, TestTemplate.class); + return AnnotationUtils.isAnnotated(parameterizedMethod, TestTemplate.class); } } diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestExtensions.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java similarity index 84% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestExtensions.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java index 127f8a4b7bedf..1f39c01b29dbf 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestExtensions.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java @@ -14,9 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.test.api.AutoStart; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterFeature; +import org.apache.kafka.common.test.api.ClusterTemplate; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.ClusterTests; +import org.apache.kafka.common.test.api.DetectThreadLeak; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.util.timer.SystemTimer; @@ -110,6 +120,13 @@ public boolean supportsTestTemplate(ExtensionContext context) { return true; } + private boolean isClusterTest(ExtensionContext context) { + Method method = context.getRequiredTestMethod(); + return method.getDeclaredAnnotation(ClusterTemplate.class) != null || + method.getDeclaredAnnotation(ClusterTest.class) != null || + method.getDeclaredAnnotation(ClusterTests.class) != null; + } + @Override public Stream provideTestTemplateInvocationContexts(ExtensionContext context) { ClusterTestDefaults defaults = getClusterTestDefaults(context.getRequiredTestClass()); @@ -133,30 +150,29 @@ public Stream provideTestTemplateInvocationContex generatedContexts.addAll(processClusterTests(context, clusterTestsAnnot.value(), defaults)); } - if (generatedContexts.isEmpty()) { - throw new IllegalStateException("Please annotate test methods with @ClusterTemplate, @ClusterTest, or " + - "@ClusterTests when using the ClusterTestExtensions provider"); - } - return generatedContexts.stream(); } @Override public void beforeEach(ExtensionContext context) { - DetectThreadLeak detectThreadLeak = DetectThreadLeak.of(thread -> + if (isClusterTest(context)) { + DetectThreadLeak detectThreadLeak = DetectThreadLeak.of(thread -> SKIPPED_THREAD_PREFIX.stream().noneMatch(prefix -> thread.getName().startsWith(prefix))); - getStore(context).put(DETECT_THREAD_LEAK_KEY, detectThreadLeak); + getStore(context).put(DETECT_THREAD_LEAK_KEY, detectThreadLeak); + } } @Override public void afterEach(ExtensionContext context) { - DetectThreadLeak detectThreadLeak = getStore(context).remove(DETECT_THREAD_LEAK_KEY, DetectThreadLeak.class); - if (detectThreadLeak == null) { - return; - } - List threads = detectThreadLeak.newThreads(); - assertTrue(threads.isEmpty(), "Thread leak detected: " + + if (isClusterTest(context)) { + DetectThreadLeak detectThreadLeak = getStore(context).remove(DETECT_THREAD_LEAK_KEY, DetectThreadLeak.class); + if (detectThreadLeak == null) { + return; + } + List threads = detectThreadLeak.newThreads(); + assertTrue(threads.isEmpty(), "Thread leak detected: " + threads.stream().map(Thread::getName).collect(Collectors.joining(", "))); + } } private Store getStore(ExtensionContext context) { @@ -174,6 +190,21 @@ private int getTestRepeatCount() { return count; } + private TestTemplateInvocationContext invocationContextForClusterType( + Type type, + String baseDisplayName, + ClusterConfig config + ) { + switch (type) { + case KRAFT: + return new RaftClusterInvocationContext(baseDisplayName, config, false); + case CO_KRAFT: + return new RaftClusterInvocationContext(baseDisplayName, config, true); + default: + throw new IllegalArgumentException("Unsupported @Type value " + type); + } + } + List processClusterTemplate(ExtensionContext context, ClusterTemplate annot) { if (annot.value().trim().isEmpty()) { throw new IllegalStateException("ClusterTemplate value can't be empty string."); @@ -184,7 +215,7 @@ List processClusterTemplate(ExtensionContext cont List contexts = IntStream.range(0, repeatCount) .mapToObj(__ -> generateClusterConfigurations(context, annot.value()).stream()) .flatMap(Function.identity()) - .flatMap(config -> config.clusterTypes().stream().map(type -> type.invocationContexts(baseDisplayName, config))) + .flatMap(config -> config.clusterTypes().stream().map(type -> invocationContextForClusterType(type, baseDisplayName, config))) .collect(Collectors.toList()); if (contexts.isEmpty()) { @@ -259,7 +290,7 @@ private List processClusterTestInternal( .build(); return Arrays.stream(types) - .map(type -> type.invocationContexts(context.getRequiredTestMethod().getName(), config)) + .map(type -> invocationContextForClusterType(type, context.getRequiredTestMethod().getName(), config)) .collect(Collectors.toList()); } diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/RaftClusterInvocationContext.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/RaftClusterInvocationContext.java similarity index 98% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/RaftClusterInvocationContext.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/RaftClusterInvocationContext.java index 22a009b394e6e..76bb24a32b46c 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/RaftClusterInvocationContext.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/RaftClusterInvocationContext.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; import kafka.network.SocketServer; import kafka.server.BrokerServer; @@ -22,9 +22,12 @@ import kafka.server.KafkaBroker; import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.KafkaClusterTestKit; import org.apache.kafka.common.test.TestKitNodes; import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.metadata.BrokerState; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; diff --git a/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.jupiter.api.extension.Extension new file mode 100644 index 0000000000000..d7f05be7df0d4 --- /dev/null +++ b/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.jupiter.api.extension.Extension @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.kafka.common.test.junit.ClusterTestExtensions \ No newline at end of file diff --git a/test-common/test-common-api/src/test/resources/log4j2.yaml b/test-common/test-common-runtime/src/main/resources/log4j2.yaml similarity index 100% rename from test-common/test-common-api/src/test/resources/log4j2.yaml rename to test-common/test-common-runtime/src/main/resources/log4j2.yaml diff --git a/test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java similarity index 100% rename from test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java diff --git a/test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java similarity index 100% rename from test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java diff --git a/test-common/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java similarity index 100% rename from test-common/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsTest.java similarity index 97% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsTest.java index 6020364927400..27fd4a3e0a61f 100644 --- a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsTest.java +++ b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; import kafka.server.ControllerServer; @@ -40,14 +40,22 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.JaasUtils; import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.AutoStart; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTemplate; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.ClusterTests; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.extension.ExtendWith; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -84,7 +92,6 @@ @ClusterConfigProperty(key = "default.key", value = "default.value"), @ClusterConfigProperty(id = 0, key = "queued.max.requests", value = "100"), }) // Set defaults for a few params in @ClusterTest(s) -@ExtendWith(ClusterTestExtensions.class) public class ClusterTestExtensionsTest { private final ClusterInstance clusterInstance; @@ -371,7 +378,7 @@ public void testControllerRestart(ClusterInstance cluster) throws ExecutionExcep } ) public void testSaslPlaintext(ClusterInstance clusterInstance) throws CancellationException, ExecutionException, InterruptedException { - Assertions.assertEquals(SecurityProtocol.SASL_PLAINTEXT, clusterInstance.config().brokerSecurityProtocol()); + assertEquals(SecurityProtocol.SASL_PLAINTEXT, clusterInstance.config().brokerSecurityProtocol()); // default ClusterInstance#admin helper with admin credentials try (Admin admin = clusterInstance.admin()) { diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsUnitTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsUnitTest.java similarity index 95% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsUnitTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsUnitTest.java index bc21379eaa914..819006612c46f 100644 --- a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsUnitTest.java +++ b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsUnitTest.java @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; + +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterTemplate; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Flaky.java b/test-common/test-common-util/src/main/java/org/apache/kafka/common/test/api/Flaky.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Flaky.java rename to test-common/test-common-util/src/main/java/org/apache/kafka/common/test/api/Flaky.java diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java b/test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java similarity index 100% rename from test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java rename to test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java b/test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java similarity index 100% rename from test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java rename to test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java diff --git a/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter b/test-common/test-common-util/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter similarity index 100% rename from test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter rename to test-common/test-common-util/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter diff --git a/test-common/test-common-runtime/src/main/resources/junit-platform.properties b/test-common/test-common-util/src/main/resources/junit-platform.properties similarity index 100% rename from test-common/test-common-runtime/src/main/resources/junit-platform.properties rename to test-common/test-common-util/src/main/resources/junit-platform.properties diff --git a/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java b/test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java similarity index 100% rename from test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java rename to test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java diff --git a/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java b/test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java similarity index 100% rename from test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java rename to test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java diff --git a/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java index 12f4dab8801c6..090975e6f0771 100644 --- a/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java @@ -25,11 +25,10 @@ import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Exit; @@ -40,7 +39,6 @@ import org.apache.logging.log4j.Level; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.File; import java.io.IOException; @@ -91,7 +89,6 @@ @ClusterConfigProperty(key = AUTHORIZER_CLASS_NAME_CONFIG, value = AclCommandTest.STANDARD_AUTHORIZER)} ) -@ExtendWith(ClusterTestExtensions.class) public class AclCommandTest { public static final String STANDARD_AUTHORIZER = "org.apache.kafka.metadata.authorizer.StandardAuthorizer"; private static final String LOCALHOST = "localhost:9092"; diff --git a/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java index 3f20bb4e50201..49534a0ca8245 100644 --- a/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java @@ -24,15 +24,12 @@ import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.ApiVersionsResponse; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.server.config.ServerConfigs; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import java.util.Iterator; @@ -43,7 +40,6 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(serverProperties = { @ClusterConfigProperty(key = ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, value = "true"), }) diff --git a/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java b/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java index b21795411e269..8e7a77c292583 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java @@ -19,13 +19,11 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.common.errors.UnsupportedEndpointTypeException; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.ByteArrayOutputStream; import java.io.PrintStream; @@ -41,7 +39,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class ClusterToolTest { @ClusterTest diff --git a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java index 0dee52602164d..f798ed702a6d1 100644 --- a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java @@ -24,16 +24,14 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.server.common.AdminCommandFailedException; import org.apache.kafka.server.common.AdminOperationException; import com.fasterxml.jackson.core.JsonProcessingException; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; import java.nio.file.NoSuchFileException; @@ -48,7 +46,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class DeleteRecordsCommandTest { @ClusterTest diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index d923e46cf3138..969a081917017 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -17,9 +17,8 @@ package org.apache.kafka.tools; import org.apache.kafka.clients.admin.MockAdminClient; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.MetadataVersion; @@ -27,7 +26,6 @@ import net.sourceforge.argparse4j.inf.Namespace; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collections; @@ -46,7 +44,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class FeatureCommandTest { private final List testingFeatures = Arrays.stream(Feature.FEATURES).collect(Collectors.toList()); diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 1f030b9426d9a..5bb23cabdd98a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -28,13 +28,12 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.server.config.ServerLogConfigs; @@ -43,8 +42,6 @@ import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -65,7 +62,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) @ClusterTestDefaults(serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), diff --git a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java index 0bc71a2b813aa..767891579778a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java @@ -32,10 +32,9 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.test.TestUtils; @@ -44,7 +43,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.List; @@ -63,7 +61,6 @@ import static org.mockito.Mockito.when; @Timeout(value = 60) -@ExtendWith(ClusterTestExtensions.class) public class GroupsCommandTest { private final String bootstrapServer = "localhost:9092"; diff --git a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java index 42a25a69e7f30..23bd9a5c19ece 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java @@ -24,14 +24,12 @@ import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.server.common.AdminCommandFailedException; -import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; import org.mockito.MockedStatic; import org.mockito.Mockito; @@ -60,7 +58,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3, serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), @ClusterConfigProperty(key = "auto.leader.rebalance.enable", value = "false"), diff --git a/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java index a6ac8a199597e..8407c1fe9ffe8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java @@ -23,15 +23,13 @@ import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collections; @@ -49,7 +47,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class LogDirsCommandTest { private static final String TOPIC = "test-log-dirs-topic"; diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java index 484f09ec5ef00..13b05e306491c 100644 --- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java @@ -17,15 +17,13 @@ package org.apache.kafka.tools; import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.ClusterTests; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.File; import java.io.IOException; @@ -38,7 +36,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) class MetadataQuorumCommandTest { /** diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java index bde7f3f2532d5..1fb2a87f8dbc0 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java @@ -48,12 +48,11 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.metadata.LeaderAndIsr; @@ -63,7 +62,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.ArrayList; import java.util.Arrays; @@ -99,7 +97,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@ExtendWith(ClusterTestExtensions.class) public class TopicCommandTest { private final short defaultReplicationFactor = 1; private final int defaultNumPartitions = 1; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java index fa3f401784389..eef6ac4cb4937 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java @@ -34,9 +34,8 @@ import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; import org.apache.kafka.coordinator.group.generated.GroupMetadataKeyJsonConverter; @@ -57,7 +56,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -96,7 +94,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@ExtendWith(ClusterTestExtensions.class) public class ConsoleConsumerTest { private final String topic = "test-topic"; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java index 271b496873f58..7d552dcbe483e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java @@ -25,15 +25,13 @@ import org.apache.kafka.common.errors.GroupNotEmptyException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.test.TestUtils; import org.apache.kafka.tools.ToolsTestUtils; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.HashMap; @@ -66,8 +64,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; - -@ExtendWith(value = ClusterTestExtensions.class) public class DeleteConsumerGroupsTest { private static List generator() { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java index b71e90474fc17..133c9bfb53a1e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java @@ -31,14 +31,12 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.Collections; @@ -52,7 +50,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; -@ExtendWith(ClusterTestExtensions.class) public class DeleteOffsetsConsumerGroupCommandIntegrationTest { public static final String TOPIC_PREFIX = "foo."; public static final String GROUP_PREFIX = "test.group."; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java index 8646858902b2f..b30c3674514b8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java @@ -35,10 +35,9 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.test.TestUtils; @@ -46,7 +45,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.ArrayList; import java.util.Arrays; @@ -75,7 +73,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; -@ExtendWith(value = ClusterTestExtensions.class) public class DescribeConsumerGroupTest { private static final String TOPIC_PREFIX = "test.topic."; private static final String GROUP_PREFIX = "test.group."; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java index b16bf8fc0bf03..16db6fb93bc33 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java @@ -28,16 +28,14 @@ import org.apache.kafka.common.GroupType; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.test.TestUtils; import org.apache.kafka.tools.ToolsTestUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.ArrayList; import java.util.Arrays; @@ -65,7 +63,6 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; -@ExtendWith(ClusterTestExtensions.class) public class ListConsumerGroupTest { private static final String TOPIC_PREFIX = "test.topic."; private static final String TOPIC_PARTITIONS_GROUP_PREFIX = "test.topic.partitions.group."; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java index a49597d638f9a..bcc06301d6351 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java @@ -29,14 +29,11 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.io.BufferedWriter; import java.io.File; import java.io.FileWriter; @@ -92,7 +89,6 @@ * - scope=topics+partitions, scenario=to-earliest * - export/import */ -@ExtendWith(value = ClusterTestExtensions.class) public class ResetConsumerGroupOffsetTest { private static final String TOPIC_PREFIX = "foo-"; diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java index ae174d63c6ae5..1430197216da8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java @@ -40,11 +40,10 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.ClusterTests; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Time; @@ -55,7 +54,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.AbstractMap; @@ -112,7 +110,6 @@ @ClusterConfigProperty(id = 3, key = "broker.rack", value = "rack1"), @ClusterConfigProperty(id = 4, key = "broker.rack", value = "rack1"), }) -@ExtendWith(ClusterTestExtensions.class) public class ReassignPartitionsCommandTest { private final ClusterInstance clusterInstance; private final Map> unthrottledBrokerConfigs = IntStream