KAFKA-17329: DefaultStatePersister implementation (#17270)

Adds the DefaultStatePersister and other supporting classes for managing share state.

* Added DefaultStatePersister implementation. This is the entry point for callers who wish to invoke the share state RPC API.
* Added PersisterStateManager which is used by DefaultStatePersister to manage and send the RPCs over the network.
* Added code to BrokerServer and BrokerMetadataPublisher to instantiate the appropriate persister based on the config value for group.share.persister.class.name. If this is not specified, the DefaultStatePersister will be used. To force use of NoOpStatePersister, set the config to empty. This is an internal config, not to be exposed to the end user. This will be used to factory plug the appropriate persister.
* Using this persister, the internal __share_group_state topic will come to life and will be used for persistence of share group info.

Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>
This commit is contained in:
Sushant Mahajan 2024-10-28 23:41:04 +05:30 committed by GitHub
parent 14a9130f6f
commit 5f92f60bff
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
24 changed files with 4898 additions and 310 deletions

View File

@ -995,9 +995,13 @@ project(':share') {
dependencies {
implementation project(':server-common')
testImplementation project(':clients').sourceSets.test.output
testImplementation project(':server-common').sourceSets.test.output
implementation libs.slf4jApi
testImplementation libs.junitJupiter
testImplementation libs.mockitoCore
testImplementation libs.slf4jReload4j
testRuntimeOnly libs.junitPlatformLanucher

View File

@ -26,6 +26,7 @@
<allow pkg="java" />
<allow pkg="org.slf4j" />
<allow pkg="org.junit" />
<allow pkg="org.mockito" />
<!-- no one depends on the server -->
<disallow pkg="kafka" />
@ -47,6 +48,13 @@
<subpackage name="fetch">
<allow class="org.apache.kafka.server.storage.log.FetchParams"/>
</subpackage>
<subpackage name="persister">
<allow pkg="org.apache.kafka.clients" />
<allow pkg="org.apache.kafka.common.internals" />
<allow pkg="org.apache.kafka.server.util" />
<allow pkg="org.apache.kafka.test" />
</subpackage>
</subpackage>
</subpackage>

View File

@ -23,7 +23,7 @@ import kafka.log.LogManager
import kafka.log.remote.RemoteLogManager
import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.raft.KafkaRaftManager
import kafka.server.metadata.{AclPublisher, BrokerMetadataPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, ScramPublisher}
import kafka.server.metadata.{AclPublisher, BrokerMetadataPublisher, ClientQuotaMetadataManager, DelegationTokenPublisher, DynamicClientQuotaPublisher, DynamicConfigPublisher, KRaftMetadataCache, ScramPublisher, ShareCoordinatorMetadataCacheHelperImpl}
import kafka.server.share.SharePartitionManager
import kafka.utils.CoreUtils
import org.apache.kafka.common.config.ConfigException
@ -46,7 +46,7 @@ import org.apache.kafka.server.{AssignmentsManager, BrokerFeatures, ClientMetric
import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.{ApiMessageAndVersion, DirectoryEventHandler, TopicIdPartition}
import org.apache.kafka.server.config.ConfigType
import org.apache.kafka.server.share.persister.{NoOpShareStatePersister, Persister}
import org.apache.kafka.server.share.persister.{DefaultStatePersister, NoOpShareStatePersister, Persister, PersisterStateManager}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, KafkaYammerMetrics}
import org.apache.kafka.server.network.{EndpointReadyFutures, KafkaAuthorizerServerInfo}
@ -124,7 +124,7 @@ class BrokerServer(
var transactionCoordinator: TransactionCoordinator = _
var shareCoordinator: Option[ShareCoordinator] = _
var shareCoordinator: Option[ShareCoordinator] = None
var clientToControllerChannelManager: NodeToControllerChannelManager = _
@ -356,8 +356,12 @@ class BrokerServer(
/* initializing the groupConfigManager */
groupConfigManager = new GroupConfigManager(config.groupCoordinatorConfig.extractGroupConfigMap(config.shareGroupConfig))
/* create share coordinator */
shareCoordinator = createShareCoordinator()
/* create persister */
persister = createShareStatePersister()
groupCoordinator = createGroupCoordinator()
val producerIdManagerSupplier = () => ProducerIdManager.rpc(
@ -423,8 +427,6 @@ class BrokerServer(
config.shareGroupConfig.shareGroupMaxGroups * config.groupCoordinatorConfig.shareGroupMaxSize,
KafkaServer.MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS)
persister = NoOpShareStatePersister.getInstance()
sharePartitionManager = new SharePartitionManager(
replicaManager,
time,
@ -435,7 +437,7 @@ class BrokerServer(
config.shareGroupConfig.shareFetchMaxFetchRecords,
persister,
groupConfigManager,
new Metrics()
metrics
)
// Create the request processor objects.
@ -646,33 +648,68 @@ class BrokerServer(
}
private def createShareCoordinator(): Option[ShareCoordinator] = {
if (!config.shareGroupConfig.isShareGroupEnabled) {
return None
}
val time = Time.SYSTEM
val timer = new SystemTimerReaper(
"share-coordinator-reaper",
new SystemTimer("share-coordinator")
)
if (config.shareGroupConfig.isShareGroupEnabled &&
config.shareGroupConfig.shareGroupPersisterClassName().nonEmpty) {
val time = Time.SYSTEM
val timer = new SystemTimerReaper(
"share-coordinator-reaper",
new SystemTimer("share-coordinator")
)
val serde = new ShareCoordinatorRecordSerde
val loader = new CoordinatorLoaderImpl[CoordinatorRecord](
time,
replicaManager,
serde,
config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize()
)
val writer = new CoordinatorPartitionWriter(
replicaManager
)
Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig)
.withTimer(timer)
.withTime(time)
.withLoader(loader)
.withWriter(writer)
.withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics))
.withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics))
.build())
val serde = new ShareCoordinatorRecordSerde
val loader = new CoordinatorLoaderImpl[CoordinatorRecord](
time,
replicaManager,
serde,
config.shareCoordinatorConfig.shareCoordinatorLoadBufferSize()
)
val writer = new CoordinatorPartitionWriter(
replicaManager
)
Some(new ShareCoordinatorService.Builder(config.brokerId, config.shareCoordinatorConfig)
.withTimer(timer)
.withTime(time)
.withLoader(loader)
.withWriter(writer)
.withCoordinatorRuntimeMetrics(new ShareCoordinatorRuntimeMetrics(metrics))
.withCoordinatorMetrics(new ShareCoordinatorMetrics(metrics))
.build())
} else {
None
}
}
private def createShareStatePersister(): Persister = {
if (config.shareGroupConfig.isShareGroupEnabled &&
config.shareGroupConfig.shareGroupPersisterClassName.nonEmpty) {
val klass = Utils.loadClass(config.shareGroupConfig.shareGroupPersisterClassName, classOf[Object]).asInstanceOf[Class[Persister]]
if (klass.getName.equals(classOf[DefaultStatePersister].getName)) {
klass.getConstructor(classOf[PersisterStateManager])
.newInstance(
new PersisterStateManager(
NetworkUtils.buildNetworkClient("Persister", config, metrics, Time.SYSTEM, new LogContext(s"[Persister broker=${config.brokerId}]")),
new ShareCoordinatorMetadataCacheHelperImpl(metadataCache, key => shareCoordinator.get.partitionFor(key), config.interBrokerListenerName),
Time.SYSTEM,
new SystemTimerReaper(
"persister-state-manager-reaper",
new SystemTimer("persister")
)
)
)
} else if (klass.getName.equals(classOf[NoOpShareStatePersister].getName)) {
info("Using no op persister")
new NoOpShareStatePersister()
} else {
error("Unknown persister specified. Persister is only factory pluggable!")
throw new IllegalArgumentException("Unknown persiser specified " + config.shareGroupConfig.shareGroupPersisterClassName)
}
} else {
// in case share coordinator not enabled or
// persister class name deliberately empty (key=)
info("Using no op persister")
new NoOpShareStatePersister()
}
}
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
@ -778,9 +815,13 @@ class BrokerServer(
if (socketServer != null)
CoreUtils.swallow(socketServer.shutdown(), this)
Utils.closeQuietly(brokerTopicStats, "broker topic stats")
Utils.closeQuietly(sharePartitionManager, "share partition manager")
if (persister != null)
CoreUtils.swallow(persister.stop(), this)
isShuttingDown.set(false)
CoreUtils.swallow(lifecycleManager.close(), this)

View File

@ -216,6 +216,14 @@ class BrokerMetadataPublisher(
s"coordinator with local changes in $deltaName", t)
}
try {
// Propagate the new image to the share coordinator.
shareCoordinator.foreach(coordinator => coordinator.onNewMetadataImage(newImage, delta))
} catch {
case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " +
s"coordinator with local changes in $deltaName", t)
}
if (_firstPublish) {
finishInitializingReplicaManager()
}

View File

@ -0,0 +1,102 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server.metadata;
import kafka.server.MetadataCache;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.message.MetadataResponseData;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.MetadataResponse;
import org.apache.kafka.server.share.persister.ShareCoordinatorMetadataCacheHelper;
import java.util.HashSet;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import scala.jdk.javaapi.CollectionConverters;
import scala.jdk.javaapi.OptionConverters;
public class ShareCoordinatorMetadataCacheHelperImpl implements ShareCoordinatorMetadataCacheHelper {
private final MetadataCache metadataCache;
private final Function<String, Integer> keyToPartitionMapper;
private final ListenerName interBrokerListenerName;
public ShareCoordinatorMetadataCacheHelperImpl(
MetadataCache metadataCache,
Function<String, Integer> keyToPartitionMapper,
ListenerName interBrokerListenerName
) {
Objects.requireNonNull(metadataCache, "metadataCache must not be null");
Objects.requireNonNull(keyToPartitionMapper, "keyToPartitionMapper must not be null");
Objects.requireNonNull(interBrokerListenerName, "interBrokerListenerName must not be null");
this.metadataCache = metadataCache;
this.keyToPartitionMapper = keyToPartitionMapper;
this.interBrokerListenerName = interBrokerListenerName;
}
@Override
public boolean containsTopic(String topic) {
return metadataCache.contains(topic);
}
@Override
public Node getShareCoordinator(String key, String internalTopicName) {
if (metadataCache.contains(internalTopicName)) {
Set<String> topicSet = new HashSet<>();
topicSet.add(internalTopicName);
List<MetadataResponseData.MetadataResponseTopic> topicMetadata = CollectionConverters.asJava(
metadataCache.getTopicMetadata(
CollectionConverters.asScala(topicSet),
interBrokerListenerName,
false,
false
)
);
if (topicMetadata == null || topicMetadata.isEmpty() || topicMetadata.get(0).errorCode() != Errors.NONE.code()) {
return Node.noNode();
} else {
int partition = keyToPartitionMapper.apply(key);
Optional<MetadataResponseData.MetadataResponsePartition> response = topicMetadata.get(0).partitions().stream()
.filter(responsePart -> responsePart.partitionIndex() == partition
&& responsePart.leaderId() != MetadataResponse.NO_LEADER_ID)
.findFirst();
if (response.isPresent()) {
return OptionConverters.toJava(metadataCache.getAliveBrokerNode(response.get().leaderId(), interBrokerListenerName))
.orElse(Node.noNode());
} else {
return Node.noNode();
}
}
}
return Node.noNode();
}
@Override
public List<Node> getClusterNodes() {
return CollectionConverters.asJava(metadataCache.getAliveBrokerNodes(interBrokerListenerName).toSeq());
}
}

View File

@ -1238,7 +1238,7 @@ public class SharePartitionManagerTest {
Timer timer = Mockito.mock(SystemTimerReaper.class);
Persister persister = Mockito.mock(Persister.class);
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
.withTimer(timer).withShareGroupPersister(persister).build();
.withTimer(timer).withShareGroupPersister(persister).build();
// Verify that 0 calls are made to timer.close() and persister.stop().
Mockito.verify(timer, times(0)).close();
@ -2270,7 +2270,7 @@ public class SharePartitionManagerTest {
private Time time = new MockTime();
private ShareSessionCache cache = new ShareSessionCache(10, 1000);
private Map<SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
private Persister persister = NoOpShareStatePersister.getInstance();
private Persister persister = new NoOpShareStatePersister();
private Timer timer = new MockTimer();
private Metrics metrics = new Metrics();

View File

@ -5127,7 +5127,8 @@ public class SharePartitionTest {
private int defaultAcquisitionLockTimeoutMs = 30000;
private int maxDeliveryCount = MAX_DELIVERY_COUNT;
private int maxInflightMessages = MAX_IN_FLIGHT_MESSAGES;
private Persister persister = NoOpShareStatePersister.getInstance();
private Persister persister = new NoOpShareStatePersister();
private final ReplicaManager replicaManager = Mockito.mock(ReplicaManager.class);
private GroupConfigManager groupConfigManager = Mockito.mock(GroupConfigManager.class);

View File

@ -52,8 +52,10 @@ import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.time.Duration;
import java.util.ArrayList;
@ -89,25 +91,31 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
@Timeout(600)
@Timeout(1200)
@Tag("integration")
public class ShareConsumerTest {
private KafkaClusterTestKit cluster;
private final TopicPartition tp = new TopicPartition("topic", 0);
private final TopicPartition warmupTp = new TopicPartition("warmup", 0);
private static final String DEFAULT_STATE_PERSISTER = "org.apache.kafka.server.share.persister.DefaultStatePersister";
private static final String NO_OP_PERSISTER = "org.apache.kafka.server.share.persister.NoOpShareStatePersister";
@BeforeEach
public void createCluster() throws Exception {
public void createCluster(TestInfo testInfo) throws Exception {
String persisterClassName = NO_OP_PERSISTER;
if (testInfo.getDisplayName().contains(".persister=")) {
persisterClassName = testInfo.getDisplayName().split("=")[1];
}
cluster = new KafkaClusterTestKit.Builder(
new TestKitNodes.Builder()
.setNumBrokerNodes(1)
.setNumControllerNodes(1)
.build())
.setNumBrokerNodes(1)
.setNumControllerNodes(1)
.build())
.setConfigProp("auto.create.topics.enable", "false")
.setConfigProp("group.coordinator.rebalance.protocols", "classic,consumer,share")
.setConfigProp("group.share.enable", "true")
.setConfigProp("group.share.partition.max.record.locks", "10000")
.setConfigProp("group.share.persister.class.name", "org.apache.kafka.server.group.share.NoOpShareStatePersister")
.setConfigProp("group.share.persister.class.name", persisterClassName)
.setConfigProp("group.share.record.lock.duration.ms", "15000")
.setConfigProp("offsets.topic.replication.factor", "1")
.setConfigProp("share.coordinator.state.topic.min.isr", "1")
@ -129,8 +137,9 @@ public class ShareConsumerTest {
cluster.close();
}
@Test
public void testPollNoSubscribeFails() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testPollNoSubscribeFails(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
assertEquals(Collections.emptySet(), shareConsumer.subscription());
// "Consumer is not subscribed to any topics."
@ -138,8 +147,9 @@ public class ShareConsumerTest {
shareConsumer.close();
}
@Test
public void testSubscribeAndPollNoRecords() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscribeAndPollNoRecords(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
Set<String> subscription = Collections.singleton(tp.topic());
shareConsumer.subscribe(subscription);
@ -149,8 +159,9 @@ public class ShareConsumerTest {
assertEquals(0, records.count());
}
@Test
public void testSubscribePollUnsubscribe() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscribePollUnsubscribe(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
Set<String> subscription = Collections.singleton(tp.topic());
shareConsumer.subscribe(subscription);
@ -162,8 +173,9 @@ public class ShareConsumerTest {
assertEquals(0, records.count());
}
@Test
public void testSubscribePollSubscribe() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscribePollSubscribe(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
Set<String> subscription = Collections.singleton(tp.topic());
shareConsumer.subscribe(subscription);
@ -177,8 +189,9 @@ public class ShareConsumerTest {
assertEquals(0, records.count());
}
@Test
public void testSubscribeUnsubscribePollFails() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscribeUnsubscribePollFails(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
Set<String> subscription = Collections.singleton(tp.topic());
shareConsumer.subscribe(subscription);
@ -192,8 +205,9 @@ public class ShareConsumerTest {
assertEquals(0, records.count());
}
@Test
public void testSubscribeSubscribeEmptyPollFails() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscribeSubscribeEmptyPollFails(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
Set<String> subscription = Collections.singleton(tp.topic());
shareConsumer.subscribe(subscription);
@ -207,8 +221,9 @@ public class ShareConsumerTest {
assertEquals(0, records.count());
}
@Test
public void testSubscriptionAndPoll() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscriptionAndPoll(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -220,8 +235,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testSubscriptionAndPollMultiple() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscriptionAndPollMultiple(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -239,8 +255,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testAcknowledgementCommitCallbackSuccessfulAcknowledgement(String persister) {
Map<TopicPartition, Set<Long>> partitionOffsetsMap = new HashMap<>();
Map<TopicPartition, Exception> partitionExceptionMap = new HashMap<>();
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
@ -265,8 +282,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testAcknowledgementCommitCallbackOnClose() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testAcknowledgementCommitCallbackOnClose(String persister) {
Map<TopicPartition, Set<Long>> partitionOffsetsMap = new HashMap<>();
Map<TopicPartition, Exception> partitionExceptionMap = new HashMap<>();
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
@ -291,8 +309,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testAcknowledgementCommitCallbackInvalidRecordStateException() throws Exception {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testAcknowledgementCommitCallbackInvalidRecordStateException(String persister) throws Exception {
Map<TopicPartition, Set<Long>> partitionOffsetsMap = new HashMap<>();
Map<TopicPartition, Exception> partitionExceptionMap = new HashMap<>();
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
@ -350,8 +369,9 @@ public class ShareConsumerTest {
}
}
@Test
public void testHeaders() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testHeaders(String persister) {
int numRecords = 1;
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
record.headers().add("headerKey", "headerValue".getBytes());
@ -389,13 +409,15 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testHeadersSerializerDeserializer() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testHeadersSerializerDeserializer(String persister) {
testHeadersSerializeDeserialize(new BaseConsumerTest.SerializerImpl(), new BaseConsumerTest.DeserializerImpl());
}
@Test
public void testMaxPollRecords() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testMaxPollRecords(String persister) {
int maxPollRecords = 2;
int numRecords = 10000;
@ -404,7 +426,7 @@ public class ShareConsumerTest {
produceMessagesWithTimestamp(numRecords, startingTimestamp);
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(),
"group1", Collections.singletonMap(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords)));
"group1", Collections.singletonMap(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, String.valueOf(maxPollRecords)));
shareConsumer.subscribe(Collections.singleton(tp.topic()));
List<ConsumerRecord<byte[], byte[]>> records = consumeRecords(shareConsumer, numRecords);
long i = 0L;
@ -425,8 +447,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testControlRecordsSkipped() throws Exception {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testControlRecordsSkipped(String persister) throws Exception {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> transactionalProducer = createProducer(new ByteArraySerializer(), new ByteArraySerializer(), "T1");
@ -467,8 +490,9 @@ public class ShareConsumerTest {
transactionalProducer.close();
}
@Test
public void testExplicitAcknowledgeSuccess() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgeSuccess(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -484,8 +508,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testExplicitAcknowledgeCommitSuccess() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgeCommitSuccess(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -503,8 +528,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testExplicitAcknowledgementCommitAsync() throws InterruptedException {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgementCommitAsync(String persister) throws InterruptedException {
ProducerRecord<byte[], byte[]> record1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
ProducerRecord<byte[], byte[]> record2 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
ProducerRecord<byte[], byte[]> record3 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
@ -555,8 +581,9 @@ public class ShareConsumerTest {
assertNull(partitionExceptionMap1.get(tp));
}
@Test
public void testExplicitAcknowledgementCommitAsyncPartialBatch() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgementCommitAsyncPartialBatch(String persister) {
ProducerRecord<byte[], byte[]> record1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
ProducerRecord<byte[], byte[]> record2 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
ProducerRecord<byte[], byte[]> record3 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
@ -614,8 +641,9 @@ public class ShareConsumerTest {
assertNull(partitionExceptionMap.get(tp));
}
@Test
public void testExplicitAcknowledgeReleasePollAccept() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgeReleasePollAccept(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -633,8 +661,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testExplicitAcknowledgeReleaseAccept() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgeReleaseAccept(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -650,8 +679,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testExplicitAcknowledgeReleaseClose() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgeReleaseClose(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -665,8 +695,9 @@ public class ShareConsumerTest {
}
@Test
public void testExplicitAcknowledgeThrowsNotInBatch() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testExplicitAcknowledgeThrowsNotInBatch(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -683,8 +714,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testImplicitAcknowledgeFailsExplicit() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testImplicitAcknowledgeFailsExplicit(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -700,8 +732,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testImplicitAcknowledgeCommitSync() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testImplicitAcknowledgeCommitSync(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -719,8 +752,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testImplicitAcknowledgementCommitAsync() throws InterruptedException {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testImplicitAcknowledgementCommitAsync(String persister) throws InterruptedException {
ProducerRecord<byte[], byte[]> record1 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
ProducerRecord<byte[], byte[]> record2 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
ProducerRecord<byte[], byte[]> record3 = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
@ -756,8 +790,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testFetchRecordLargerThanMaxPartitionFetchBytes() throws Exception {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testFetchRecordLargerThanMaxPartitionFetchBytes(String persister) throws Exception {
int maxPartitionFetchBytes = 10000;
ProducerRecord<byte[], byte[]> smallRecord = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
ProducerRecord<byte[], byte[]> bigRecord = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), new byte[maxPartitionFetchBytes]);
@ -766,7 +801,7 @@ public class ShareConsumerTest {
producer.send(bigRecord).get();
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(),
"group1", Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxPartitionFetchBytes)));
"group1", Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxPartitionFetchBytes)));
shareConsumer.subscribe(Collections.singleton(tp.topic()));
ConsumerRecords<byte[], byte[]> records = shareConsumer.poll(Duration.ofMillis(5000));
assertEquals(1, records.count());
@ -774,8 +809,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testMultipleConsumersWithDifferentGroupIds() throws InterruptedException {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testMultipleConsumersWithDifferentGroupIds(String persister) throws InterruptedException {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
@ -822,8 +858,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testMultipleConsumersInGroupSequentialConsumption() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testMultipleConsumersInGroupSequentialConsumption(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
KafkaShareConsumer<byte[], byte[]> shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
@ -857,8 +894,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testMultipleConsumersInGroupConcurrentConsumption() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testMultipleConsumersInGroupConcurrentConsumption(String persister) {
AtomicInteger totalMessagesConsumed = new AtomicInteger(0);
int consumerCount = 4;
@ -900,8 +938,9 @@ public class ShareConsumerTest {
}
}
@Test
public void testMultipleConsumersInMultipleGroupsConcurrentConsumption() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testMultipleConsumersInMultipleGroupsConcurrentConsumption(String persister) {
AtomicInteger totalMessagesConsumedGroup1 = new AtomicInteger(0);
AtomicInteger totalMessagesConsumedGroup2 = new AtomicInteger(0);
AtomicInteger totalMessagesConsumedGroup3 = new AtomicInteger(0);
@ -1007,8 +1046,9 @@ public class ShareConsumerTest {
}
}
@Test
public void testConsumerCloseInGroupSequential() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testConsumerCloseInGroupSequential(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
KafkaShareConsumer<byte[], byte[]> shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
@ -1050,8 +1090,9 @@ public class ShareConsumerTest {
assertEquals(totalMessages, consumer1MessageCount + consumer2MessageCount);
}
@Test
public void testMultipleConsumersInGroupFailureConcurrentConsumption() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testMultipleConsumersInGroupFailureConcurrentConsumption(String persister) {
AtomicInteger totalMessagesConsumed = new AtomicInteger(0);
int consumerCount = 4;
@ -1112,12 +1153,13 @@ public class ShareConsumerTest {
}
}
@Test
public void testAcquisitionLockTimeoutOnConsumer() throws InterruptedException {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testAcquisitionLockTimeoutOnConsumer(String persister) throws InterruptedException {
ProducerRecord<byte[], byte[]> producerRecord1 = new ProducerRecord<>(tp.topic(), tp.partition(), null,
"key_1".getBytes(), "value_1".getBytes());
"key_1".getBytes(), "value_1".getBytes());
ProducerRecord<byte[], byte[]> producerRecord2 = new ProducerRecord<>(tp.topic(), tp.partition(), null,
"key_2".getBytes(), "value_2".getBytes());
"key_2".getBytes(), "value_2".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
KafkaShareConsumer<byte[], byte[]> shareConsumer1 = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
shareConsumer1.subscribe(Collections.singleton(tp.topic()));
@ -1162,8 +1204,9 @@ public class ShareConsumerTest {
* Test to verify that the acknowledgement commit callback cannot invoke methods of KafkaShareConsumer.
* The exception thrown is verified in {@link TestableAcknowledgeCommitCallbackWithShareConsumer}
*/
@Test
public void testAcknowledgeCommitCallbackCallsShareConsumerDisallowed() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testAcknowledgeCommitCallbackCallsShareConsumerDisallowed(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -1202,8 +1245,9 @@ public class ShareConsumerTest {
* Test to verify that the acknowledgement commit callback can invoke KafkaShareConsumer.wakeup() and it
* wakes up the enclosing poll.
*/
@Test
public void testAcknowledgeCommitCallbackCallsShareConsumerWakeup() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testAcknowledgeCommitCallbackCallsShareConsumerWakeup(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -1240,8 +1284,9 @@ public class ShareConsumerTest {
* Test to verify that the acknowledgement commit callback can throw an exception, and it is propagated
* to the caller of poll().
*/
@Test
public void testAcknowledgeCommitCallbackThrowsException() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testAcknowledgeCommitCallbackThrowsException(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -1273,8 +1318,9 @@ public class ShareConsumerTest {
* Test to verify that calling Thread.interrupt() before KafkaShareConsumer.poll(Duration)
* causes it to throw InterruptException
*/
@Test
public void testPollThrowsInterruptExceptionIfInterrupted() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testPollThrowsInterruptExceptionIfInterrupted(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
shareConsumer.subscribe(Collections.singleton(tp.topic()));
@ -1295,8 +1341,9 @@ public class ShareConsumerTest {
* Test to verify that InvalidTopicException is thrown if the consumer subscribes
* to an invalid topic.
*/
@Test
public void testSubscribeOnInvalidTopicThrowsInvalidTopicException() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscribeOnInvalidTopicThrowsInvalidTopicException(String persister) {
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
shareConsumer.subscribe(Collections.singleton("topic abc"));
@ -1310,8 +1357,9 @@ public class ShareConsumerTest {
* Test to ensure that a wakeup when records are buffered doesn't prevent the records
* being returned on the next poll.
*/
@Test
public void testWakeupWithFetchedRecordsAvailable() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testWakeupWithFetchedRecordsAvailable(String persister) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
producer.send(record);
@ -1328,8 +1376,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testSubscriptionFollowedByTopicCreation() throws InterruptedException {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscriptionFollowedByTopicCreation(String persister) throws InterruptedException {
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "group1");
String topic = "foo";
@ -1354,8 +1403,9 @@ public class ShareConsumerTest {
producer.close();
}
@Test
public void testSubscriptionAndPollFollowedByTopicDeletion() throws InterruptedException, ExecutionException {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testSubscriptionAndPollFollowedByTopicDeletion(String persister) throws InterruptedException, ExecutionException {
String topic1 = "bar";
String topic2 = "baz";
createTopic(topic1);
@ -1371,11 +1421,11 @@ public class ShareConsumerTest {
producer.send(recordTopic1).get();
TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1,
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
producer.send(recordTopic2);
TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1,
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
// Topic bar is deleted, hence poll should not give any results.
deleteTopic(topic1);
@ -1385,17 +1435,18 @@ public class ShareConsumerTest {
producer.send(recordTopic2);
// Poll should give the record from the non-deleted topic baz.
TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1,
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
producer.send(recordTopic2);
TestUtils.waitForCondition(() -> shareConsumer.poll(Duration.ofMillis(2000)).count() == 1,
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
DEFAULT_MAX_WAIT_MS, 100L, () -> "incorrect number of records");
shareConsumer.close();
producer.close();
}
@Test
public void testLsoMovementByRecordsDeletion() {
@ParameterizedTest(name = "{displayName}.persister={0}")
@ValueSource(strings = {NO_OP_PERSISTER, DEFAULT_STATE_PERSISTER})
public void testLsoMovementByRecordsDeletion(String persister) {
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), 0, null, "key".getBytes(), "value".getBytes());
@ -1489,7 +1540,7 @@ public class ShareConsumerTest {
try (KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer())) {
for (int i = 0; i < messageCount; i++) {
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(tp.topic(), tp.partition(), startingTimestamp + i,
("key " + i).getBytes(), ("value " + i).getBytes());
("key " + i).getBytes(), ("value " + i).getBytes());
producer.send(record);
}
producer.flush();
@ -1519,7 +1570,7 @@ public class ShareConsumerTest {
Map<TopicPartition, Exception> partitionExceptionMap = new HashMap<>();
if (maxFetchBytes.isPresent()) {
shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), groupId,
Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxFetchBytes.get())));
Collections.singletonMap(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(maxFetchBytes.get())));
} else {
shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), groupId);
}
@ -1629,8 +1680,8 @@ public class ShareConsumerTest {
private void warmup() throws InterruptedException, ExecutionException, TimeoutException {
createTopic(warmupTp.topic());
TestUtils.waitForCondition(() ->
!cluster.brokers().get(0).metadataCache().getAliveBrokerNodes(new ListenerName("EXTERNAL")).isEmpty(),
DEFAULT_MAX_WAIT_MS, 100L, () -> "cache not up yet");
!cluster.brokers().get(0).metadataCache().getAliveBrokerNodes(new ListenerName("EXTERNAL")).isEmpty(),
DEFAULT_MAX_WAIT_MS, 100L, () -> "cache not up yet");
ProducerRecord<byte[], byte[]> record = new ProducerRecord<>(warmupTp.topic(), warmupTp.partition(), null, "key".getBytes(), "value".getBytes());
KafkaProducer<byte[], byte[]> producer = createProducer(new ByteArraySerializer(), new ByteArraySerializer());
KafkaShareConsumer<byte[], byte[]> shareConsumer = createShareConsumer(new ByteArrayDeserializer(), new ByteArrayDeserializer(), "warmupgroup1");
@ -1639,7 +1690,7 @@ public class ShareConsumerTest {
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();

View File

@ -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")

View File

@ -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)

View File

@ -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) {

View File

@ -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) {

View File

@ -35,6 +35,7 @@ import static org.apache.kafka.common.config.ConfigDef.Range.between;
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 <code>org.apache.kafka.server.share.Persister</code> 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 class ShareGroupConfig {
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 class ShareGroupConfig {
return shareFetchPurgatoryPurgeIntervalRequests;
}
public String shareGroupPersisterClassName() {
return shareGroupPersisterClassName;
}
public int shareFetchMaxFetchRecords() {
return shareFetchMaxFetchRecords;
}
@ -160,7 +173,6 @@ public class ShareGroupConfig {
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));
}
/**

View File

@ -22,6 +22,8 @@ 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.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
);
}

View File

@ -42,6 +42,8 @@ import org.apache.kafka.coordinator.common.runtime.CoordinatorShardBuilderSuppli
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 class ShareCoordinatorService implements ShareCoordinator {
@Override
public int partitionFor(String key) {
throwIfNotActive();
return Utils.abs(key.hashCode()) % numPartitions;
}
@ -513,6 +516,7 @@ public class ShareCoordinatorService implements ShareCoordinator {
@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 class ShareCoordinatorService implements ShareCoordinator {
@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 @@ public class ShareCoordinatorService implements ShareCoordinator {
private static <P> boolean isEmpty(List<P> list) {
return list == null || list.isEmpty();
}
private void throwIfNotActive() {
if (!isActive.get()) {
throw Errors.COORDINATOR_NOT_AVAILABLE.exception();
}
}
}

View File

@ -168,7 +168,6 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
@Override
public void onLoaded(MetadataImage newImage) {
this.metadataImage = newImage;
coordinatorMetrics.activateMetricsShard(metricsShard);
}

View File

@ -92,7 +92,7 @@ class ShareCoordinatorShardTest {
);
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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
writeAndReplayDefaultRecord(shard);
shard.onLoaded(null);
shard.onNewMetadataImage(null, null);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);

View File

@ -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<InitializeShareGroupStateResult> 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<WriteShareGroupStateResult> writeState(WriteShareGroupStateParameters request) throws IllegalArgumentException {
validate(request);
GroupTopicPartitionData<PartitionStateBatchData> gtp = request.groupTopicPartitionData();
String groupId = gtp.groupId();
Map<Uuid, Map<Integer, CompletableFuture<WriteShareGroupStateResponse>>> futureMap = new HashMap<>();
List<PersisterStateManager.WriteStateHandler> handlers = new ArrayList<>();
gtp.topicsData().forEach(topicData -> {
topicData.partitions().forEach(partitionData -> {
CompletableFuture<WriteShareGroupStateResponse> 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<Void> combinedFuture = CompletableFuture.allOf(
handlers.stream()
.map(PersisterStateManager.WriteStateHandler::result)
.toArray(CompletableFuture[]::new));
return combinedFuture.thenApply(v -> writeResponsesToResult(futureMap));
}
private WriteShareGroupStateResult writeResponsesToResult(
Map<Uuid, Map<Integer, CompletableFuture<WriteShareGroupStateResponse>>> futureMap
) {
List<TopicData<PartitionErrorData>> topicsData = futureMap.keySet().stream()
.map(topicId -> {
List<PartitionErrorData> partitionErrData = futureMap.get(topicId).entrySet().stream()
.map(partitionFuture -> {
int partition = partitionFuture.getKey();
CompletableFuture<WriteShareGroupStateResponse> 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<ReadShareGroupStateResult> readState(ReadShareGroupStateParameters request) throws IllegalArgumentException {
validate(request);
GroupTopicPartitionData<PartitionIdLeaderEpochData> gtp = request.groupTopicPartitionData();
String groupId = gtp.groupId();
Map<Uuid, Map<Integer, CompletableFuture<ReadShareGroupStateResponse>>> futureMap = new HashMap<>();
List<PersisterStateManager.ReadStateHandler> handlers = new ArrayList<>();
gtp.topicsData().forEach(topicData -> {
topicData.partitions().forEach(partitionData -> {
CompletableFuture<ReadShareGroupStateResponse> 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<Void>
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
handlers.stream()
.map(PersisterStateManager.ReadStateHandler::result)
.toArray(CompletableFuture[]::new));
// Transform the combined CompletableFuture<Void> into CompletableFuture<ReadShareGroupStateResult>
return combinedFuture.thenApply(v -> readResponsesToResult(futureMap));
}
private ReadShareGroupStateResult readResponsesToResult(
Map<Uuid, Map<Integer, CompletableFuture<ReadShareGroupStateResponse>>> futureMap
) {
List<TopicData<PartitionAllData>> topicsData = futureMap.keySet().stream()
.map(topicId -> {
List<PartitionAllData> partitionAllData = futureMap.get(topicId).entrySet().stream()
.map(partitionFuture -> {
int partition = partitionFuture.getKey();
CompletableFuture<ReadShareGroupStateResponse> 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<DeleteShareGroupStateResult> 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<ReadShareGroupStateSummaryResult> 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();
}
}

View File

@ -28,15 +28,7 @@ import java.util.stream.Collectors;
*/
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

View File

@ -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.

View File

@ -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<RPCType, Set<Node>> 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<Node, Map<RPCType, Map<String, List<PersisterStateManagerHandler>>>> 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<Node, Map<RPCType, Map<String, List<PersisterStateManagerHandler>>>> 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.
* <p>
* 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<ClientResponse> 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<FindCoordinatorRequest> 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<FindCoordinatorResponseData.Coordinator> 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<ClientResponse> callback) {
this.onCompleteCallback = callback;
}
}
public class WriteStateHandler extends PersisterStateManagerHandler {
private final int stateEpoch;
private final int leaderEpoch;
private final long startOffset;
private final List<PersisterStateBatch> batches;
private final CompletableFuture<WriteShareGroupStateResponse> result;
private final BackoffManager writeStateBackoff;
public WriteStateHandler(
String groupId,
Uuid topicId,
int partition,
int stateEpoch,
int leaderEpoch,
long startOffset,
List<PersisterStateBatch> batches,
CompletableFuture<WriteShareGroupStateResponse> 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<PersisterStateBatch> batches,
CompletableFuture<WriteShareGroupStateResponse> result,
Consumer<ClientResponse> 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<WriteShareGroupStateResponseData.PartitionResult> 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<WriteShareGroupStateResponse> 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<ReadShareGroupStateResponse> result;
private final BackoffManager readStateBackoff;
public ReadStateHandler(
String groupId,
Uuid topicId,
int partition,
int leaderEpoch,
CompletableFuture<ReadShareGroupStateResponse> result,
long backoffMs,
long backoffMaxMs,
int maxRPCRetryAttempts,
Consumer<ClientResponse> 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<ReadShareGroupStateResponse> result,
Consumer<ClientResponse> 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<ReadShareGroupStateRequest> 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<ReadShareGroupStateResponseData.PartitionResult> 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<ReadShareGroupStateResponse> result() {
return result;
}
@Override
protected boolean isBatchable() {
return true;
}
@Override
protected RPCType rpcType() {
return RPCType.READ;
}
}
private class SendThread extends InterBrokerSendThread {
private final ConcurrentLinkedQueue<PersisterStateManagerHandler> 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<Node> 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.
* <p>
* 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<RequestAndCompletionHandler> 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<RequestAndCompletionHandler> 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<RPCType, Set<Node>> 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<Uuid, List<WriteShareGroupStateRequestData.PartitionData>> 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<Uuid, List<ReadShareGroupStateRequestData.PartitionData>> 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())));
}
}
}

View File

@ -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<Node> getClusterNodes();
}

View File

@ -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<Node> 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<PartitionStateBatchData>()
.setGroupId(null).build()).build());
});
// topicsData is empty
assertThrows(IllegalArgumentException.class, () -> {
DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build();
defaultStatePersister.writeState(new WriteShareGroupStateParameters.Builder()
.setGroupTopicPartitionData(new GroupTopicPartitionData.Builder<PartitionStateBatchData>()
.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<PartitionStateBatchData>()
.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<PartitionStateBatchData>()
.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<PartitionStateBatchData>()
.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<PartitionIdLeaderEpochData>()
.setGroupId(null).build()).build());
});
// topicsData is empty
assertThrows(IllegalArgumentException.class, () -> {
DefaultStatePersister defaultStatePersister = DefaultStatePersisterBuilder.builder().build();
defaultStatePersister.readState(new ReadShareGroupStateParameters.Builder()
.setGroupTopicPartitionData(new GroupTopicPartitionData.Builder<PartitionIdLeaderEpochData>()
.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<PartitionIdLeaderEpochData>()
.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<PartitionIdLeaderEpochData>()
.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<PartitionIdLeaderEpochData>()
.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<WriteShareGroupStateResult> resultFuture = defaultStatePersister.writeState(request);
WriteShareGroupStateResult result = null;
try {
result = resultFuture.get();
} catch (Exception e) {
fail("Unexpected exception", e);
}
HashSet<PartitionData> resultMap = new HashSet<>();
result.topicsData().forEach(
topicData -> topicData.partitions().forEach(
partitionData -> resultMap.add((PartitionData) partitionData)
)
);
HashSet<PartitionData> 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<ReadShareGroupStateResult> resultFuture = defaultStatePersister.readState(request);
ReadShareGroupStateResult result = null;
try {
result = resultFuture.get();
} catch (Exception e) {
fail("Unexpected exception", e);
}
HashSet<PartitionData> resultMap = new HashSet<>();
result.topicsData().forEach(
topicData -> topicData.partitions().forEach(
partitionData -> resultMap.add((PartitionData) partitionData)
)
);
HashSet<PartitionData> 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);
}
}
}