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 extends PartitionIdData> data) {
+ String groupId = data.groupId();
+ if (groupId == null || groupId.isEmpty()) {
+ throw new IllegalArgumentException(prefix + " groupId cannot be null or empty.");
+ }
+
+ List extends TopicData extends PartitionIdData>> topicsData = data.topicsData();
+ if (isEmpty(topicsData)) {
+ throw new IllegalArgumentException(prefix + " topics data cannot be null or empty.");
+ }
+
+ for (TopicData extends PartitionIdData> 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 extends AbstractRequest> 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 extends AbstractResponse> 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 extends AbstractRequest> 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 extends AbstractRequest> 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 extends AbstractRequest> coalesceRequests(String groupId, RPCType rpcType, List extends PersisterStateManagerHandler> 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 extends AbstractRequest> coalesceWrites(String groupId, List extends PersisterStateManagerHandler> 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 extends AbstractRequest> coalesceReads(String groupId, List extends PersisterStateManagerHandler> 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 extends AbstractRequest> 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 extends AbstractRequest> 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);
+ }
+ }
+}