diff --git a/build.gradle b/build.gradle index c07516a4d2c73..7352ae58f37c0 100644 --- a/build.gradle +++ b/build.gradle @@ -995,9 +995,13 @@ project(':share') { dependencies { implementation project(':server-common') + testImplementation project(':clients').sourceSets.test.output + testImplementation project(':server-common').sourceSets.test.output + implementation libs.slf4jApi testImplementation libs.junitJupiter + testImplementation libs.mockitoCore testImplementation libs.slf4jReload4j testRuntimeOnly libs.junitPlatformLanucher diff --git a/checkstyle/import-control-share.xml b/checkstyle/import-control-share.xml index 7b60f032dbe1c..89422b705456f 100644 --- a/checkstyle/import-control-share.xml +++ b/checkstyle/import-control-share.xml @@ -26,6 +26,7 @@ + @@ -47,6 +48,13 @@ + + + + + + + diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index ff483f105842e..6b90d419aab7d 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -23,7 +23,7 @@ import kafka.log.LogManager import kafka.log.remote.RemoteLogManager import kafka.network.{DataPlaneAcceptor, SocketServer} import kafka.raft.KafkaRaftManager -import kafka.server.metadata.{AclPublisher, BrokerMetadataPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, ScramPublisher} +import kafka.server.metadata.{AclPublisher, BrokerMetadataPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, ScramPublisher, ShareCoordinatorMetadataCacheHelperImpl} import kafka.server.share.SharePartitionManager import kafka.utils.CoreUtils import org.apache.kafka.common.config.ConfigException @@ -46,7 +46,7 @@ import org.apache.kafka.server.{AssignmentsManager, BrokerFeatures, ClientMetric import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{ApiMessageAndVersion, DirectoryEventHandler, TopicIdPartition} import org.apache.kafka.server.config.ConfigType -import org.apache.kafka.server.share.persister.{NoOpShareStatePersister, Persister} +import org.apache.kafka.server.share.persister.{DefaultStatePersister, NoOpShareStatePersister, Persister, PersisterStateManager} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, KafkaYammerMetrics} import org.apache.kafka.server.network.{EndpointReadyFutures, KafkaAuthorizerServerInfo} @@ -124,7 +124,7 @@ class BrokerServer( var transactionCoordinator: TransactionCoordinator = _ - var shareCoordinator: Option[ShareCoordinator] = _ + var shareCoordinator: Option[ShareCoordinator] = None var clientToControllerChannelManager: NodeToControllerChannelManager = _ @@ -356,8 +356,12 @@ class BrokerServer( /* initializing the groupConfigManager */ groupConfigManager = new GroupConfigManager(config.groupCoordinatorConfig.extractGroupConfigMap(config.shareGroupConfig)) + /* create share coordinator */ shareCoordinator = createShareCoordinator() + /* create persister */ + persister = createShareStatePersister() + groupCoordinator = createGroupCoordinator() val producerIdManagerSupplier = () => ProducerIdManager.rpc( @@ -423,8 +427,6 @@ class BrokerServer( config.shareGroupConfig.shareGroupMaxGroups * config.groupCoordinatorConfig.shareGroupMaxSize, KafkaServer.MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS) - persister = NoOpShareStatePersister.getInstance() - sharePartitionManager = new SharePartitionManager( replicaManager, time, @@ -435,7 +437,7 @@ class BrokerServer( config.shareGroupConfig.shareFetchMaxFetchRecords, persister, groupConfigManager, - new Metrics() + metrics ) // Create the request processor objects. @@ -646,33 +648,68 @@ class BrokerServer( } private def createShareCoordinator(): Option[ShareCoordinator] = { - if (!config.shareGroupConfig.isShareGroupEnabled) { - return None + if (config.shareGroupConfig.isShareGroupEnabled && + config.shareGroupConfig.shareGroupPersisterClassName().nonEmpty) { + val time = Time.SYSTEM + val timer = new SystemTimerReaper( + "share-coordinator-reaper", + new SystemTimer("share-coordinator") + ) + + val serde = new ShareCoordinatorRecordSerde + val loader = new CoordinatorLoaderImpl[CoordinatorRecord]( + time, + replicaManager, + serde, + config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() + ) + val writer = new CoordinatorPartitionWriter( + replicaManager + ) + Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig) + .withTimer(timer) + .withTime(time) + .withLoader(loader) + .withWriter(writer) + .withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics)) + .withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics)) + .build()) + } else { + None + } + } + + private def createShareStatePersister(): Persister = { + if (config.shareGroupConfig.isShareGroupEnabled && + config.shareGroupConfig.shareGroupPersisterClassName.nonEmpty) { + val klass = Utils.loadClass(config.shareGroupConfig.shareGroupPersisterClassName, classOf[Object]).asInstanceOf[Class[Persister]] + + if (klass.getName.equals(classOf[DefaultStatePersister].getName)) { + klass.getConstructor(classOf[PersisterStateManager]) + .newInstance( + new PersisterStateManager( + NetworkUtils.buildNetworkClient("Persister", config, metrics, Time.SYSTEM, new LogContext(s"[Persister broker=${config.brokerId}]")), + new ShareCoordinatorMetadataCacheHelperImpl(metadataCache, key => shareCoordinator.get.partitionFor(key), config.interBrokerListenerName), + Time.SYSTEM, + new SystemTimerReaper( + "persister-state-manager-reaper", + new SystemTimer("persister") + ) + ) + ) + } else if (klass.getName.equals(classOf[NoOpShareStatePersister].getName)) { + info("Using no op persister") + new NoOpShareStatePersister() + } else { + error("Unknown persister specified. Persister is only factory pluggable!") + throw new IllegalArgumentException("Unknown persiser specified " + config.shareGroupConfig.shareGroupPersisterClassName) + } + } else { + // in case share coordinator not enabled or + // persister class name deliberately empty (key=) + info("Using no op persister") + new NoOpShareStatePersister() } - val time = Time.SYSTEM - val timer = new SystemTimerReaper( - "share-coordinator-reaper", - new SystemTimer("share-coordinator") - ) - - val serde = new ShareCoordinatorRecordSerde - val loader = new CoordinatorLoaderImpl[CoordinatorRecord]( - time, - replicaManager, - serde, - config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize() - ) - val writer = new CoordinatorPartitionWriter( - replicaManager - ) - Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig) - .withTimer(timer) - .withTime(time) - .withLoader(loader) - .withWriter(writer) - .withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics)) - .withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics)) - .build()) } protected def createRemoteLogManager(): Option[RemoteLogManager] = { @@ -778,9 +815,13 @@ class BrokerServer( if (socketServer != null) CoreUtils.swallow(socketServer.shutdown(), this) + Utils.closeQuietly(brokerTopicStats, "broker topic stats") Utils.closeQuietly(sharePartitionManager, "share partition manager") + if (persister != null) + CoreUtils.swallow(persister.stop(), this) + isShuttingDown.set(false) CoreUtils.swallow(lifecycleManager.close(), this) diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index 29a002b40b0b2..957677c2b4181 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -216,6 +216,14 @@ class BrokerMetadataPublisher( s"coordinator with local changes in $deltaName", t) } + try { + // Propagate the new image to the share coordinator. + shareCoordinator.foreach(coordinator => coordinator.onNewMetadataImage(newImage, delta)) + } catch { + case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " + + s"coordinator with local changes in $deltaName", t) + } + if (_firstPublish) { finishInitializingReplicaManager() } diff --git a/core/src/main/scala/kafka/server/metadata/ShareCoordinatorMetadataCacheHelperImpl.java b/core/src/main/scala/kafka/server/metadata/ShareCoordinatorMetadataCacheHelperImpl.java new file mode 100644 index 0000000000000..caa7b348f5ebb --- /dev/null +++ b/core/src/main/scala/kafka/server/metadata/ShareCoordinatorMetadataCacheHelperImpl.java @@ -0,0 +1,102 @@ +/* + * 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.server.metadata; + +import kafka.server.MetadataCache; + +import org.apache.kafka.common.Node; +import org.apache.kafka.common.message.MetadataResponseData; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.server.share.persister.ShareCoordinatorMetadataCacheHelper; + +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; + +import scala.jdk.javaapi.CollectionConverters; +import scala.jdk.javaapi.OptionConverters; + +public class ShareCoordinatorMetadataCacheHelperImpl implements ShareCoordinatorMetadataCacheHelper { + private final MetadataCache metadataCache; + private final Function keyToPartitionMapper; + private final ListenerName interBrokerListenerName; + + public ShareCoordinatorMetadataCacheHelperImpl( + MetadataCache metadataCache, + Function keyToPartitionMapper, + ListenerName interBrokerListenerName + ) { + Objects.requireNonNull(metadataCache, "metadataCache must not be null"); + Objects.requireNonNull(keyToPartitionMapper, "keyToPartitionMapper must not be null"); + Objects.requireNonNull(interBrokerListenerName, "interBrokerListenerName must not be null"); + + this.metadataCache = metadataCache; + this.keyToPartitionMapper = keyToPartitionMapper; + this.interBrokerListenerName = interBrokerListenerName; + } + + @Override + public boolean containsTopic(String topic) { + return metadataCache.contains(topic); + } + + @Override + public Node getShareCoordinator(String key, String internalTopicName) { + if (metadataCache.contains(internalTopicName)) { + Set topicSet = new HashSet<>(); + topicSet.add(internalTopicName); + + List topicMetadata = CollectionConverters.asJava( + metadataCache.getTopicMetadata( + CollectionConverters.asScala(topicSet), + interBrokerListenerName, + false, + false + ) + ); + + if (topicMetadata == null || topicMetadata.isEmpty() || topicMetadata.get(0).errorCode() != Errors.NONE.code()) { + return Node.noNode(); + } else { + int partition = keyToPartitionMapper.apply(key); + Optional response = topicMetadata.get(0).partitions().stream() + .filter(responsePart -> responsePart.partitionIndex() == partition + && responsePart.leaderId() != MetadataResponse.NO_LEADER_ID) + .findFirst(); + + if (response.isPresent()) { + return OptionConverters.toJava(metadataCache.getAliveBrokerNode(response.get().leaderId(), interBrokerListenerName)) + .orElse(Node.noNode()); + } else { + return Node.noNode(); + } + } + } + return Node.noNode(); + } + + @Override + public List getClusterNodes() { + return CollectionConverters.asJava(metadataCache.getAliveBrokerNodes(interBrokerListenerName).toSeq()); + } +} diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index 0ef58ef849168..b24d0dc6e8b8f 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -1238,7 +1238,7 @@ public void testCloseSharePartitionManager() throws Exception { Timer timer = Mockito.mock(SystemTimerReaper.class); Persister persister = Mockito.mock(Persister.class); SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder() - .withTimer(timer).withShareGroupPersister(persister).build(); + .withTimer(timer).withShareGroupPersister(persister).build(); // Verify that 0 calls are made to timer.close() and persister.stop(). Mockito.verify(timer, times(0)).close(); @@ -2270,7 +2270,7 @@ static class SharePartitionManagerBuilder { private Time time = new MockTime(); private ShareSessionCache cache = new ShareSessionCache(10, 1000); private Map partitionCacheMap = new HashMap<>(); - private Persister persister = NoOpShareStatePersister.getInstance(); + private Persister persister = new NoOpShareStatePersister(); private Timer timer = new MockTimer(); private Metrics metrics = new Metrics(); diff --git a/core/src/test/java/kafka/server/share/SharePartitionTest.java b/core/src/test/java/kafka/server/share/SharePartitionTest.java index 1ef17b44eba03..c008e772956f7 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionTest.java @@ -5127,7 +5127,8 @@ private static class SharePartitionBuilder { private int defaultAcquisitionLockTimeoutMs = 30000; private int maxDeliveryCount = MAX_DELIVERY_COUNT; private int maxInflightMessages = MAX_IN_FLIGHT_MESSAGES; - private Persister persister = NoOpShareStatePersister.getInstance(); + + private Persister persister = new NoOpShareStatePersister(); private final ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class); private GroupConfigManager groupConfigManager = Mockito.mock(GroupConfigManager.class); diff --git a/core/src/test/java/kafka/test/api/ShareConsumerTest.java b/core/src/test/java/kafka/test/api/ShareConsumerTest.java index eef153e4a3bfe..da1877b43abc7 100644 --- a/core/src/test/java/kafka/test/api/ShareConsumerTest.java +++ b/core/src/test/java/kafka/test/api/ShareConsumerTest.java @@ -52,8 +52,10 @@ 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.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.time.Duration; import java.util.ArrayList; @@ -89,25 +91,31 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; -@Timeout(600) +@Timeout(1200) @Tag("integration") public class ShareConsumerTest { private KafkaClusterTestKit cluster; private final TopicPartition tp = new TopicPartition("topic", 0); private final TopicPartition warmupTp = new TopicPartition("warmup", 0); + private static final String DEFAULT_STATE_PERSISTER = "org.apache.kafka.server.share.persister.DefaultStatePersister"; + private static final String NO_OP_PERSISTER = "org.apache.kafka.server.share.persister.NoOpShareStatePersister"; @BeforeEach - public void createCluster() throws Exception { + public void createCluster(TestInfo testInfo) throws Exception { + String persisterClassName = NO_OP_PERSISTER; + if (testInfo.getDisplayName().contains(".persister=")) { + persisterClassName = testInfo.getDisplayName().split("=")[1]; + } cluster = new KafkaClusterTestKit.Builder( new TestKitNodes.Builder() - .setNumBrokerNodes(1) - .setNumControllerNodes(1) - .build()) + .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.persister.class.name", "org.apache.kafka.server.group.share.NoOpShareStatePersister") + .setConfigProp("group.share.persister.class.name", persisterClassName) .setConfigProp("group.share.record.lock.duration.ms", "15000") .setConfigProp("offsets.topic.replication.factor", "1") .setConfigProp("share.coordinator.state.topic.min.isr", "1") @@ -129,8 +137,9 @@ public void destroyCluster() throws Exception { cluster.close(); } - @Test - public void testPollNoSubscribeFails() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testPollNoSubscribeFails(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); assertEquals(Collections.emptySet(), shareConsumer.subscription()); // "Consumer is not subscribed to any topics." @@ -138,8 +147,9 @@ public void testPollNoSubscribeFails() { shareConsumer.close(); } - @Test - public void testSubscribeAndPollNoRecords() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscribeAndPollNoRecords(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); @@ -149,8 +159,9 @@ public void testSubscribeAndPollNoRecords() { assertEquals(0, records.count()); } - @Test - public void testSubscribePollUnsubscribe() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscribePollUnsubscribe(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); @@ -162,8 +173,9 @@ public void testSubscribePollUnsubscribe() { assertEquals(0, records.count()); } - @Test - public void testSubscribePollSubscribe() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscribePollSubscribe(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); @@ -177,8 +189,9 @@ public void testSubscribePollSubscribe() { assertEquals(0, records.count()); } - @Test - public void testSubscribeUnsubscribePollFails() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscribeUnsubscribePollFails(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); @@ -192,8 +205,9 @@ public void testSubscribeUnsubscribePollFails() { assertEquals(0, records.count()); } - @Test - public void testSubscribeSubscribeEmptyPollFails() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscribeSubscribeEmptyPollFails(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); Set subscription = Collections.singleton(tp.topic()); shareConsumer.subscribe(subscription); @@ -207,8 +221,9 @@ public void testSubscribeSubscribeEmptyPollFails() { assertEquals(0, records.count()); } - @Test - public void testSubscriptionAndPoll() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscriptionAndPoll(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -220,8 +235,9 @@ public void testSubscriptionAndPoll() { producer.close(); } - @Test - public void testSubscriptionAndPollMultiple() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscriptionAndPollMultiple(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -239,8 +255,9 @@ public void testSubscriptionAndPollMultiple() { producer.close(); } - @Test - public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement(String persister) { Map> partitionOffsetsMap = new HashMap<>(); Map partitionExceptionMap = new HashMap<>(); ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -265,8 +282,9 @@ public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement() { producer.close(); } - @Test - public void testAcknowledgementCommitCallbackOnClose() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testAcknowledgementCommitCallbackOnClose(String persister) { Map> partitionOffsetsMap = new HashMap<>(); Map partitionExceptionMap = new HashMap<>(); ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -291,8 +309,9 @@ public void testAcknowledgementCommitCallbackOnClose() { producer.close(); } - @Test - public void testAcknowledgementCommitCallbackInvalidRecordStateException() throws Exception { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testAcknowledgementCommitCallbackInvalidRecordStateException(String persister) throws Exception { Map> partitionOffsetsMap = new HashMap<>(); Map partitionExceptionMap = new HashMap<>(); ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -350,8 +369,9 @@ public void onComplete(Map> offsetsMap, Exception ex } } - @Test - public void testHeaders() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testHeaders(String persister) { int numRecords = 1; ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); record.headers().add("headerKey", "headerValue".getBytes()); @@ -389,13 +409,15 @@ private void testHeadersSerializeDeserialize(Serializer serializer, Dese producer.close(); } - @Test - public void testHeadersSerializerDeserializer() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testHeadersSerializerDeserializer(String persister) { testHeadersSerializeDeserialize(new BaseConsumerTest.SerializerImpl(), new BaseConsumerTest.DeserializerImpl()); } - @Test - public void testMaxPollRecords() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testMaxPollRecords(String persister) { int maxPollRecords = 2; int numRecords = 10000; @@ -404,7 +426,7 @@ public void testMaxPollRecords() { produceMessagesWithTimestamp(numRecords, startingTimestamp); KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), - "group1", Collections.singletonMap(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords))); + "group1", Collections.singletonMap(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords))); shareConsumer.subscribe(Collections.singleton(tp.topic())); List> records = consumeRecords(shareConsumer, numRecords); long i = 0L; @@ -425,8 +447,9 @@ public void testMaxPollRecords() { producer.close(); } - @Test - public void testControlRecordsSkipped() throws Exception { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testControlRecordsSkipped(String persister) throws Exception { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer transactionalProducer = createProducer(new ByteArraySerializer(), new ByteArraySerializer(), "T1"); @@ -467,8 +490,9 @@ public void testControlRecordsSkipped() throws Exception { transactionalProducer.close(); } - @Test - public void testExplicitAcknowledgeSuccess() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgeSuccess(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -484,8 +508,9 @@ public void testExplicitAcknowledgeSuccess() { producer.close(); } - @Test - public void testExplicitAcknowledgeCommitSuccess() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgeCommitSuccess(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -503,8 +528,9 @@ public void testExplicitAcknowledgeCommitSuccess() { producer.close(); } - @Test - public void testExplicitAcknowledgementCommitAsync() throws InterruptedException { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgementCommitAsync(String persister) throws InterruptedException { 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()); ProducerRecord record3 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -555,8 +581,9 @@ public void testExplicitAcknowledgementCommitAsync() throws InterruptedException assertNull(partitionExceptionMap1.get(tp)); } - @Test - public void testExplicitAcknowledgementCommitAsyncPartialBatch() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgementCommitAsyncPartialBatch(String persister) { 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()); ProducerRecord record3 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -614,8 +641,9 @@ public void testExplicitAcknowledgementCommitAsyncPartialBatch() { assertNull(partitionExceptionMap.get(tp)); } - @Test - public void testExplicitAcknowledgeReleasePollAccept() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgeReleasePollAccept(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -633,8 +661,9 @@ public void testExplicitAcknowledgeReleasePollAccept() { producer.close(); } - @Test - public void testExplicitAcknowledgeReleaseAccept() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgeReleaseAccept(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -650,8 +679,9 @@ public void testExplicitAcknowledgeReleaseAccept() { producer.close(); } - @Test - public void testExplicitAcknowledgeReleaseClose() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgeReleaseClose(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -665,8 +695,9 @@ public void testExplicitAcknowledgeReleaseClose() { } - @Test - public void testExplicitAcknowledgeThrowsNotInBatch() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testExplicitAcknowledgeThrowsNotInBatch(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -683,8 +714,9 @@ public void testExplicitAcknowledgeThrowsNotInBatch() { producer.close(); } - @Test - public void testImplicitAcknowledgeFailsExplicit() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testImplicitAcknowledgeFailsExplicit(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -700,8 +732,9 @@ public void testImplicitAcknowledgeFailsExplicit() { producer.close(); } - @Test - public void testImplicitAcknowledgeCommitSync() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testImplicitAcknowledgeCommitSync(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -719,8 +752,9 @@ public void testImplicitAcknowledgeCommitSync() { producer.close(); } - @Test - public void testImplicitAcknowledgementCommitAsync() throws InterruptedException { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testImplicitAcknowledgementCommitAsync(String persister) throws InterruptedException { 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()); ProducerRecord record3 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); @@ -756,8 +790,9 @@ public void testImplicitAcknowledgementCommitAsync() throws InterruptedException producer.close(); } - @Test - public void testFetchRecordLargerThanMaxPartitionFetchBytes() throws Exception { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testFetchRecordLargerThanMaxPartitionFetchBytes(String persister) throws Exception { int maxPartitionFetchBytes = 10000; 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]); @@ -766,7 +801,7 @@ public void testFetchRecordLargerThanMaxPartitionFetchBytes() throws Exception { producer.send(bigRecord).get(); KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), - "group1", Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxPartitionFetchBytes))); + "group1", Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxPartitionFetchBytes))); shareConsumer.subscribe(Collections.singleton(tp.topic())); ConsumerRecords records = shareConsumer.poll(Duration.ofMillis(5000)); assertEquals(1, records.count()); @@ -774,8 +809,9 @@ public void testFetchRecordLargerThanMaxPartitionFetchBytes() throws Exception { producer.close(); } - @Test - public void testMultipleConsumersWithDifferentGroupIds() throws InterruptedException { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testMultipleConsumersWithDifferentGroupIds(String persister) throws InterruptedException { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); @@ -822,8 +858,9 @@ public void testMultipleConsumersWithDifferentGroupIds() throws InterruptedExcep producer.close(); } - @Test - public void testMultipleConsumersInGroupSequentialConsumption() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testMultipleConsumersInGroupSequentialConsumption(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); @@ -857,8 +894,9 @@ public void testMultipleConsumersInGroupSequentialConsumption() { producer.close(); } - @Test - public void testMultipleConsumersInGroupConcurrentConsumption() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testMultipleConsumersInGroupConcurrentConsumption(String persister) { AtomicInteger totalMessagesConsumed = new AtomicInteger(0); int consumerCount = 4; @@ -900,8 +938,9 @@ public void testMultipleConsumersInGroupConcurrentConsumption() { } } - @Test - public void testMultipleConsumersInMultipleGroupsConcurrentConsumption() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testMultipleConsumersInMultipleGroupsConcurrentConsumption(String persister) { AtomicInteger totalMessagesConsumedGroup1 = new AtomicInteger(0); AtomicInteger totalMessagesConsumedGroup2 = new AtomicInteger(0); AtomicInteger totalMessagesConsumedGroup3 = new AtomicInteger(0); @@ -1007,8 +1046,9 @@ public void testMultipleConsumersInMultipleGroupsConcurrentConsumption() { } } - @Test - public void testConsumerCloseInGroupSequential() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testConsumerCloseInGroupSequential(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); @@ -1050,8 +1090,9 @@ public void testConsumerCloseInGroupSequential() { assertEquals(totalMessages, consumer1MessageCount + consumer2MessageCount); } - @Test - public void testMultipleConsumersInGroupFailureConcurrentConsumption() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testMultipleConsumersInGroupFailureConcurrentConsumption(String persister) { AtomicInteger totalMessagesConsumed = new AtomicInteger(0); int consumerCount = 4; @@ -1112,12 +1153,13 @@ public void testMultipleConsumersInGroupFailureConcurrentConsumption() { } } - @Test - public void testAcquisitionLockTimeoutOnConsumer() throws InterruptedException { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testAcquisitionLockTimeoutOnConsumer(String persister) throws InterruptedException { ProducerRecord producerRecord1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, - "key_1".getBytes(), "value_1".getBytes()); + "key_1".getBytes(), "value_1".getBytes()); ProducerRecord producerRecord2 = new ProducerRecord<>(tp.topic(), tp.partition(), null, - "key_2".getBytes(), "value_2".getBytes()); + "key_2".getBytes(), "value_2".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); KafkaShareConsumer shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); shareConsumer1.subscribe(Collections.singleton(tp.topic())); @@ -1162,8 +1204,9 @@ public void testAcquisitionLockTimeoutOnConsumer() throws InterruptedException { * Test to verify that the acknowledgement commit callback cannot invoke methods of KafkaShareConsumer. * The exception thrown is verified in {@link TestableAcknowledgeCommitCallbackWithShareConsumer} */ - @Test - public void testAcknowledgeCommitCallbackCallsShareConsumerDisallowed() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testAcknowledgeCommitCallbackCallsShareConsumerDisallowed(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -1202,8 +1245,9 @@ public void onComplete(Map> offsetsMap, Exception ex * Test to verify that the acknowledgement commit callback can invoke KafkaShareConsumer.wakeup() and it * wakes up the enclosing poll. */ - @Test - public void testAcknowledgeCommitCallbackCallsShareConsumerWakeup() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testAcknowledgeCommitCallbackCallsShareConsumerWakeup(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -1240,8 +1284,9 @@ 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 - public void testAcknowledgeCommitCallbackThrowsException() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testAcknowledgeCommitCallbackThrowsException(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -1273,8 +1318,9 @@ public void onComplete(Map> offsetsMap, Exception ex * Test to verify that calling Thread.interrupt() before KafkaShareConsumer.poll(Duration) * causes it to throw InterruptException */ - @Test - public void testPollThrowsInterruptExceptionIfInterrupted() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testPollThrowsInterruptExceptionIfInterrupted(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); shareConsumer.subscribe(Collections.singleton(tp.topic())); @@ -1295,8 +1341,9 @@ public void testPollThrowsInterruptExceptionIfInterrupted() { * Test to verify that InvalidTopicException is thrown if the consumer subscribes * to an invalid topic. */ - @Test - public void testSubscribeOnInvalidTopicThrowsInvalidTopicException() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscribeOnInvalidTopicThrowsInvalidTopicException(String persister) { KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); shareConsumer.subscribe(Collections.singleton("topic abc")); @@ -1310,8 +1357,9 @@ 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 - public void testWakeupWithFetchedRecordsAvailable() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testWakeupWithFetchedRecordsAvailable(String persister) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); producer.send(record); @@ -1328,8 +1376,9 @@ public void testWakeupWithFetchedRecordsAvailable() { producer.close(); } - @Test - public void testSubscriptionFollowedByTopicCreation() throws InterruptedException { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscriptionFollowedByTopicCreation(String persister) throws InterruptedException { KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1"); String topic = "foo"; @@ -1354,8 +1403,9 @@ public void testSubscriptionFollowedByTopicCreation() throws InterruptedExceptio producer.close(); } - @Test - public void testSubscriptionAndPollFollowedByTopicDeletion() throws InterruptedException, ExecutionException { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testSubscriptionAndPollFollowedByTopicDeletion(String persister) throws InterruptedException, ExecutionException { String topic1 = "bar"; String topic2 = "baz"; createTopic(topic1); @@ -1371,11 +1421,11 @@ public void testSubscriptionAndPollFollowedByTopicDeletion() throws InterruptedE producer.send(recordTopic1).get(); TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1, - DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); + DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); producer.send(recordTopic2); TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1, - DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); + DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); // Topic bar is deleted, hence poll should not give any results. deleteTopic(topic1); @@ -1385,17 +1435,18 @@ public void testSubscriptionAndPollFollowedByTopicDeletion() throws InterruptedE producer.send(recordTopic2); // Poll should give the record from the non-deleted topic baz. TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1, - DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); + DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); producer.send(recordTopic2); TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1, - DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); + DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records"); shareConsumer.close(); producer.close(); } - @Test - public void testLsoMovementByRecordsDeletion() { + @ParameterizedTest(name = "{displayName}.persister={0}") + @ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER}) + public void testLsoMovementByRecordsDeletion(String persister) { KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); ProducerRecord record = new ProducerRecord<>(tp.topic(), 0, null, "key".getBytes(), "value".getBytes()); @@ -1489,7 +1540,7 @@ private void produceMessagesWithTimestamp(int messageCount, long startingTimesta try (KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) { for (int i = 0; i < messageCount; i++) { ProducerRecord record = new ProducerRecord<>(tp.topic(), tp.partition(), startingTimestamp + i, - ("key " + i).getBytes(), ("value " + i).getBytes()); + ("key " + i).getBytes(), ("value " + i).getBytes()); producer.send(record); } producer.flush(); @@ -1519,7 +1570,7 @@ private void consumeMessages(AtomicInteger totalMessagesConsumed, Map partitionExceptionMap = new HashMap<>(); if (maxFetchBytes.isPresent()) { shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), groupId, - Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxFetchBytes.get()))); + Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxFetchBytes.get()))); } else { shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), groupId); } @@ -1629,8 +1680,8 @@ private KafkaShareConsumer createShareConsumer(Deserializer keyD private void warmup() throws InterruptedException, ExecutionException, TimeoutException { createTopic(warmupTp.topic()); TestUtils.waitForCondition(() -> - !cluster.brokers().get(0).metadataCache().getAliveBrokerNodes(new ListenerName("EXTERNAL")).isEmpty(), - DEFAULT_MAX_WAIT_MS, 100L, () -> "cache not up yet"); + !cluster.brokers().get(0).metadataCache().getAliveBrokerNodes(new ListenerName("EXTERNAL")).isEmpty(), + DEFAULT_MAX_WAIT_MS, 100L, () -> "cache not up yet"); ProducerRecord record = new ProducerRecord<>(warmupTp.topic(), warmupTp.partition(), null, "key".getBytes(), "value".getBytes()); KafkaProducer producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer()); KafkaShareConsumer shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "warmupgroup1"); @@ -1639,7 +1690,7 @@ private void warmup() throws InterruptedException, ExecutionException, TimeoutEx producer.send(record).get(15000, TimeUnit.MILLISECONDS); shareConsumer.subscribe(subscription); TestUtils.waitForCondition( - () -> shareConsumer.poll(Duration.ofMillis(5000)).count() == 1, 30000, 200L, () -> "warmup record not received"); + () -> shareConsumer.poll(Duration.ofMillis(5000)).count() == 1, 30000, 200L, () -> "warmup record not received"); } finally { producer.close(); shareConsumer.close(); diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 076e2e6247263..1c5c2d3b0cdd4 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1144,6 +1144,7 @@ class KafkaConfigTest { case ShareGroupConfig.SHARE_GROUP_MAX_GROUPS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case GroupCoordinatorConfig.SHARE_GROUP_MAX_SIZE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case ShareGroupConfig.SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case ShareGroupConfig.SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG => //ignore string case ShareGroupConfig.SHARE_FETCH_MAX_FETCH_RECORDS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index 62d2eb4357374..cdc8cf5dd758a 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -16,10 +16,7 @@ */ 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.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, 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 @@ -34,13 +31,16 @@ import java.util.Collections import scala.collection.convert.ImplicitConversions.`list asScalaBuffer` import scala.jdk.CollectionConverters._ -@Timeout(120) +@Timeout(1200) @ExtendWith(value = Array(classOf[ClusterTestExtensions])) -@ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1) +@ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "") +)) @Tag("integration") class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster){ private final val MAX_PARTITION_BYTES = 10000 + private final val MAX_WAIT_MS = 5000 @AfterEach def tearDown(): Unit = { @@ -81,14 +81,31 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(Errors.UNSUPPORTED_VERSION.code(), shareAcknowledgeResponse.data().errorCode()) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - ), - brokers = 2 + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + brokers = 2 + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ), + brokers = 2 + ), + ) ) def testShareFetchRequestToNonLeaderReplica(): Unit = { val groupId: String = "group" @@ -118,18 +135,33 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(leader, partitionData.currentLeader().leaderId()) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ) ) ) def testShareFetchRequestSuccess(): Unit = { val groupId: String = "group" val memberId = Uuid.randomUuid() - val topic = "topic" val partition = 0 @@ -166,12 +198,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedPartitionData, partitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ) ) ) def testShareFetchRequestSuccessMultiplePartitions(): Unit = { @@ -234,14 +282,31 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo }) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") - ), - brokers = 3 + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ), + brokers = 3 + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ), + brokers = 3 + ), + ) ) def testShareFetchRequestSuccessMultiplePartitionsMultipleBrokers(): Unit = { val groupId: String = "group" @@ -326,12 +391,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedPartitionData3, partitionData3) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareAcknowledgeRequestSuccessAccept(): Unit = { @@ -423,12 +504,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchRequestPiggybackedAccept(): Unit = { @@ -524,12 +621,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareAcknowledgeRequestSuccessRelease(): Unit = { @@ -618,12 +731,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchRequestPiggybackedRelease(): Unit = { @@ -697,12 +826,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareAcknowledgeRequestSuccessReject(): Unit = { @@ -794,12 +939,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchRequestPiggybackedReject(): Unit = { @@ -895,13 +1056,30 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), - new ClusterConfigProperty(key = "group.share.delivery.count.limit", value = "2") // Setting max delivery count config to 2 + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.delivery.count.limit", value = "2") // Setting max delivery count config to 2 + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true"), + new ClusterConfigProperty(key = "group.share.delivery.count.limit", value = "2") // Setting max delivery count config to 2 + ) + ), ) ) def testShareAcknowledgeRequestMaxDeliveryAttemptExhausted(): Unit = { @@ -1037,12 +1215,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedFetchPartitionData, fetchPartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchBrokerRespectsPartitionsSizeLimit(): Unit = { @@ -1091,12 +1285,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareFetchResponsePartitions(expectedPartitionData, partitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchRequestSuccessfulSharingBetweenMultipleConsumers(): Unit = { @@ -1167,12 +1377,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertTrue(partitionData2.acquiredRecords().get(0).lastOffset() < partitionData3.acquiredRecords().get(0).firstOffset()) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchRequestNoSharingBetweenMultipleConsumersFromDifferentGroups(): Unit = { @@ -1247,12 +1473,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(partitionData3.acquiredRecords(), expectedAcquiredRecords(Collections.singletonList(0), Collections.singletonList(9), Collections.singletonList(1))) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareSessionCloseWithShareFetch(): Unit = { @@ -1338,12 +1580,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(0, shareFetchResponseData.responses().size()) // responses list will be empty because there are no responses for the final fetch request } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareSessionCloseWithShareAcknowledge(): Unit = { @@ -1439,12 +1697,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo compareAcknowledgeResponsePartitions(expectedAcknowledgePartitionData, acknowledgePartitionData) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchInitialEpochWithAcknowledgements(): Unit = { @@ -1479,12 +1753,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(Errors.INVALID_REQUEST.code(), shareFetchResponseData.errorCode) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareAcknowledgeInitialRequestError(): Unit = { @@ -1569,12 +1859,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareFetchResponseData.errorCode) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareAcknowledgeRequestInvalidShareSessionEpoch(): Unit = { @@ -1629,12 +1935,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(Errors.INVALID_SHARE_SESSION_EPOCH.code, shareAcknowledgeResponseData.errorCode) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchRequestShareSessionNotFound(): Unit = { @@ -1685,12 +2007,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(Errors.SHARE_SESSION_NOT_FOUND.code, shareFetchResponseData.errorCode) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareAcknowledgeRequestShareSessionNotFound(): Unit = { @@ -1746,12 +2084,28 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(Errors.SHARE_SESSION_NOT_FOUND.code, shareAcknowledgeResponseData.errorCode) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), - new ClusterConfigProperty(key = "group.share.enable", value = "true"), - new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), - new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + @ClusterTests( + Array( + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1") + ) + ), + new ClusterTest( + serverProperties = Array( + new ClusterConfigProperty(key = "group.coordinator.rebalance.protocols", value = "classic,consumer,share"), + new ClusterConfigProperty(key = "group.share.enable", value = "true"), + new ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "group.share.persister.class.name", value = "org.apache.kafka.server.share.persister.DefaultStatePersister"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), + new ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "1"), + new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + ) + ), ) ) def testShareFetchRequestForgetTopicPartitions(): Unit = { @@ -1845,7 +2199,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo send: Seq[TopicIdPartition], forget: Seq[TopicIdPartition], acknowledgementsMap: Map[TopicIdPartition, util.List[ShareFetchRequestData.AcknowledgementBatch]], - maxWaitMs: Int = Int.MaxValue, + maxWaitMs: Int = MAX_WAIT_MS, minBytes: Int = 0, maxBytes: Int = Int.MaxValue): ShareFetchRequest = { ShareFetchRequest.Builder.forConsumer(groupId, metadata, maxWaitMs, minBytes, maxBytes, maxPartitionBytes, send.asJava, forget.asJava, acknowledgementsMap.asJava) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala index 4f7a2e204dbdd..507f83b0de8cb 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala @@ -40,7 +40,9 @@ import scala.jdk.CollectionConverters._ @Timeout(120) @ExtendWith(value = Array(classOf[ClusterTestExtensions])) -@ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1) +@ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( + new ClusterConfigProperty(key = ShareGroupConfig.SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG, value = "") +)) @Tag("integration") class ShareGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index fa9ad9da7b2d1..4b04e3e837d81 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -16,9 +16,7 @@ */ 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 org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, Type} import org.apache.kafka.common.test.api.RaftClusterInvocationContext.RaftClusterInstance import kafka.utils.TestUtils import kafka.utils.TestUtils.waitForAllPartitionsMetadata @@ -35,6 +33,9 @@ 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 = "") +)) @Tag("integration") class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java index 15dbca6b9fa85..8c7e7a15c1f39 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java @@ -35,6 +35,7 @@ import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.INT; import static org.apache.kafka.common.config.ConfigDef.Type.SHORT; +import static org.apache.kafka.common.config.ConfigDef.Type.STRING; public class ShareGroupConfig { /** Share Group Configurations **/ @@ -72,6 +73,11 @@ public class ShareGroupConfig { public static final int SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_DEFAULT = 1000; public static final String SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_DOC = "The purge interval (in number of requests) of the share fetch request purgatory"; + public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG = "group.share.persister.class.name"; + public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DEFAULT = "org.apache.kafka.server.share.persister.DefaultStatePersister"; + public static final String SHARE_GROUP_PERSISTER_CLASS_NAME_DOC = "The class name of share persister for share group. The class should implement " + + "the org.apache.kafka.server.share.Persister interface."; + // Broker temporary configuration to limit the number of records fetched by a share fetch request. public static final String SHARE_FETCH_MAX_FETCH_RECORDS_CONFIG = "share.fetch.max.fetch.records"; public static final int SHARE_FETCH_MAX_FETCH_RECORDS_DEFAULT = Integer.MAX_VALUE; @@ -86,7 +92,8 @@ public class ShareGroupConfig { .define(SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG, INT, SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_DEFAULT, between(30000, 3600000), MEDIUM, SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_DOC) .define(SHARE_GROUP_MAX_GROUPS_CONFIG, SHORT, SHARE_GROUP_MAX_GROUPS_DEFAULT, between(1, 100), MEDIUM, SHARE_GROUP_MAX_GROUPS_DOC) .define(SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG, INT, SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_DEFAULT, between(100, 10000), MEDIUM, SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_DOC) - .define(SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG, INT, SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_DEFAULT, MEDIUM, SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_DOC); + .define(SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG, INT, SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_DEFAULT, MEDIUM, SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_DOC) + .defineInternal(SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG, STRING, SHARE_GROUP_PERSISTER_CLASS_NAME_DEFAULT, null, MEDIUM, SHARE_GROUP_PERSISTER_CLASS_NAME_DOC); private final boolean isShareGroupEnabled; private final int shareGroupPartitionMaxRecordLocks; @@ -96,6 +103,7 @@ public class ShareGroupConfig { private final int shareGroupMaxRecordLockDurationMs; private final int shareGroupMinRecordLockDurationMs; private final int shareFetchPurgatoryPurgeIntervalRequests; + private final String shareGroupPersisterClassName; private final int shareFetchMaxFetchRecords; public ShareGroupConfig(AbstractConfig config) { @@ -112,6 +120,7 @@ public ShareGroupConfig(AbstractConfig config) { shareGroupMaxRecordLockDurationMs = config.getInt(ShareGroupConfig.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG); shareGroupMinRecordLockDurationMs = config.getInt(ShareGroupConfig.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG); shareFetchPurgatoryPurgeIntervalRequests = config.getInt(ShareGroupConfig.SHARE_FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG); + shareGroupPersisterClassName = config.getString(ShareGroupConfig.SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG); shareFetchMaxFetchRecords = config.getInt(ShareGroupConfig.SHARE_FETCH_MAX_FETCH_RECORDS_CONFIG); validate(); } @@ -149,6 +158,10 @@ public int shareFetchPurgatoryPurgeIntervalRequests() { return shareFetchPurgatoryPurgeIntervalRequests; } + public String shareGroupPersisterClassName() { + return shareGroupPersisterClassName; + } + public int shareFetchMaxFetchRecords() { return shareFetchMaxFetchRecords; } @@ -160,7 +173,6 @@ private void validate() { Utils.require(shareGroupMaxRecordLockDurationMs >= shareGroupRecordLockDurationMs, String.format("%s must be greater than or equal to %s", SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG, SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG)); - } /** diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java index c8e5c46578a4a..67f9d606414fe 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java @@ -22,6 +22,8 @@ import org.apache.kafka.common.message.WriteShareGroupStateRequestData; import org.apache.kafka.common.message.WriteShareGroupStateResponseData; import org.apache.kafka.common.requests.RequestContext; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; import java.util.OptionalInt; import java.util.Properties; @@ -91,4 +93,15 @@ public interface ShareCoordinator { * @param partitionLeaderEpoch - Leader epoch of the partition (internal topic). Empty optional means deleted. */ void onResignation(int partitionIndex, OptionalInt partitionLeaderEpoch); + + /** + * A new metadata image is available. + * + * @param newImage The new metadata image. + * @param delta The metadata delta. + */ + void onNewMetadataImage( + MetadataImage newImage, + MetadataDelta delta + ); } diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java index 0242ee3f54e11..870d8c540889a 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java @@ -42,6 +42,8 @@ import org.apache.kafka.coordinator.common.runtime.MultiThreadedEventProcessor; import org.apache.kafka.coordinator.common.runtime.PartitionWriter; import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics; +import org.apache.kafka.image.MetadataDelta; +import org.apache.kafka.image.MetadataImage; import org.apache.kafka.server.config.ShareCoordinatorConfig; import org.apache.kafka.server.record.BrokerCompressionType; import org.apache.kafka.server.share.SharePartitionKey; @@ -200,6 +202,7 @@ public ShareCoordinatorService( @Override public int partitionFor(String key) { + throwIfNotActive(); return Utils.abs(key.hashCode()) % numPartitions; } @@ -513,6 +516,7 @@ private static boolean isGroupIdEmpty(String groupId) { @Override public void onElection(int partitionIndex, int partitionLeaderEpoch) { + throwIfNotActive(); runtime.scheduleLoadOperation( new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, partitionIndex), partitionLeaderEpoch @@ -521,12 +525,19 @@ public void onElection(int partitionIndex, int partitionLeaderEpoch) { @Override public void onResignation(int partitionIndex, OptionalInt partitionLeaderEpoch) { + throwIfNotActive(); runtime.scheduleUnloadOperation( new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, partitionIndex), partitionLeaderEpoch ); } + @Override + public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) { + throwIfNotActive(); + this.runtime.onNewMetadataImage(newImage, delta); + } + private TopicPartition topicPartitionFor(SharePartitionKey key) { return new TopicPartition(Topic.SHARE_GROUP_STATE_TOPIC_NAME, partitionFor(key.toString())); } @@ -534,4 +545,10 @@ private TopicPartition topicPartitionFor(SharePartitionKey key) { private static

boolean isEmpty(List

list) { return list == null || list.isEmpty(); } + + private void throwIfNotActive() { + if (!isActive.get()) { + throw Errors.COORDINATOR_NOT_AVAILABLE.exception(); + } + } } diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java index ef4b90e021808..2fa0d2162d3eb 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java @@ -168,7 +168,6 @@ public ShareCoordinatorShard build() { @Override public void onLoaded(MetadataImage newImage) { - this.metadataImage = newImage; coordinatorMetrics.activateMetricsShard(metricsShard); } diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java index 0423ca6fe1c33..a197d76368a33 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java @@ -92,7 +92,7 @@ ShareCoordinatorShard build() { ); when(metadataImage.topics().getTopic((Uuid) any())).thenReturn(mock(TopicImage.class)); when(metadataImage.topics().getPartition(any(), anyInt())).thenReturn(mock(PartitionRegistration.class)); - shard.onLoaded(metadataImage); + shard.onNewMetadataImage(metadataImage, null); return shard; } @@ -357,7 +357,7 @@ public void testWriteStateInvalidRequestData() { @Test public void testWriteNullMetadataImage() { ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); - shard.onLoaded(null); + shard.onNewMetadataImage(null, null); SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION); @@ -590,7 +590,7 @@ public void testReadNullMetadataImage() { writeAndReplayDefaultRecord(shard); - shard.onLoaded(null); + shard.onNewMetadataImage(null, null); SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION); diff --git a/share/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java b/share/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java new file mode 100644 index 0000000000000..15ecc89f78d2f --- /dev/null +++ b/share/src/main/java/org/apache/kafka/server/share/persister/DefaultStatePersister.java @@ -0,0 +1,323 @@ +/* + * 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.server.share.persister; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ReadShareGroupStateResponse; +import org.apache.kafka.common.requests.WriteShareGroupStateResponse; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; + +/** + * The default implementation of the {@link Persister} interface which is used by the + * group coordinator and share-partition leaders to manage the durable share-partition state. + * This implementation uses inter-broker RPCs to make requests to the share coordinator + * which is responsible for persisting the share-partition state. + */ +public class DefaultStatePersister implements Persister { + private final PersisterStateManager stateManager; + + private static final Logger log = LoggerFactory.getLogger(DefaultStatePersister.class); + + public DefaultStatePersister(PersisterStateManager stateManager) { + this.stateManager = stateManager; + this.stateManager.start(); + } + + @Override + public void stop() { + try { + if (stateManager != null) { + stateManager.stop(); + } + } catch (Exception e) { + log.error("Unable to stop state manager", e); + } + } + + /** + * Used by the group coordinator to initialize the share-partition state. + * This is an inter-broker RPC authorized as a cluster action. + * + * @param request InitializeShareGroupStateParameters + * @return A completable future of InitializeShareGroupStateResult + */ + public CompletableFuture initializeState(InitializeShareGroupStateParameters request) throws IllegalArgumentException { + throw new RuntimeException("not implemented"); + } + + /** + * Used by share-partition leaders to write share-partition state to a share coordinator. + * This is an inter-broker RPC authorized as a cluster action. + * + * @param request WriteShareGroupStateParameters + * @return A completable future of WriteShareGroupStateResult + */ + public CompletableFuture writeState(WriteShareGroupStateParameters request) throws IllegalArgumentException { + validate(request); + GroupTopicPartitionData gtp = request.groupTopicPartitionData(); + String groupId = gtp.groupId(); + + Map>> futureMap = new HashMap<>(); + List handlers = new ArrayList<>(); + + gtp.topicsData().forEach(topicData -> { + topicData.partitions().forEach(partitionData -> { + CompletableFuture future = futureMap + .computeIfAbsent(topicData.topicId(), k -> new HashMap<>()) + .computeIfAbsent(partitionData.partition(), k -> new CompletableFuture<>()); + + handlers.add( + stateManager.new WriteStateHandler( + groupId, + topicData.topicId(), + partitionData.partition(), + partitionData.stateEpoch(), + partitionData.leaderEpoch(), + partitionData.startOffset(), + partitionData.stateBatches(), + future, null) + ); + }); + }); + + for (PersisterStateManager.PersisterStateManagerHandler handler : handlers) { + stateManager.enqueue(handler); + } + + CompletableFuture combinedFuture = CompletableFuture.allOf( + handlers.stream() + .map(PersisterStateManager.WriteStateHandler::result) + .toArray(CompletableFuture[]::new)); + + return combinedFuture.thenApply(v -> writeResponsesToResult(futureMap)); + } + + private WriteShareGroupStateResult writeResponsesToResult( + Map>> futureMap + ) { + List> topicsData = futureMap.keySet().stream() + .map(topicId -> { + List partitionErrData = futureMap.get(topicId).entrySet().stream() + .map(partitionFuture -> { + int partition = partitionFuture.getKey(); + CompletableFuture future = partitionFuture.getValue(); + try { + WriteShareGroupStateResponse partitionResponse = future.get(); + return partitionResponse.data().results().get(0).partitions().stream() + .map(partitionResult -> PartitionFactory.newPartitionErrorData( + partitionResult.partition(), + partitionResult.errorCode(), + partitionResult.errorMessage())) + .collect(Collectors.toList()); + } catch (InterruptedException | ExecutionException e) { + log.error("Unexpected exception while writing data to share coordinator", e); + return Collections.singletonList(PartitionFactory.newPartitionErrorData( + partition, + Errors.UNKNOWN_SERVER_ERROR.code(), // No specific public error code exists for InterruptedException / ExecutionException + "Error writing state to share coordinator: " + e.getMessage()) + ); + } + }) + .flatMap(List::stream) + .collect(Collectors.toList()); + return new TopicData<>(topicId, partitionErrData); + }) + .collect(Collectors.toList()); + return new WriteShareGroupStateResult.Builder() + .setTopicsData(topicsData) + .build(); + } + + /** + * Used by share-partition leaders to read share-partition state from a share coordinator. + * This is an inter-broker RPC authorized as a cluster action. + * + * @param request ReadShareGroupStateParameters + * @return A completable future of ReadShareGroupStateResult + */ + public CompletableFuture readState(ReadShareGroupStateParameters request) throws IllegalArgumentException { + validate(request); + GroupTopicPartitionData gtp = request.groupTopicPartitionData(); + String groupId = gtp.groupId(); + Map>> futureMap = new HashMap<>(); + List handlers = new ArrayList<>(); + + gtp.topicsData().forEach(topicData -> { + topicData.partitions().forEach(partitionData -> { + CompletableFuture future = futureMap + .computeIfAbsent(topicData.topicId(), k -> new HashMap<>()) + .computeIfAbsent(partitionData.partition(), k -> new CompletableFuture<>()); + + handlers.add( + stateManager.new ReadStateHandler( + groupId, + topicData.topicId(), + partitionData.partition(), + partitionData.leaderEpoch(), + future, + null) + ); + }); + }); + + for (PersisterStateManager.PersisterStateManagerHandler handler : handlers) { + stateManager.enqueue(handler); + } + + // Combine all futures into a single CompletableFuture + CompletableFuture combinedFuture = CompletableFuture.allOf( + handlers.stream() + .map(PersisterStateManager.ReadStateHandler::result) + .toArray(CompletableFuture[]::new)); + + // Transform the combined CompletableFuture into CompletableFuture + return combinedFuture.thenApply(v -> readResponsesToResult(futureMap)); + } + + private ReadShareGroupStateResult readResponsesToResult( + Map>> futureMap + ) { + List> topicsData = futureMap.keySet().stream() + .map(topicId -> { + List partitionAllData = futureMap.get(topicId).entrySet().stream() + .map(partitionFuture -> { + int partition = partitionFuture.getKey(); + CompletableFuture future = partitionFuture.getValue(); + try { + ReadShareGroupStateResponse partitionResponse = future.get(); + return partitionResponse.data().results().get(0).partitions().stream() + .map(partitionResult -> PartitionFactory.newPartitionAllData( + partitionResult.partition(), + partitionResult.stateEpoch(), + partitionResult.startOffset(), + partitionResult.errorCode(), + partitionResult.errorMessage(), + partitionResult.stateBatches().stream().map(PersisterStateBatch::from).collect(Collectors.toList()) + )) + .collect(Collectors.toList()); + } catch (InterruptedException | ExecutionException e) { + log.error("Unexpected exception while getting data from share coordinator", e); + return Collections.singletonList(PartitionFactory.newPartitionAllData( + partition, + -1, + -1, + Errors.UNKNOWN_SERVER_ERROR.code(), // No specific public error code exists for InterruptedException / ExecutionException + "Error reading state from share coordinator: " + e.getMessage(), + Collections.emptyList()) + ); + } + }) + .flatMap(List::stream) + .collect(Collectors.toList()); + return new TopicData<>(topicId, partitionAllData); + }) + .collect(Collectors.toList()); + return new ReadShareGroupStateResult.Builder() + .setTopicsData(topicsData) + .build(); + } + + /** + * Used by the group coordinator to delete share-partition state from a share coordinator. + * This is an inter-broker RPC authorized as a cluster action. + * + * @param request DeleteShareGroupStateParameters + * @return A completable future of DeleteShareGroupStateResult + */ + public CompletableFuture deleteState(DeleteShareGroupStateParameters request) throws IllegalArgumentException { + throw new RuntimeException("not implemented"); + } + + /** + * Used by the group coordinator to read the offset information from share-partition state from a share coordinator. + * This is an inter-broker RPC authorized as a cluster action. + * + * @param request ReadShareGroupStateSummaryParameters + * @return A completable future of ReadShareGroupStateSummaryResult + */ + public CompletableFuture readSummary(ReadShareGroupStateSummaryParameters request) throws IllegalArgumentException { + throw new RuntimeException("not implemented"); + } + + private static void validate(WriteShareGroupStateParameters params) { + String prefix = "Write share group parameters"; + if (params == null) { + throw new IllegalArgumentException(prefix + " cannot be null."); + } + if (params.groupTopicPartitionData() == null) { + throw new IllegalArgumentException(prefix + " data cannot be null."); + } + + validateGroupTopicPartitionData(prefix, params.groupTopicPartitionData()); + } + + private static void validate(ReadShareGroupStateParameters params) { + String prefix = "Read share group parameters"; + if (params == null) { + throw new IllegalArgumentException(prefix + " cannot be null."); + } + if (params.groupTopicPartitionData() == null) { + throw new IllegalArgumentException(prefix + " data cannot be null."); + } + + validateGroupTopicPartitionData(prefix, params.groupTopicPartitionData()); + } + + private static void validateGroupTopicPartitionData(String prefix, GroupTopicPartitionData data) { + String groupId = data.groupId(); + if (groupId == null || groupId.isEmpty()) { + throw new IllegalArgumentException(prefix + " groupId cannot be null or empty."); + } + + List> topicsData = data.topicsData(); + if (isEmpty(topicsData)) { + throw new IllegalArgumentException(prefix + " topics data cannot be null or empty."); + } + + for (TopicData topicData : topicsData) { + if (topicData.topicId() == null) { + throw new IllegalArgumentException(prefix + " topicId cannot be null."); + } + if (isEmpty(topicData.partitions())) { + throw new IllegalArgumentException(prefix + " partitions cannot be null or empty."); + } + for (PartitionIdData partitionData : topicData.partitions()) { + if (partitionData.partition() < 0) { + throw new IllegalArgumentException( + String.format("%s has invalid partitionId - %s %s %d", prefix, groupId, topicData.topicId(), partitionData.partition())); + } + } + } + } + + private static boolean isEmpty(List list) { + return list == null || list.isEmpty(); + } +} diff --git a/share/src/main/java/org/apache/kafka/server/share/persister/NoOpShareStatePersister.java b/share/src/main/java/org/apache/kafka/server/share/persister/NoOpShareStatePersister.java index f2c7e56b2b5e0..9d37b114b2730 100644 --- a/share/src/main/java/org/apache/kafka/server/share/persister/NoOpShareStatePersister.java +++ b/share/src/main/java/org/apache/kafka/server/share/persister/NoOpShareStatePersister.java @@ -28,15 +28,7 @@ */ public class NoOpShareStatePersister implements Persister { - private NoOpShareStatePersister() { - } - - private static final class InstanceHolder { - static final Persister INSTANCE = new NoOpShareStatePersister(); - } - - public static Persister getInstance() { - return InstanceHolder.INSTANCE; + public NoOpShareStatePersister() { } @Override diff --git a/share/src/main/java/org/apache/kafka/server/share/persister/Persister.java b/share/src/main/java/org/apache/kafka/server/share/persister/Persister.java index 63782a927ac7a..49073c83cd682 100644 --- a/share/src/main/java/org/apache/kafka/server/share/persister/Persister.java +++ b/share/src/main/java/org/apache/kafka/server/share/persister/Persister.java @@ -17,17 +17,14 @@ package org.apache.kafka.server.share.persister; -import org.apache.kafka.common.annotation.InterfaceStability; - import java.util.concurrent.CompletableFuture; /** * This interface introduces methods which can be used by callers to interact with the * persistence implementation responsible for storing share group/partition states. - * For KIP-932, the default {@link Persister} use a share coordinator which stores information in - * an internal topic, but this interface allows for other variations as well. + * For KIP-932, the default {@link Persister} uses a share coordinator to store information in + * an internal topic. */ -@InterfaceStability.Evolving public interface Persister { /** * Initialize the share partition state. diff --git a/share/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java b/share/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java new file mode 100644 index 0000000000000..1f0fddf798348 --- /dev/null +++ b/share/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java @@ -0,0 +1,976 @@ +/* + * 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.server.share.persister; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.RequestCompletionHandler; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.internals.Topic; +import org.apache.kafka.common.message.FindCoordinatorRequestData; +import org.apache.kafka.common.message.FindCoordinatorResponseData; +import org.apache.kafka.common.message.ReadShareGroupStateRequestData; +import org.apache.kafka.common.message.ReadShareGroupStateResponseData; +import org.apache.kafka.common.message.WriteShareGroupStateRequestData; +import org.apache.kafka.common.message.WriteShareGroupStateResponseData; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.requests.ReadShareGroupStateRequest; +import org.apache.kafka.common.requests.ReadShareGroupStateResponse; +import org.apache.kafka.common.requests.WriteShareGroupStateRequest; +import org.apache.kafka.common.requests.WriteShareGroupStateResponse; +import org.apache.kafka.common.utils.ExponentialBackoff; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.share.SharePartitionKey; +import org.apache.kafka.server.util.InterBrokerSendThread; +import org.apache.kafka.server.util.RequestAndCompletionHandler; +import org.apache.kafka.server.util.timer.Timer; +import org.apache.kafka.server.util.timer.TimerTask; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * This class encapsulates various handler classes corresponding to share + * state RPCs. It also holds an {@link InterBrokerSendThread} specialization + * which manages the sending the RPC requests over the network. + * This class is for the exclusive purpose of being used with {@link DefaultStatePersister} + * but can be extended for other {@link Persister} implementations as well. + */ +public class PersisterStateManager { + private SendThread sender; + private final AtomicBoolean isStarted = new AtomicBoolean(false); + public static final long REQUEST_BACKOFF_MS = 1_000L; + public static final long REQUEST_BACKOFF_MAX_MS = 30_000L; + private static final int MAX_FIND_COORD_ATTEMPTS = 5; + private final Time time; + private final Timer timer; + private final ShareCoordinatorMetadataCacheHelper cacheHelper; + // holds the set of share coord nodes for each RPC type which is currently sent but not completed + private final Map> inFlight = new HashMap<>(); + + // Mapping for batchable RPCs. The top level grouping is based on destination share coordinator node. + // Since kafkaApis for each RPC type are separate, we cannot batch different types of RPCs. Hence, we need + // RPCType'd key inner map. + // The RPC schemas defined in kip-932 have a single group id per request. Hence, we cannot batch RPCs + // with different groupIds and therefore, another inner map keyed on groupId is needed. + // Finally, the value is a list of handlers + private final Map>>> nodeRPCMap = new HashMap<>(); + + // Final object to serve synchronization needs. + private final Object nodeMapLock = new Object(); + + // Called when the generateRequests method is executed by InterBrokerSendThread, returning requests. + // Mainly for testing and introspection purpose to inspect the state of the nodeRPC map + // when generateRequests is called. + private Runnable generateCallback; + + private static class BackoffManager { + private final int maxAttempts; + private int attempts; + private final ExponentialBackoff backoff; + + BackoffManager(int maxAttempts, long initialBackoffMs, long maxBackoffMs) { + this.maxAttempts = maxAttempts; + this.backoff = new ExponentialBackoff( + initialBackoffMs, + CommonClientConfigs.RETRY_BACKOFF_EXP_BASE, + maxBackoffMs, + CommonClientConfigs.RETRY_BACKOFF_JITTER + ); + } + + void incrementAttempt() { + attempts++; + } + + void resetAttempts() { + attempts = 0; + } + + boolean canAttempt() { + return attempts < maxAttempts; + } + + long backOff() { + return this.backoff.backoff(attempts); + } + } + + public enum RPCType { + READ, + WRITE, + DELETE, + SUMMARY, + UNKNOWN + } + + public PersisterStateManager(KafkaClient client, ShareCoordinatorMetadataCacheHelper cacheHelper, Time time, Timer timer) { + if (client == null) { + throw new IllegalArgumentException("Kafkaclient must not be null."); + } + if (time == null) { + throw new IllegalArgumentException("Time must not be null."); + } + if (timer == null) { + throw new IllegalArgumentException("Timer must not be null."); + } + if (cacheHelper == null) { + throw new IllegalArgumentException("CacheHelper must not be null."); + } + this.time = time; + this.timer = timer; + this.cacheHelper = cacheHelper; + this.sender = new SendThread( + "PersisterStateManager", + client, + Math.toIntExact(CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS), //30 seconds + this.time, + true, + new Random(this.time.milliseconds())); + } + + public void enqueue(PersisterStateManagerHandler handler) { + this.sender.enqueue(handler); + } + + public void start() { + if (isStarted.compareAndSet(false, true)) { + this.sender.start(); + isStarted.set(true); + } + } + + public void stop() throws Exception { + if (isStarted.compareAndSet(true, false)) { + this.sender.shutdown(); + Utils.closeQuietly(this.timer, "PersisterStateManager timer"); + } + } + + // test visibility + Map>>> nodeRPCMap() { + return nodeRPCMap; + } + + public void setGenerateCallback(Runnable generateCallback) { + this.generateCallback = generateCallback; + } + + /** + * Parent class of all RPCs. Uses template pattern to implement core methods. + * Various child classes can extend this class to define how to handle RPC specific + * responses, retries, batching etc. + *

+ * Since the find coordinator RPC/lookup is a necessary pre-condition for all + * share state RPCs, the infra code for it is encapsulated in this class itself. + */ + public abstract class PersisterStateManagerHandler implements RequestCompletionHandler { + protected Node coordinatorNode; + protected final String groupId; + protected final Uuid topicId; + protected final int partition; + private final BackoffManager findCoordBackoff; + protected final Logger log = LoggerFactory.getLogger(getClass()); + private Consumer onCompleteCallback; + + public PersisterStateManagerHandler( + String groupId, + Uuid topicId, + int partition, + long backoffMs, + long backoffMaxMs, + int maxRPCRetryAttempts + ) { + this.groupId = groupId; + this.topicId = topicId; + this.partition = partition; + this.findCoordBackoff = new BackoffManager(maxRPCRetryAttempts, backoffMs, backoffMaxMs); + this.onCompleteCallback = response -> { + }; // noop + } + + /** + * Child class must create appropriate builder object for the handled RPC + * + * @return builder for the request + */ + protected abstract AbstractRequest.Builder requestBuilder(); + + /** + * Handles the response for an RPC. + * + * @param response - Client response + */ + protected abstract void handleRequestResponse(ClientResponse response); + + /** + * Returns true if the response is valid for the respective child class. + * + * @param response - Client response + * @return - boolean + */ + protected abstract boolean isResponseForRequest(ClientResponse response); + + /** + * Handle invalid find coordinator response. If error is UNKNOWN_SERVER_ERROR. Look at the + * exception details to figure out the problem. + * + * @param error + * @param exception + */ + protected abstract void findCoordinatorErrorResponse(Errors error, Exception exception); + + /** + * Child class must provide a descriptive name for the implementation. + * + * @return String + */ + protected abstract String name(); + + /** + * Child class must return appropriate type of RPC here + * + * @return String + */ + protected abstract RPCType rpcType(); + + /** + * Child class should return the appropriate completable future encapsulating + * the response for the RPC. + * + * @return A completable future of RPC response + */ + protected abstract CompletableFuture result(); + + /** + * Returns builder for share coordinator + * + * @return builder for find coordinator + */ + protected AbstractRequest.Builder findShareCoordinatorBuilder() { + return new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData() + .setKeyType(FindCoordinatorRequest.CoordinatorType.SHARE.id()) + .setKey(coordinatorKey())); + } + + public void addRequestToNodeMap(Node node, PersisterStateManagerHandler handler) { + if (!handler.isBatchable()) { + return; + } + synchronized (nodeMapLock) { + nodeRPCMap.computeIfAbsent(node, k -> new HashMap<>()) + .computeIfAbsent(handler.rpcType(), k -> new HashMap<>()) + .computeIfAbsent(handler.groupId, k -> new LinkedList<>()) + .add(handler); + } + sender.wakeup(); + } + + /** + * Returns true is coordinator node is not yet set + * + * @return boolean + */ + protected boolean lookupNeeded() { + if (coordinatorNode != null) { + return false; + } + if (cacheHelper.containsTopic(Topic.SHARE_GROUP_STATE_TOPIC_NAME)) { + log.debug("{} internal topic already exists.", Topic.SHARE_GROUP_STATE_TOPIC_NAME); + Node node = cacheHelper.getShareCoordinator(coordinatorKey(), Topic.SHARE_GROUP_STATE_TOPIC_NAME); + if (node != Node.noNode()) { + log.debug("Found coordinator node in cache: {}", node); + coordinatorNode = node; + addRequestToNodeMap(node, this); + return false; + } + } + return true; + } + + /** + * Returns the String key to be used as share coordinator key + * + * @return String + */ + protected String coordinatorKey() { + return SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + } + + /** + * Returns true if the RPC response if for Find Coordinator RPC. + * + * @param response - Client response object + * @return boolean + */ + protected boolean isFindCoordinatorResponse(ClientResponse response) { + return response != null && response.requestHeader().apiKey() == ApiKeys.FIND_COORDINATOR; + } + + @Override + public void onComplete(ClientResponse response) { + if (onCompleteCallback != null) { + onCompleteCallback.accept(response); + } + if (response != null && response.hasResponse()) { + if (isFindCoordinatorResponse(response)) { + handleFindCoordinatorResponse(response); + } else if (isResponseForRequest(response)) { + handleRequestResponse(response); + } + } + sender.wakeup(); + } + + protected void resetCoordinatorNode() { + coordinatorNode = null; + } + + /** + * Handles the response for find coordinator RPC and sets appropriate state. + * + * @param response - Client response for find coordinator RPC + */ + protected void handleFindCoordinatorResponse(ClientResponse response) { + log.debug("Find coordinator response received - {}", response); + + // Incrementing the number of find coordinator attempts + findCoordBackoff.incrementAttempt(); + List coordinators = ((FindCoordinatorResponse) response.responseBody()).coordinators(); + if (coordinators.size() != 1) { + log.error("Find coordinator response for {} is invalid", coordinatorKey()); + findCoordinatorErrorResponse(Errors.UNKNOWN_SERVER_ERROR, new IllegalStateException("Invalid response with multiple coordinators.")); + return; + } + + FindCoordinatorResponseData.Coordinator coordinatorData = coordinators.get(0); + Errors error = Errors.forCode(coordinatorData.errorCode()); + + switch (error) { + case NONE: + log.debug("Find coordinator response valid. Enqueuing actual request."); + findCoordBackoff.resetAttempts(); + coordinatorNode = new Node(coordinatorData.nodeId(), coordinatorData.host(), coordinatorData.port()); + // now we want the actual share state RPC call to happen + if (this.isBatchable()) { + addRequestToNodeMap(coordinatorNode, this); + } else { + enqueue(this); + } + break; + + case COORDINATOR_NOT_AVAILABLE: // retryable error codes + case COORDINATOR_LOAD_IN_PROGRESS: + log.warn("Received retryable error in find coordinator: {}", error.message()); + if (!findCoordBackoff.canAttempt()) { + log.error("Exhausted max retries to find coordinator without success."); + findCoordinatorErrorResponse(error, new Exception("Exhausted max retries to find coordinator without success.")); + break; + } + resetCoordinatorNode(); + timer.add(new PersisterTimerTask(findCoordBackoff.backOff(), this)); + break; + + default: + log.error("Unable to find coordinator."); + findCoordinatorErrorResponse(error, null); + } + } + + // Visible for testing + public Node getCoordinatorNode() { + return coordinatorNode; + } + + protected abstract boolean isBatchable(); + + /** + * This method can be called by child class objects to register a callback + * which will be called when the onComplete cb is called on request completion. + * + * @param callback + */ + protected void setOnCompleteCallback(Consumer callback) { + this.onCompleteCallback = callback; + } + } + + public class WriteStateHandler extends PersisterStateManagerHandler { + private final int stateEpoch; + private final int leaderEpoch; + private final long startOffset; + private final List batches; + private final CompletableFuture result; + private final BackoffManager writeStateBackoff; + + public WriteStateHandler( + String groupId, + Uuid topicId, + int partition, + int stateEpoch, + int leaderEpoch, + long startOffset, + List batches, + CompletableFuture result, + long backoffMs, + long backoffMaxMs, + int maxRPCRetryAttempts + ) { + super(groupId, topicId, partition, backoffMs, backoffMaxMs, maxRPCRetryAttempts); + this.stateEpoch = stateEpoch; + this.leaderEpoch = leaderEpoch; + this.startOffset = startOffset; + this.batches = batches; + this.result = result; + this.writeStateBackoff = new BackoffManager(maxRPCRetryAttempts, backoffMs, backoffMaxMs); + } + + public WriteStateHandler( + String groupId, + Uuid topicId, + int partition, + int stateEpoch, + int leaderEpoch, + long startOffset, + List batches, + CompletableFuture result, + Consumer onCompleteCallback + ) { + this( + groupId, + topicId, + partition, + stateEpoch, + leaderEpoch, + startOffset, + batches, + result, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_FIND_COORD_ATTEMPTS + ); + } + + @Override + protected String name() { + return "WriteStateHandler"; + } + + @Override + protected AbstractRequest.Builder requestBuilder() { + throw new RuntimeException("Write requests are batchable, hence individual requests not needed."); + } + + @Override + protected boolean isResponseForRequest(ClientResponse response) { + return response.requestHeader().apiKey() == ApiKeys.WRITE_SHARE_GROUP_STATE; + } + + @Override + protected void handleRequestResponse(ClientResponse response) { + log.debug("Write state response received - {}", response); + writeStateBackoff.incrementAttempt(); + + // response can be a combined one for large number of requests + // we need to deconstruct it + WriteShareGroupStateResponse combinedResponse = (WriteShareGroupStateResponse) response.responseBody(); + + for (WriteShareGroupStateResponseData.WriteStateResult writeStateResult : combinedResponse.data().results()) { + if (writeStateResult.topicId().equals(topicId)) { + Optional partitionStateData = + writeStateResult.partitions().stream().filter(partitionResult -> partitionResult.partition() == partition) + .findFirst(); + + if (partitionStateData.isPresent()) { + Errors error = Errors.forCode(partitionStateData.get().errorCode()); + switch (error) { + case NONE: + writeStateBackoff.resetAttempts(); + WriteShareGroupStateResponseData.WriteStateResult result = WriteShareGroupStateResponse.toResponseWriteStateResult( + topicId, + Collections.singletonList(partitionStateData.get()) + ); + this.result.complete(new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData().setResults(Collections.singletonList(result)))); + return; + + // check retryable errors + case COORDINATOR_NOT_AVAILABLE: + case COORDINATOR_LOAD_IN_PROGRESS: + log.warn("Received retryable error in write state RPC: {}", error.message()); + if (!writeStateBackoff.canAttempt()) { + log.error("Exhausted max retries for write state RPC without success."); + writeStateErrorResponse(error, new Exception("Exhausted max retries to complete write state RPC without success.")); + return; + } + super.resetCoordinatorNode(); + timer.add(new PersisterTimerTask(writeStateBackoff.backOff(), this)); + return; + + default: + log.error("Unable to perform write state RPC: {}", error.message()); + writeStateErrorResponse(error, null); + return; + } + } + } + } + + // no response found specific topic partition + IllegalStateException exception = new IllegalStateException( + "Failed to write state for partition " + partition + " in topic " + topicId + " for group " + groupId + ); + writeStateErrorResponse(Errors.forException(exception), exception); + } + + private void writeStateErrorResponse(Errors error, Exception exception) { + this.result.complete(new WriteShareGroupStateResponse( + WriteShareGroupStateResponse.toErrorResponseData(topicId, partition, error, "Error in write state RPC. " + + (exception == null ? error.message() : exception.getMessage())))); + } + + @Override + protected void findCoordinatorErrorResponse(Errors error, Exception exception) { + this.result.complete(new WriteShareGroupStateResponse( + WriteShareGroupStateResponse.toErrorResponseData(topicId, partition, error, "Error in find coordinator. " + + (exception == null ? error.message() : exception.getMessage())))); + } + + protected CompletableFuture result() { + return result; + } + + @Override + protected boolean isBatchable() { + return true; + } + + @Override + protected RPCType rpcType() { + return RPCType.WRITE; + } + } + + public class ReadStateHandler extends PersisterStateManagerHandler { + private final int leaderEpoch; + private final String coordinatorKey; + private final CompletableFuture result; + private final BackoffManager readStateBackoff; + + public ReadStateHandler( + String groupId, + Uuid topicId, + int partition, + int leaderEpoch, + CompletableFuture result, + long backoffMs, + long backoffMaxMs, + int maxRPCRetryAttempts, + Consumer onCompleteCallback + ) { + super(groupId, topicId, partition, backoffMs, backoffMaxMs, maxRPCRetryAttempts); + this.leaderEpoch = leaderEpoch; + this.coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + this.result = result; + this.readStateBackoff = new BackoffManager(maxRPCRetryAttempts, backoffMs, backoffMaxMs); + } + + public ReadStateHandler( + String groupId, + Uuid topicId, + int partition, + int leaderEpoch, + CompletableFuture result, + Consumer onCompleteCallback + ) { + this( + groupId, + topicId, + partition, + leaderEpoch, + result, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_FIND_COORD_ATTEMPTS, + onCompleteCallback + ); + } + + @Override + protected String name() { + return "ReadStateHandler"; + } + + @Override + protected AbstractRequest.Builder requestBuilder() { + throw new RuntimeException("Read requests are batchable, hence individual requests not needed."); + } + + @Override + protected boolean isResponseForRequest(ClientResponse response) { + return response.requestHeader().apiKey() == ApiKeys.READ_SHARE_GROUP_STATE; + } + + @Override + protected void handleRequestResponse(ClientResponse response) { + log.debug("Read state response received - {}", response); + readStateBackoff.incrementAttempt(); + + ReadShareGroupStateResponse combinedResponse = (ReadShareGroupStateResponse) response.responseBody(); + for (ReadShareGroupStateResponseData.ReadStateResult readStateResult : combinedResponse.data().results()) { + if (readStateResult.topicId().equals(topicId)) { + Optional partitionStateData = + readStateResult.partitions().stream().filter(partitionResult -> partitionResult.partition() == partition) + .findFirst(); + + if (partitionStateData.isPresent()) { + Errors error = Errors.forCode(partitionStateData.get().errorCode()); + switch (error) { + case NONE: + readStateBackoff.resetAttempts(); + ReadShareGroupStateResponseData.ReadStateResult result = ReadShareGroupStateResponse.toResponseReadStateResult( + topicId, + Collections.singletonList(partitionStateData.get()) + ); + this.result.complete(new ReadShareGroupStateResponse(new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList(result)))); + return; + + // check retryable errors + case COORDINATOR_NOT_AVAILABLE: + case COORDINATOR_LOAD_IN_PROGRESS: + log.warn("Received retryable error in read state RPC: {}", error.message()); + if (!readStateBackoff.canAttempt()) { + log.error("Exhausted max retries for read state RPC without success."); + readStateErrorReponse(error, new Exception("Exhausted max retries to complete read state RPC without success.")); + return; + } + super.resetCoordinatorNode(); + timer.add(new PersisterTimerTask(readStateBackoff.backOff(), this)); + return; + + default: + log.error("Unable to perform read state RPC: {}", error.message()); + readStateErrorReponse(error, null); + return; + } + } + } + } + + // no response found specific topic partition + IllegalStateException exception = new IllegalStateException( + "Failed to read state for partition " + partition + " in topic " + topicId + " for group " + groupId + ); + readStateErrorReponse(Errors.forException(exception), exception); + } + + protected void readStateErrorReponse(Errors error, Exception exception) { + this.result.complete(new ReadShareGroupStateResponse( + ReadShareGroupStateResponse.toErrorResponseData(topicId, partition, error, "Error in find coordinator. " + + (exception == null ? error.message() : exception.getMessage())))); + } + + @Override + protected void findCoordinatorErrorResponse(Errors error, Exception exception) { + this.result.complete(new ReadShareGroupStateResponse( + ReadShareGroupStateResponse.toErrorResponseData(topicId, partition, error, "Error in read state RPC. " + + (exception == null ? error.message() : exception.getMessage())))); + } + + protected CompletableFuture result() { + return result; + } + + @Override + protected boolean isBatchable() { + return true; + } + + @Override + protected RPCType rpcType() { + return RPCType.READ; + } + } + + private class SendThread extends InterBrokerSendThread { + private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); + private final Random random; + + public SendThread(String name, KafkaClient networkClient, int requestTimeoutMs, Time time, boolean isInterruptible, Random random) { + super(name, networkClient, requestTimeoutMs, time, isInterruptible); + this.random = random; + } + + private Node randomNode() { + List nodes = cacheHelper.getClusterNodes(); + if (nodes == null || nodes.isEmpty()) { + return Node.noNode(); + } + return nodes.get(random.nextInt(nodes.size())); + } + + /** + * The incoming requests will have the keys in the following format + * groupId: [ + * topidId1: [part1, part2, part3], + * topicId2: [part1, part2, part3] + * ... + * ] + * Hence, the total number of keys would be 1 x m x n (1 is for the groupId) where m is number of topicIds + * and n is number of partitions specified per topicId. + *

+ * For each RPC, we need to identify the coordinator node first. + * If the internal share state topic is not found in the metadata cache, when RPC is received + * we will need to make a FIND_COORDINATOR RPC which will have the side effect of creating the internal + * topic as well. If the node is found in the cache, we will use it directly. + * + * @return list of requests to send + */ + @Override + public Collection generateRequests() { + // There are two sources for requests here: + // 1. A queue which will contain FIND_CORD RPCs and other non-batchable RPCs. + // 2. A hashMap keyed on the share coordinator nodes which may contain batched requests. + + if (generateCallback != null) { + generateCallback.run(); + } + List requests = new ArrayList<>(); + + // honor queue first as find coordinator + // is mandatory for batching and sending the + // request to correct destination node + if (!queue.isEmpty()) { + PersisterStateManagerHandler handler = queue.peek(); + queue.poll(); + if (handler.lookupNeeded()) { + // we need to find the coordinator node + Node randomNode = randomNode(); + if (randomNode == Node.noNode()) { + log.error("Unable to find node to use for coordinator lookup."); + // fatal failure, cannot retry or progress + // fail the RPC + handler.findCoordinatorErrorResponse(Errors.COORDINATOR_NOT_AVAILABLE, Errors.COORDINATOR_NOT_AVAILABLE.exception()); + return Collections.emptyList(); + } + log.debug("Sending find coordinator RPC"); + return Collections.singletonList(new RequestAndCompletionHandler( + time.milliseconds(), + randomNode, + handler.findShareCoordinatorBuilder(), + handler + )); + } else { + // useful for tests and + // other RPCs which might not be batchable + if (!handler.isBatchable()) { + requests.add(new RequestAndCompletionHandler( + time.milliseconds(), + handler.coordinatorNode, + handler.requestBuilder(), + handler + )); + } + } + } + + // node1: { + // group1: { + // write: [w1, w2], + // read: [r1, r2], + // delete: [d1], + // summary: [s1] + // } + // group2: { + // write: [w3, w4] + // } + // } + // For a sequence of writes, the flow would be: + // 1. 1st write request arrives + // 2. it is enqueued in the send thread + // 3. wakeup event causes the generate requests to find the coordinator + // 4. it will cause either RPC or cache lookup + // 5. once complete, the write handler is added to the nodeMap for batching and not the queue + // 6. wakeup event causes generate requests to iterate over the map and send the write request (W1) and + // remove node from the nodeMap and add it to inFlight + // 7. until W1 completes, more write requests (W2, W3, ...) could come in and get added to the nodeMap as per point 3, 4, 5. + // 8. if these belong to same node as W1. They will not be sent as the membership test with inFlight will pass. + // 9. when W1 completes, it will clear inFlight and raise wakeup event. + // 10. at this point W2, W3, etc. could be sent as a combined request thus achieving batching. + final Map> sending = new HashMap<>(); + synchronized (nodeMapLock) { + nodeRPCMap.forEach((coordNode, rpcTypesPerNode) -> + rpcTypesPerNode.forEach((rpcType, groupsPerRpcType) -> + groupsPerRpcType.forEach((groupId, handlersPerGroup) -> { + // this condition causes requests of same type and same destination node + // to not be sent immediately but get batched + if (!inFlight.containsKey(rpcType) || !inFlight.get(rpcType).contains(coordNode)) { + AbstractRequest.Builder combinedRequestPerTypePerGroup = + RequestCoalescerHelper.coalesceRequests(groupId, rpcType, handlersPerGroup); + requests.add(new RequestAndCompletionHandler( + time.milliseconds(), + coordNode, + combinedRequestPerTypePerGroup, + response -> { + inFlight.computeIfPresent(rpcType, (key, oldVal) -> { + oldVal.remove(coordNode); + return oldVal; + }); + // now the combined request has completed + // we need to create responses for individual + // requests which composed the combined request + handlersPerGroup.forEach(handler1 -> handler1.onComplete(response)); + wakeup(); + })); + sending.computeIfAbsent(rpcType, key -> new HashSet<>()).add(coordNode); + } + }))); + + sending.forEach((rpcType, nodeSet) -> { + // we need to add these nodes to inFlight + inFlight.computeIfAbsent(rpcType, key -> new HashSet<>()).addAll(nodeSet); + + // remove from nodeMap + nodeSet.forEach(node -> nodeRPCMap.computeIfPresent(node, (nodeKey, oldRPCTypeSet) -> { + oldRPCTypeSet.remove(rpcType); + return oldRPCTypeSet; + })); + }); + } // close of synchronized context + + return requests; + } + + public void enqueue(PersisterStateManagerHandler handler) { + queue.add(handler); + wakeup(); + } + } + + private final class PersisterTimerTask extends TimerTask { + private final PersisterStateManagerHandler handler; + + PersisterTimerTask(long delayMs, PersisterStateManagerHandler handler) { + super(delayMs); + this.handler = handler; + } + + @Override + public void run() { + enqueue(handler); + sender.wakeup(); + } + } + + /** + * Util class which takes in builders of requests of the same type + * and returns a combined request of the same type. This is required for + * batching requests. + */ + private static class RequestCoalescerHelper { + public static AbstractRequest.Builder coalesceRequests(String groupId, RPCType rpcType, List handlers) { + switch (rpcType) { + case WRITE: + return coalesceWrites(groupId, handlers); + case READ: + return coalesceReads(groupId, handlers); + default: + throw new RuntimeException("Unknown rpc type: " + rpcType); + } + } + + private static AbstractRequest.Builder coalesceWrites(String groupId, List handlers) { + Map> partitionData = new HashMap<>(); + handlers.forEach(persHandler -> { + assert persHandler instanceof WriteStateHandler; + WriteStateHandler handler = (WriteStateHandler) persHandler; + partitionData.computeIfAbsent(handler.topicId, topicId -> new LinkedList<>()) + .add( + new WriteShareGroupStateRequestData.PartitionData() + .setPartition(handler.partition) + .setStateEpoch(handler.stateEpoch) + .setLeaderEpoch(handler.leaderEpoch) + .setStartOffset(handler.startOffset) + .setStateBatches(handler.batches.stream() + .map(batch -> new WriteShareGroupStateRequestData.StateBatch() + .setFirstOffset(batch.firstOffset()) + .setLastOffset(batch.lastOffset()) + .setDeliveryState(batch.deliveryState()) + .setDeliveryCount(batch.deliveryCount())) + .collect(Collectors.toList())) + ); + }); + + return new WriteShareGroupStateRequest.Builder(new WriteShareGroupStateRequestData() + .setGroupId(groupId) + .setTopics(partitionData.entrySet().stream() + .map(entry -> new WriteShareGroupStateRequestData.WriteStateData() + .setTopicId(entry.getKey()) + .setPartitions(entry.getValue())) + .collect(Collectors.toList()))); + } + + private static AbstractRequest.Builder coalesceReads(String groupId, List handlers) { + Map> partitionData = new HashMap<>(); + handlers.forEach(persHandler -> { + assert persHandler instanceof ReadStateHandler; + ReadStateHandler handler = (ReadStateHandler) persHandler; + partitionData.computeIfAbsent(handler.topicId, topicId -> new LinkedList<>()) + .add( + new ReadShareGroupStateRequestData.PartitionData() + .setPartition(handler.partition) + .setLeaderEpoch(handler.leaderEpoch) + ); + }); + + return new ReadShareGroupStateRequest.Builder(new ReadShareGroupStateRequestData() + .setGroupId(groupId) + .setTopics(partitionData.entrySet().stream() + .map(entry -> new ReadShareGroupStateRequestData.ReadStateData() + .setTopicId(entry.getKey()) + .setPartitions(entry.getValue())) + .collect(Collectors.toList()))); + } + } +} diff --git a/share/src/main/java/org/apache/kafka/server/share/persister/ShareCoordinatorMetadataCacheHelper.java b/share/src/main/java/org/apache/kafka/server/share/persister/ShareCoordinatorMetadataCacheHelper.java new file mode 100644 index 0000000000000..75e9cd629602e --- /dev/null +++ b/share/src/main/java/org/apache/kafka/server/share/persister/ShareCoordinatorMetadataCacheHelper.java @@ -0,0 +1,30 @@ +/* + * 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.server.share.persister; + +import org.apache.kafka.common.Node; + +import java.util.List; + +public interface ShareCoordinatorMetadataCacheHelper { + boolean containsTopic(String topic); + + Node getShareCoordinator(String key, String internalTopicName); + + List getClusterNodes(); +} diff --git a/share/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java b/share/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java new file mode 100644 index 0000000000000..f3c285cd5cab4 --- /dev/null +++ b/share/src/test/java/org/apache/kafka/server/share/persister/DefaultStatePersisterTest.java @@ -0,0 +1,581 @@ +/* + * 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.server.share.persister; + +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.FindCoordinatorResponseData; +import org.apache.kafka.common.message.ReadShareGroupStateRequestData; +import org.apache.kafka.common.message.ReadShareGroupStateResponseData; +import org.apache.kafka.common.message.WriteShareGroupStateRequestData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.requests.ReadShareGroupStateRequest; +import org.apache.kafka.common.requests.ReadShareGroupStateResponse; +import org.apache.kafka.common.requests.WriteShareGroupStateRequest; +import org.apache.kafka.common.requests.WriteShareGroupStateResponse; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.share.SharePartitionKey; +import org.apache.kafka.server.util.MockTime; +import org.apache.kafka.server.util.timer.MockTimer; +import org.apache.kafka.server.util.timer.Timer; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +class DefaultStatePersisterTest { + private static final KafkaClient CLIENT = mock(KafkaClient.class); + private static final Time MOCK_TIME = new MockTime(); + private static final Timer MOCK_TIMER = new MockTimer(); + private static final ShareCoordinatorMetadataCacheHelper CACHE_HELPER = mock(ShareCoordinatorMetadataCacheHelper.class); + + private static final String HOST = "localhost"; + private static final int PORT = 9092; + + private static class DefaultStatePersisterBuilder { + + private KafkaClient client = CLIENT; + private Time time = MOCK_TIME; + private Timer timer = MOCK_TIMER; + private ShareCoordinatorMetadataCacheHelper cacheHelper = CACHE_HELPER; + + private DefaultStatePersisterBuilder withKafkaClient(KafkaClient client) { + this.client = client; + return this; + } + + private DefaultStatePersisterBuilder withCacheHelper(ShareCoordinatorMetadataCacheHelper cacheHelper) { + this.cacheHelper = cacheHelper; + return this; + } + + private DefaultStatePersisterBuilder withTime(Time time) { + this.time = time; + return this; + } + + private DefaultStatePersisterBuilder withTimer(Timer timer) { + this.timer = timer; + return this; + } + + public static DefaultStatePersisterBuilder builder() { + return new DefaultStatePersisterBuilder(); + } + + public DefaultStatePersister build() { + PersisterStateManager persisterStateManager = new PersisterStateManager(client, cacheHelper, time, timer); + return new DefaultStatePersister(persisterStateManager); + } + } + + private ShareCoordinatorMetadataCacheHelper getDefaultCacheHelper(Node suppliedNode) { + return new ShareCoordinatorMetadataCacheHelper() { + @Override + public boolean containsTopic(String topic) { + return false; + } + + @Override + public Node getShareCoordinator(String key, String internalTopicName) { + return Node.noNode(); + } + + @Override + public List getClusterNodes() { + return Collections.singletonList(suppliedNode); + } + }; + } + + @Test + public void testWriteStateValidate() { + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 0; + int incorrectPartition = -1; + + // Request Parameters are null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.writeState(null); + }); + + // groupTopicPartitionData is null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.writeState(new WriteShareGroupStateParameters.Builder().setGroupTopicPartitionData(null).build()); + }); + + // groupId is null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.writeState(new WriteShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(null).build()).build()); + }); + + // topicsData is empty + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.writeState(new WriteShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.emptyList()).build()).build()); + }); + + // topicId is null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.writeState(new WriteShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.singletonList(new TopicData<>(null, + Collections.singletonList(PartitionFactory.newPartitionStateBatchData( + partition, 1, 0, 0, null)))) + ).build()).build()); + }); + + // partitionData is empty + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.writeState(new WriteShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.singletonList(new TopicData<>(topicId, + Collections.emptyList())) + ).build()).build()); + }); + + // partition value is incorrect + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.writeState(new WriteShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.singletonList(new TopicData<>(topicId, + Collections.singletonList(PartitionFactory.newPartitionStateBatchData( + incorrectPartition, 1, 0, 0, null)))) + ).build()).build()); + }); + } + + @Test + public void testReadStateValidate() { + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 0; + int incorrectPartition = -1; + + // Request Parameters are null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.readState(null); + }); + + // groupTopicPartitionData is null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.readState(new ReadShareGroupStateParameters.Builder().setGroupTopicPartitionData(null).build()); + }); + + // groupId is null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.readState(new ReadShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(null).build()).build()); + }); + + // topicsData is empty + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.readState(new ReadShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.emptyList()).build()).build()); + }); + + // topicId is null + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.readState(new ReadShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.singletonList(new TopicData<>(null, + Collections.singletonList(PartitionFactory.newPartitionIdLeaderEpochData( + partition, 1)))) + ).build()).build()); + }); + + // partitionData is empty + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.readState(new ReadShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.singletonList(new TopicData<>(topicId, + Collections.emptyList())) + ).build()).build()); + }); + + // partition value is incorrect + assertThrows(IllegalArgumentException.class, () -> { + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build(); + defaultStatePersister.readState(new ReadShareGroupStateParameters.Builder() + .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder() + .setGroupId(groupId) + .setTopicsData(Collections.singletonList(new TopicData<>(topicId, + Collections.singletonList(PartitionFactory.newPartitionIdLeaderEpochData( + incorrectPartition, 1)))) + ).build()).build()); + }); + } + + @Test + public void testWriteStateSuccess() { + + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId1 = Uuid.randomUuid(); + int partition1 = 10; + + Uuid topicId2 = Uuid.randomUuid(); + int partition2 = 8; + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode1 = new Node(5, HOST, PORT); + Node coordinatorNode2 = new Node(6, HOST, PORT); + + String coordinatorKey1 = SharePartitionKey.asCoordinatorKey(groupId, topicId1, partition1); + String coordinatorKey2 = SharePartitionKey.asCoordinatorKey(groupId, topicId2, partition2); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey1), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(5) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey2), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(6) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom( + body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId1 && requestPartition == partition1; + }, + new WriteShareGroupStateResponse(WriteShareGroupStateResponse.toResponseData(topicId1, partition1)), + coordinatorNode1); + + client.prepareResponseFrom( + body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId2 && requestPartition == partition2; + }, + new WriteShareGroupStateResponse(WriteShareGroupStateResponse.toResponseData(topicId2, partition2)), + coordinatorNode2); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder() + .withKafkaClient(client) + .withCacheHelper(cacheHelper) + .build(); + + WriteShareGroupStateParameters request = WriteShareGroupStateParameters.from( + new WriteShareGroupStateRequestData() + .setGroupId(groupId) + .setTopics(Arrays.asList( + new WriteShareGroupStateRequestData.WriteStateData() + .setTopicId(topicId1) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateRequestData.PartitionData() + .setPartition(partition1) + .setStateEpoch(0) + .setLeaderEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.singletonList(new WriteShareGroupStateRequestData.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))) + )), + new WriteShareGroupStateRequestData.WriteStateData() + .setTopicId(topicId2) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateRequestData.PartitionData() + .setPartition(partition2) + .setStateEpoch(0) + .setLeaderEpoch(1) + .setStartOffset(0) + .setStateBatches(Arrays.asList( + new WriteShareGroupStateRequestData.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0), + new WriteShareGroupStateRequestData.StateBatch() + .setFirstOffset(11) + .setLastOffset(20) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))) + )) + )) + ); + + CompletableFuture resultFuture = defaultStatePersister.writeState(request); + + WriteShareGroupStateResult result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Unexpected exception", e); + } + + HashSet resultMap = new HashSet<>(); + result.topicsData().forEach( + topicData -> topicData.partitions().forEach( + partitionData -> resultMap.add((PartitionData) partitionData) + ) + ); + + + HashSet expectedResultMap = new HashSet<>(); + expectedResultMap.add((PartitionData) PartitionFactory.newPartitionErrorData(partition1, Errors.NONE.code(), null)); + + expectedResultMap.add((PartitionData) PartitionFactory.newPartitionErrorData(partition2, Errors.NONE.code(), null)); + + assertEquals(2, result.topicsData().size()); + assertEquals(expectedResultMap, resultMap); + } + + @Test + public void testReadStateSuccess() { + + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId1 = Uuid.randomUuid(); + int partition1 = 10; + + Uuid topicId2 = Uuid.randomUuid(); + int partition2 = 8; + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode1 = new Node(5, HOST, PORT); + Node coordinatorNode2 = new Node(6, HOST, PORT); + + String coordinatorKey1 = SharePartitionKey.asCoordinatorKey(groupId, topicId1, partition1); + String coordinatorKey2 = SharePartitionKey.asCoordinatorKey(groupId, topicId2, partition2); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey1), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(5) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey2), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(6) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom( + body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId1 && requestPartition == partition1; + }, + new ReadShareGroupStateResponse(ReadShareGroupStateResponse.toResponseData(topicId1, partition1, 0, 1, + Collections.singletonList(new ReadShareGroupStateResponseData.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0)))), + coordinatorNode1); + + client.prepareResponseFrom( + body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId2 && requestPartition == partition2; + }, + new ReadShareGroupStateResponse(ReadShareGroupStateResponse.toResponseData(topicId2, partition2, 0, 1, + Arrays.asList(new ReadShareGroupStateResponseData.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0), + new ReadShareGroupStateResponseData.StateBatch() + .setFirstOffset(11) + .setLastOffset(20) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0)))), + coordinatorNode2); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder() + .withKafkaClient(client) + .withCacheHelper(cacheHelper) + .build(); + + ReadShareGroupStateParameters request = ReadShareGroupStateParameters.from( + new ReadShareGroupStateRequestData() + .setGroupId(groupId) + .setTopics(Arrays.asList( + new ReadShareGroupStateRequestData.ReadStateData() + .setTopicId(topicId1) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateRequestData.PartitionData() + .setPartition(partition1) + .setLeaderEpoch(1) + )), + new ReadShareGroupStateRequestData.ReadStateData() + .setTopicId(topicId2) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateRequestData.PartitionData() + .setPartition(partition2) + .setLeaderEpoch(1) + )) + )) + ); + + CompletableFuture resultFuture = defaultStatePersister.readState(request); + + ReadShareGroupStateResult result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Unexpected exception", e); + } + + HashSet resultMap = new HashSet<>(); + result.topicsData().forEach( + topicData -> topicData.partitions().forEach( + partitionData -> resultMap.add((PartitionData) partitionData) + ) + ); + + HashSet expectedResultMap = new HashSet<>(); + expectedResultMap.add( + (PartitionData) PartitionFactory.newPartitionAllData(partition1, 1, 0, Errors.NONE.code(), + null, Collections.singletonList(new PersisterStateBatch(0, 10, (byte) 0, (short) 1) + ))); + + expectedResultMap.add( + (PartitionData) PartitionFactory.newPartitionAllData(partition2, 1, 0, Errors.NONE.code(), + null, Arrays.asList( + new PersisterStateBatch(0, 10, (byte) 0, (short) 1), + new PersisterStateBatch(11, 20, (byte) 0, (short) 1) + ))); + + assertEquals(2, result.topicsData().size()); + assertEquals(expectedResultMap, resultMap); + } + + @Test + public void testDefaultPersisterClose() { + PersisterStateManager psm = mock(PersisterStateManager.class); + DefaultStatePersister dsp = new DefaultStatePersister(psm); + try { + verify(psm, times(0)).stop(); + + dsp.stop(); + + verify(psm, times(1)).stop(); + } catch (Exception e) { + fail("Unexpected exception", e); + } + } +} diff --git a/share/src/test/java/org/apache/kafka/server/share/persister/PersisterStateManagerTest.java b/share/src/test/java/org/apache/kafka/server/share/persister/PersisterStateManagerTest.java new file mode 100644 index 0000000000000..edcebd0d3d232 --- /dev/null +++ b/share/src/test/java/org/apache/kafka/server/share/persister/PersisterStateManagerTest.java @@ -0,0 +1,2075 @@ +/* + * 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.server.share.persister; + +import org.apache.kafka.clients.ClientResponse; +import org.apache.kafka.clients.KafkaClient; +import org.apache.kafka.clients.MockClient; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.message.FindCoordinatorResponseData; +import org.apache.kafka.common.message.ReadShareGroupStateResponseData; +import org.apache.kafka.common.message.WriteShareGroupStateResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.requests.ReadShareGroupStateRequest; +import org.apache.kafka.common.requests.ReadShareGroupStateResponse; +import org.apache.kafka.common.requests.WriteShareGroupStateRequest; +import org.apache.kafka.common.requests.WriteShareGroupStateResponse; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.share.SharePartitionKey; +import org.apache.kafka.server.util.MockTime; +import org.apache.kafka.server.util.timer.MockTimer; +import org.apache.kafka.server.util.timer.SystemTimer; +import org.apache.kafka.server.util.timer.SystemTimerReaper; +import org.apache.kafka.server.util.timer.Timer; +import org.apache.kafka.test.TestUtils; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +class PersisterStateManagerTest { + private static final KafkaClient CLIENT = mock(KafkaClient.class); + private static final Time MOCK_TIME = new MockTime(); + private static final Timer MOCK_TIMER = new MockTimer((MockTime) MOCK_TIME); + private static final ShareCoordinatorMetadataCacheHelper CACHE_HELPER = mock(ShareCoordinatorMetadataCacheHelper.class); + private static final int MAX_RPC_RETRY_ATTEMPTS = 5; + public static final long REQUEST_BACKOFF_MS = 100L; + public static final long REQUEST_BACKOFF_MAX_MS = 3000L; + + private static final String HOST = "localhost"; + private static final int PORT = 9092; + + private static class PersisterStateManagerBuilder { + + private KafkaClient client = CLIENT; + private Time time = MOCK_TIME; + private Timer timer = MOCK_TIMER; + private ShareCoordinatorMetadataCacheHelper cacheHelper = CACHE_HELPER; + + private PersisterStateManagerBuilder withKafkaClient(KafkaClient client) { + this.client = client; + return this; + } + + private PersisterStateManagerBuilder withCacheHelper(ShareCoordinatorMetadataCacheHelper cacheHelper) { + this.cacheHelper = cacheHelper; + return this; + } + + private PersisterStateManagerBuilder withTime(Time time) { + this.time = time; + return this; + } + + private PersisterStateManagerBuilder withTimer(Timer timer) { + this.timer = timer; + return this; + } + + public static PersisterStateManagerBuilder builder() { + return new PersisterStateManagerBuilder(); + } + + public PersisterStateManager build() { + return new PersisterStateManager(client, cacheHelper, time, timer); + } + } + + private abstract class TestStateHandler extends PersisterStateManager.PersisterStateManagerHandler { + private final CompletableFuture result; + + private class TestHandlerResponseData extends WriteShareGroupStateResponseData { + } + + private class TestHandlerResponse extends WriteShareGroupStateResponse { + public TestHandlerResponse(WriteShareGroupStateResponseData data) { + super(data); + } + } + + TestStateHandler( + PersisterStateManager stateManager, + String groupId, + Uuid topicId, + int partition, + CompletableFuture result, + long backoffMs, + long backoffMaxMs, + int maxFindCoordAttempts) { + stateManager.super(groupId, topicId, partition, backoffMs, backoffMaxMs, maxFindCoordAttempts); + this.result = result; + } + + @Override + protected void handleRequestResponse(ClientResponse response) { + this.result.complete(new TestHandlerResponse(new TestHandlerResponseData() + .setResults(Collections.singletonList(new WriteShareGroupStateResponseData.WriteStateResult() + .setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorMessage(Errors.NONE.message()) + .setErrorCode(Errors.NONE.code())) + ) + )) + )); + } + + @Override + protected boolean isResponseForRequest(ClientResponse response) { + return true; + } + + @Override + protected void findCoordinatorErrorResponse(Errors error, Exception exception) { + this.result.complete(new TestHandlerResponse(new TestHandlerResponseData() + .setResults(Collections.singletonList(new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorMessage(exception == null ? error.message() : exception.getMessage()) + .setErrorCode(error.code())) + ) + )) + )); + } + + @Override + protected String name() { + return "TestStateHandler"; + } + + @Override + protected boolean isBatchable() { + return false; + } + + @Override + protected PersisterStateManager.RPCType rpcType() { + return PersisterStateManager.RPCType.UNKNOWN; + } + + @Override + protected CompletableFuture result() { + return this.result; + } + } + + private ShareCoordinatorMetadataCacheHelper getDefaultCacheHelper(Node suppliedNode) { + return new ShareCoordinatorMetadataCacheHelper() { + @Override + public boolean containsTopic(String topic) { + return false; + } + + @Override + public Node getShareCoordinator(String key, String internalTopicName) { + return Node.noNode(); + } + + @Override + public List getClusterNodes() { + return Collections.singletonList(suppliedNode); + } + }; + } + + private ShareCoordinatorMetadataCacheHelper getCoordinatorCacheHelper(Node coordinatorNode) { + return new ShareCoordinatorMetadataCacheHelper() { + @Override + public boolean containsTopic(String topic) { + return true; + } + + @Override + public Node getShareCoordinator(String key, String internalTopicName) { + return coordinatorNode; + } + + @Override + public List getClusterNodes() { + return Collections.emptyList(); + } + }; + } + + private static Timer mockTimer; + + @BeforeEach + public void setUp() { + mockTimer = new SystemTimerReaper("persisterStateManagerTestTimer", + new SystemTimer("persisterStateManagerTestTimer")); + } + + @AfterEach + public void tearDown() throws Exception { + Utils.closeQuietly(mockTimer, "persisterStateManagerTestTimer"); + } + + @Test + public void testFindCoordinatorFatalError() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node suppliedNode = new Node(0, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setKey(coordinatorKey) + .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) + .setHost(Node.noNode().host()) + .setNodeId(Node.noNode().id()) + .setPort(Node.noNode().port()) + )) + ), + suppliedNode + ); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + TestStateHandler handler = spy(new TestStateHandler( + stateManager, + groupId, + topicId, + partition, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS + ) { + @Override + protected AbstractRequest.Builder requestBuilder() { + return null; + } + }); + + stateManager.enqueue(handler); + + TestStateHandler.TestHandlerResponse result = null; + try { + result = handler.result().get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + assertEquals(Errors.UNKNOWN_SERVER_ERROR.code(), result.data().results().get(0).partitions().get(0).errorCode()); + verify(handler, times(1)).findShareCoordinatorBuilder(); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testFindCoordinatorAttemptsExhausted() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node suppliedNode = new Node(0, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setKey(coordinatorKey) + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setHost(Node.noNode().host()) + .setNodeId(Node.noNode().id()) + .setPort(Node.noNode().port()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setKey(coordinatorKey) + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setHost(Node.noNode().host()) + .setNodeId(Node.noNode().id()) + .setPort(Node.noNode().port()) + )) + ), + suppliedNode + ); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + int maxAttempts = 2; + + TestStateHandler handler = spy(new TestStateHandler( + stateManager, + groupId, + topicId, + partition, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + maxAttempts + ) { + @Override + protected AbstractRequest.Builder requestBuilder() { + return null; + } + }); + + stateManager.enqueue(handler); + + TestStateHandler.TestHandlerResponse result = null; + try { + result = handler.result.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), result.data().results().get(0).partitions().get(0).errorCode()); + verify(handler, times(2)).findShareCoordinatorBuilder(); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testFindCoordinatorSuccess() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + verify(handler, times(1)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testWriteStateRequestCoordinatorFoundSuccessfully() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + List stateBatches = Arrays.asList( + new PersisterStateBatch(0, 9, (byte) 0, (short) 1), + new PersisterStateBatch(10, 19, (byte) 1, (short) 1) + ); + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.WriteStateHandler handler = spy(stateManager.new WriteStateHandler( + groupId, + topicId, + partition, + 0, + 0, + 0, + stateBatches, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + WriteShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + WriteShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(1)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testWriteStateRequestFailButCoordinatorSuccess() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + List stateBatches = Arrays.asList( + new PersisterStateBatch(0, 9, (byte) 0, (short) 1), + new PersisterStateBatch(10, 19, (byte) 1, (short) 1) + ); + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NOT_COORDINATOR.code()) + .setErrorMessage(Errors.NOT_COORDINATOR.message()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.WriteStateHandler handler = spy(stateManager.new WriteStateHandler( + groupId, + topicId, + partition, + 0, + 0, + 0, + stateBatches, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + WriteShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + WriteShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(1)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned is correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NOT_COORDINATOR.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testWriteStateRequestCoordinatorFoundOnRetry() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + List stateBatches = Arrays.asList( + new PersisterStateBatch(0, 9, (byte) 0, (short) 1), + new PersisterStateBatch(10, 19, (byte) 1, (short) 1) + ); + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.WriteStateHandler handler = spy(stateManager.new WriteStateHandler( + groupId, + topicId, + partition, + 0, + 0, + 0, + stateBatches, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + WriteShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + WriteShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(2)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testWriteStateRequestWithCoordinatorNodeLookup() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + List stateBatches = Arrays.asList( + new PersisterStateBatch(0, 9, (byte) 0, (short) 1), + new PersisterStateBatch(10, 19, (byte) 1, (short) 1) + ); + + Node coordinatorNode = new Node(1, HOST, PORT); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getCoordinatorCacheHelper(coordinatorNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.WriteStateHandler handler = spy(stateManager.new WriteStateHandler( + groupId, + topicId, + partition, + 0, + 0, + 0, + stateBatches, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + WriteShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + WriteShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(0)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + verify(handler, times(1)).onComplete(any()); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testWriteStateRequestWithRetryAndCoordinatorNodeLookup() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + List stateBatches = Arrays.asList( + new PersisterStateBatch(0, 9, (byte) 0, (short) 1), + new PersisterStateBatch(10, 19, (byte) 1, (short) 1) + ); + + Node coordinatorNode = new Node(1, HOST, PORT); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getCoordinatorCacheHelper(coordinatorNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.WriteStateHandler handler = spy(stateManager.new WriteStateHandler( + groupId, + topicId, + partition, + 0, + 0, + 0, + stateBatches, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + WriteShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + WriteShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(0)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + verify(handler, times(2)).onComplete(any()); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testWriteStateRequestFailedMaxRetriesExhausted() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + List stateBatches = Arrays.asList( + new PersisterStateBatch(0, 9, (byte) 0, (short) 1), + new PersisterStateBatch(10, 19, (byte) 1, (short) 1) + ); + + Node coordinatorNode = new Node(1, HOST, PORT); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getCoordinatorCacheHelper(coordinatorNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.WriteStateHandler handler = spy(stateManager.new WriteStateHandler( + groupId, + topicId, + partition, + 0, + 0, + 0, + stateBatches, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + 2 + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + WriteShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + WriteShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(0)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + verify(handler, times(2)).onComplete(any()); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testWriteStateRequestBatchingWithCoordinatorNodeLookup() throws ExecutionException, Exception { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + List stateBatches = Arrays.asList( + new PersisterStateBatch(0, 9, (byte) 0, (short) 1), + new PersisterStateBatch(10, 19, (byte) 1, (short) 1) + ); + + Node coordinatorNode = new Node(1, HOST, PORT); + + client.prepareResponseFrom(body -> { + WriteShareGroupStateRequest request = (WriteShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new WriteShareGroupStateResponse( + new WriteShareGroupStateResponseData() + .setResults(Collections.singletonList( + new WriteShareGroupStateResponseData.WriteStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new WriteShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getCoordinatorCacheHelper(coordinatorNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + AtomicBoolean isBatchingSuccess = new AtomicBoolean(false); + stateManager.setGenerateCallback(() -> { + Map>> handlersPerType = stateManager.nodeRPCMap().get(coordinatorNode); + if (handlersPerType != null && handlersPerType.containsKey(PersisterStateManager.RPCType.WRITE) && handlersPerType.get(PersisterStateManager.RPCType.WRITE).containsKey(groupId)) { + if (handlersPerType.get(PersisterStateManager.RPCType.WRITE).get(groupId).size() > 2) + isBatchingSuccess.set(true); + } + }); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + List handlers = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + PersisterStateManager.WriteStateHandler handler = spy(stateManager.new WriteStateHandler( + groupId, + topicId, + partition, + 0, + 0, + 0, + stateBatches, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS + )); + handlers.add(handler); + stateManager.enqueue(handler); + } + + CompletableFuture.allOf(handlers.stream() + .map(PersisterStateManager.WriteStateHandler::result).toArray(CompletableFuture[]::new)).get(); + + TestUtils.waitForCondition(isBatchingSuccess::get, TestUtils.DEFAULT_MAX_WAIT_MS, 10L, () -> "unable to verify batching"); + } + + @Test + public void testReadStateRequestCoordinatorFoundSuccessfully() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + ReadShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(1)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + assertEquals(1, partitionResult.stateEpoch()); + assertEquals(0, partitionResult.startOffset()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testReadStateRequestIllegalStateCoordinatorFoundSuccessfully() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(Uuid.randomUuid()) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(500) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + ReadShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(1)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(Errors.UNKNOWN_SERVER_ERROR.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testReadStateRequestFailButCoordinatorFoundSuccessfully() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NOT_COORDINATOR.code()) + .setErrorMessage(Errors.NOT_COORDINATOR.message()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + ReadShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(1)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NOT_COORDINATOR.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testReadStateRequestCoordinatorFoundOnRetry() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node suppliedNode = new Node(0, HOST, PORT); + Node coordinatorNode = new Node(1, HOST, PORT); + + String coordinatorKey = SharePartitionKey.asCoordinatorKey(groupId, topicId, partition); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> body instanceof FindCoordinatorRequest + && ((FindCoordinatorRequest) body).data().keyType() == FindCoordinatorRequest.CoordinatorType.SHARE.id() + && ((FindCoordinatorRequest) body).data().coordinatorKeys().get(0).equals(coordinatorKey), + new FindCoordinatorResponse( + new FindCoordinatorResponseData() + .setCoordinators(Collections.singletonList( + new FindCoordinatorResponseData.Coordinator() + .setNodeId(1) + .setHost(HOST) + .setPort(PORT) + .setErrorCode(Errors.NONE.code()) + )) + ), + suppliedNode + ); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getDefaultCacheHelper(suppliedNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + ReadShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(2)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + + // Verifying the coordinator node was populated correctly by the FIND_COORDINATOR request + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + assertEquals(1, partitionResult.stateEpoch()); + assertEquals(0, partitionResult.startOffset()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testReadStateRequestWithCoordinatorNodeLookup() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node coordinatorNode = new Node(1, HOST, PORT); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getCoordinatorCacheHelper(coordinatorNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + ReadShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(0)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + verify(handler, times(1)).onComplete(any()); + + // Verifying the coordinator node was populated correctly by the constructor + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + assertEquals(1, partitionResult.stateEpoch()); + assertEquals(0, partitionResult.startOffset()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testReadStateRequestRetryWithCoordinatorNodeLookup() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node coordinatorNode = new Node(1, HOST, PORT); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.NONE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getCoordinatorCacheHelper(coordinatorNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + MAX_RPC_RETRY_ATTEMPTS, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + ReadShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(0)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + verify(handler, times(2)).onComplete(any()); + + // Verifying the coordinator node was populated correctly by the constructor + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.NONE.code(), partitionResult.errorCode()); + assertEquals(1, partitionResult.stateEpoch()); + assertEquals(0, partitionResult.startOffset()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testReadStateRequestFailureMaxRetriesExhausted() { + MockClient client = new MockClient(MOCK_TIME); + + String groupId = "group1"; + Uuid topicId = Uuid.randomUuid(); + int partition = 10; + + Node coordinatorNode = new Node(1, HOST, PORT); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + client.prepareResponseFrom(body -> { + ReadShareGroupStateRequest request = (ReadShareGroupStateRequest) body; + String requestGroupId = request.data().groupId(); + Uuid requestTopicId = request.data().topics().get(0).topicId(); + int requestPartition = request.data().topics().get(0).partitions().get(0).partition(); + + return requestGroupId.equals(groupId) && requestTopicId == topicId && requestPartition == partition; + }, new ReadShareGroupStateResponse( + new ReadShareGroupStateResponseData() + .setResults(Collections.singletonList( + new ReadShareGroupStateResponseData.ReadStateResult() + .setTopicId(topicId) + .setPartitions(Collections.singletonList( + new ReadShareGroupStateResponseData.PartitionResult() + .setPartition(partition) + .setErrorCode(Errors.COORDINATOR_LOAD_IN_PROGRESS.code()) + .setErrorMessage("") + .setStateEpoch(1) + .setStartOffset(0) + .setStateBatches(Collections.emptyList()) + )) + )) + ), coordinatorNode); + + ShareCoordinatorMetadataCacheHelper cacheHelper = getCoordinatorCacheHelper(coordinatorNode); + + PersisterStateManager stateManager = PersisterStateManagerBuilder.builder() + .withKafkaClient(client) + .withTimer(mockTimer) + .withCacheHelper(cacheHelper) + .build(); + + stateManager.start(); + + CompletableFuture future = new CompletableFuture<>(); + + PersisterStateManager.ReadStateHandler handler = spy(stateManager.new ReadStateHandler( + groupId, + topicId, + partition, + 0, + future, + REQUEST_BACKOFF_MS, + REQUEST_BACKOFF_MAX_MS, + 2, + null + )); + + stateManager.enqueue(handler); + + CompletableFuture resultFuture = handler.result(); + + ReadShareGroupStateResponse result = null; + try { + result = resultFuture.get(); + } catch (Exception e) { + fail("Failed to get result from future", e); + } + + ReadShareGroupStateResponseData.PartitionResult partitionResult = result.data().results().get(0).partitions().get(0); + + verify(handler, times(0)).findShareCoordinatorBuilder(); + verify(handler, times(0)).requestBuilder(); + verify(handler, times(2)).onComplete(any()); + + // Verifying the coordinator node was populated correctly by the constructor + assertEquals(coordinatorNode, handler.getCoordinatorNode()); + + // Verifying the result returned in correct + assertEquals(partition, partitionResult.partition()); + assertEquals(Errors.COORDINATOR_LOAD_IN_PROGRESS.code(), partitionResult.errorCode()); + + try { + // Stopping the state manager + stateManager.stop(); + } catch (Exception e) { + fail("Failed to stop state manager", e); + } + } + + @Test + public void testPersisterStateManagerClose() { + KafkaClient client = mock(KafkaClient.class); + Timer timer = mock(Timer.class); + PersisterStateManager psm = PersisterStateManagerBuilder + .builder() + .withTimer(timer) + .withKafkaClient(client) + .build(); + + try { + verify(client, times(0)).close(); + verify(timer, times(0)).close(); + + psm.start(); + psm.stop(); + + verify(client, times(1)).close(); + verify(timer, times(1)).close(); + } catch (Exception e) { + fail("unexpected exception", e); + } + } +}