diff --git a/.github/scripts/junit.py b/.github/scripts/junit.py index 95b5d4e4610de..550ea4935116d 100644 --- a/.github/scripts/junit.py +++ b/.github/scripts/junit.py @@ -361,7 +361,7 @@ def split_report_path(base_path: str, report_path: str) -> Tuple[str, str]: failure_messages.append(f"Gradle task had a failure exit code. Failing this script.") if thread_dump_url: - failure_messages.append(f"Thread dump available at {thread_dump_url}. Failing this script.") + failure_messages.append(f"Thread dump available at {thread_dump_url} and the script will now fail.") if junit_report_url: report_md = f"Download [JUnit HTML report]({junit_report_url})" diff --git a/build.gradle b/build.gradle index 2e35057165c53..036682572e48b 100644 --- a/build.gradle +++ b/build.gradle @@ -369,7 +369,6 @@ subprojects { // Fix for avoiding inclusion of runtime dependencies marked as 'shadow' in MANIFEST Class-Path. // https://github.com/GradleUp/shadow/issues/324 - afterEvaluate { pom.withXml { xml -> def dependenciesNode = xml.asNode().get('dependencies') ?: xml.asNode().appendNode('dependencies') project.configurations.shadowed.allDependencies.each { @@ -380,7 +379,6 @@ subprojects { dependencyNode.appendNode('scope', 'runtime') } } - } } afterEvaluate { @@ -2230,6 +2228,7 @@ project(':storage') { } dependencies { + implementation project(':metadata') implementation project(':storage:storage-api') implementation project(':server-common') implementation project(':clients') diff --git a/checkstyle/import-control-storage.xml b/checkstyle/import-control-storage.xml index 639cb6dc1d04c..6628de8d45345 100644 --- a/checkstyle/import-control-storage.xml +++ b/checkstyle/import-control-storage.xml @@ -94,6 +94,8 @@ + + diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 4e0f338af5d46..2d4c279e7073c 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -98,7 +98,7 @@ files="(AbstractFetch|ClientTelemetryReporter|ConsumerCoordinator|CommitRequestManager|FetchCollector|OffsetFetcherUtils|KafkaProducer|Sender|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords|FetchSessionHandler|MockAdminClient).java"/> + files="(AbstractRequest|AbstractResponse|KerberosLogin|WorkerSinkTaskTest|TransactionManagerTest|SenderTest|KafkaAdminClient|ConsumerCoordinatorTest|KafkaAdminClientTest|KafkaProducerTest).java"/> diff --git a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java index e402a4344c109..d48f620880ef3 100644 --- a/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java +++ b/clients/clients-integration-tests/src/test/java/org/apache/kafka/clients/consumer/ShareConsumerTest.java @@ -122,8 +122,7 @@ @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "1"), @ClusterConfigProperty(key = "transaction.state.log.min.isr", value = "1"), - @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1"), - @ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + @ClusterConfigProperty(key = "transaction.state.log.replication.factor", value = "1") }, types = {Type.KRAFT} ) @@ -1859,8 +1858,7 @@ public void testShareAutoOffsetResetByDurationInvalidFormat() throws Exception { @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "3"), @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), - @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3"), - @ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3") } ) @Timeout(90) @@ -2011,8 +2009,7 @@ public void testShareConsumerAfterCoordinatorMovement() throws Exception { @ClusterConfigProperty(key = "offsets.topic.num.partitions", value = "3"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "3"), @ClusterConfigProperty(key = "share.coordinator.state.topic.num.partitions", value = "3"), - @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3"), - @ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") + @ClusterConfigProperty(key = "share.coordinator.state.topic.replication.factor", value = "3") } ) @Timeout(150) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java index 5ef72f327d637..99111a70d4bae 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/AlterConsumerGroupOffsetsHandler.java @@ -108,7 +108,7 @@ public OffsetCommitRequest.Builder buildBatchedRequest( .setGroupId(groupId.idValue) .setTopics(new ArrayList<>(offsetData.values())); - return new OffsetCommitRequest.Builder(data); + return OffsetCommitRequest.Builder.forTopicNames(data); } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java index c5911e4303e72..e37b0427355a4 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/internals/DeleteShareGroupOffsetsHandler.java @@ -99,8 +99,7 @@ DeleteShareGroupOffsetsRequest.Builder buildBatchedRequest(int brokerId, Set - * This is an early access feature under development which is introduced by KIP-932. - * It is not suitable for production use until it is fully implemented and released. + * This is a preview feature introduced by KIP-932. It is not yet recommended for production use. * *

Cross-Version Compatibility

* This client can communicate with brokers that are a version that supports share groups. You will receive an @@ -100,7 +99,7 @@ * of the topic-partitions that match its subscriptions. Records are acquired for delivery to this consumer with a * time-limited acquisition lock. While a record is acquired, it is not available for another consumer. By default, * the lock duration is 30 seconds, but it can also be controlled using the group {@code group.share.record.lock.duration.ms} - * configuration parameter. The idea is that the lock is automatically released once the lock duration has elapsed, and + * configuration property. The idea is that the lock is automatically released once the lock duration has elapsed, and * then the record is available to be given to another consumer. The consumer which holds the lock can deal with it in * the following ways: *
    @@ -116,8 +115,8 @@ * {@code group.share.record.lock.partition.limit}. By limiting the duration of the acquisition lock and automatically * releasing the locks, the broker ensures delivery progresses even in the presence of consumer failures. *

    - * The consumer can choose to use implicit or explicit acknowledgement of the records it processes by configuring the - * consumer {@code share.acknowledgement.mode} property. + * The consumer can choose to use implicit or explicit acknowledgement of the records it processes by using the + * consumer {@code share.acknowledgement.mode} configuration property. *

    * If the application sets the property to "implicit" or does not set it at all, then the consumer is using * implicit acknowledgement. In this mode, the application acknowledges delivery by: @@ -129,7 +128,7 @@ * the delivered records as processed successfully and commits the acknowledgements to Kafka. *

  • Calling {@link #close()} which releases any acquired records without acknowledgement.
  • *
- * If the application sets the property to "explicit", then the consumer is using explicit acknowledgment. + * If the application sets the property to "explicit", then the consumer is using explicit acknowledgement. * The application must acknowledge all records returned from {@link #poll(Duration)} using * {@link #acknowledge(ConsumerRecord, AcknowledgeType)} before its next call to {@link #poll(Duration)}. * If the application calls {@link #poll(Duration)} without having acknowledged all records, an @@ -162,6 +161,7 @@ * props.setProperty("group.id", "test"); * props.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * * KafkaShareConsumer<String, String> consumer = new KafkaShareConsumer<>(props); * consumer.subscribe(Arrays.asList("foo")); * while (true) { @@ -181,6 +181,7 @@ * props.setProperty("group.id", "test"); * props.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); + * * KafkaShareConsumer<String, String> consumer = new KafkaShareConsumer<>(props); * consumer.subscribe(Arrays.asList("foo")); * while (true) { @@ -203,6 +204,7 @@ * props.setProperty("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer"); * props.setProperty("share.acknowledgement.mode", "explicit"); + * * KafkaShareConsumer<String, String> consumer = new KafkaShareConsumer<>(props); * consumer.subscribe(Arrays.asList("foo")); * while (true) { @@ -443,7 +445,7 @@ public void unsubscribe() { } /** - * Fetch data for the topics specified using {@link #subscribe(Collection)}. It is an error to not have + * Deliver records for the topics specified using {@link #subscribe(Collection)}. It is an error to not have * subscribed to any topics before polling for data. * *

@@ -452,13 +454,14 @@ public void unsubscribe() { * * @param timeout The maximum time to block (must not be greater than {@link Long#MAX_VALUE} milliseconds) * - * @return map of topic to records since the last fetch for the subscribed list of topics + * @return map of topic to records * * @throws AuthenticationException if authentication fails. See the exception for more details * @throws AuthorizationException if caller lacks Read access to any of the subscribed * topics or to the share group. See the exception for more details * @throws IllegalArgumentException if the timeout value is negative - * @throws IllegalStateException if the consumer is not subscribed to any topics + * @throws IllegalStateException if the consumer is not subscribed to any topics, or it is using + * explicit acknowledgement and has not acknowledged all records previously delivered * @throws ArithmeticException if the timeout is greater than {@link Long#MAX_VALUE} milliseconds. * @throws InvalidTopicException if the current subscription contains any invalid * topic (per {@link org.apache.kafka.common.internals.Topic#validate(String)}) @@ -475,11 +478,12 @@ public ConsumerRecords poll(Duration timeout) { * Acknowledge successful delivery of a record returned on the last {@link #poll(Duration)} call. * The acknowledgement is committed on the next {@link #commitSync()}, {@link #commitAsync()} or * {@link #poll(Duration)} call. + *

This method can only be used if the consumer is using explicit acknowledgement. * * @param record The record to acknowledge * - * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer has already - * used implicit acknowledgement + * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer is not using + * explicit acknowledgement */ @Override public void acknowledge(ConsumerRecord record) { @@ -489,14 +493,14 @@ public void acknowledge(ConsumerRecord record) { /** * Acknowledge delivery of a record returned on the last {@link #poll(Duration)} call indicating whether * it was processed successfully. The acknowledgement is committed on the next {@link #commitSync()}, - * {@link #commitAsync()} or {@link #poll(Duration)} call. By using this method, the consumer is using - * explicit acknowledgement. + * {@link #commitAsync()} or {@link #poll(Duration)} call. + *

This method can only be used if the consumer is using explicit acknowledgement. * * @param record The record to acknowledge * @param type The acknowledgement type which indicates whether it was processed successfully * - * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer has already - * used implicit acknowledgement + * @throws IllegalStateException if the record is not waiting to be acknowledged, or the consumer is not using + * explicit acknowledgement */ @Override public void acknowledge(ConsumerRecord record, AcknowledgeType type) { @@ -585,7 +589,7 @@ public void setAcknowledgementCommitCallback(AcknowledgementCommitCallback callb * client to complete the request. *

* Client telemetry is controlled by the {@link ConsumerConfig#ENABLE_METRICS_PUSH_CONFIG} - * configuration option. + * configuration property. * * @param timeout The maximum time to wait for consumer client to determine its client instance ID. * The value must be non-negative. Specifying a timeout of zero means do not diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java index 249904f87b0ae..1acd5634f441a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java @@ -186,7 +186,7 @@ public class AsyncKafkaConsumer implements ConsumerDelegate { */ private class BackgroundEventProcessor implements EventProcessor { - private Optional streamsGroupRebalanceCallbacks = Optional.empty(); + private Optional streamsRebalanceListener = Optional.empty(); private final Optional streamsRebalanceData; public BackgroundEventProcessor() { @@ -202,7 +202,7 @@ private void setStreamsRebalanceListener(final StreamsRebalanceListener streamsR throw new IllegalStateException("Background event processor was not created to be used with Streams " + "rebalance protocol events"); } - this.streamsGroupRebalanceCallbacks = Optional.of(streamsRebalanceListener); + this.streamsRebalanceListener = Optional.of(streamsRebalanceListener); } @Override @@ -277,20 +277,15 @@ private void processStreamsOnAllTasksLostCallbackNeededEvent(final StreamsOnAllT private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set activeTasksToRevoke, final CompletableFuture future) { - final Optional error; - final Optional exceptionFromCallback = streamsGroupRebalanceCallbacks().onTasksRevoked(activeTasksToRevoke); - if (exceptionFromCallback.isPresent()) { - error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task revocation callback throws an error")); - } else { - error = Optional.empty(); - } + final Optional exceptionFromCallback = streamsRebalanceListener().onTasksRevoked(activeTasksToRevoke); + final Optional error = exceptionFromCallback.map(e -> ConsumerUtils.maybeWrapAsKafkaException(e, "Task revocation callback throws an error")); return new StreamsOnTasksRevokedCallbackCompletedEvent(future, error); } private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallback(final StreamsRebalanceData.Assignment assignment, final CompletableFuture future) { final Optional error; - final Optional exceptionFromCallback = streamsGroupRebalanceCallbacks().onTasksAssigned(assignment); + final Optional exceptionFromCallback = streamsRebalanceListener().onTasksAssigned(assignment); if (exceptionFromCallback.isPresent()) { error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "Task assignment callback throws an error")); } else { @@ -302,7 +297,7 @@ private StreamsOnTasksAssignedCallbackCompletedEvent invokeOnTasksAssignedCallba private StreamsOnAllTasksLostCallbackCompletedEvent invokeOnAllTasksLostCallback(final CompletableFuture future) { final Optional error; - final Optional exceptionFromCallback = streamsGroupRebalanceCallbacks().onAllTasksLost(); + final Optional exceptionFromCallback = streamsRebalanceListener().onAllTasksLost(); if (exceptionFromCallback.isPresent()) { error = Optional.of(ConsumerUtils.maybeWrapAsKafkaException(exceptionFromCallback.get(), "All tasks lost callback throws an error")); } else { @@ -318,8 +313,8 @@ private StreamsRebalanceData streamsRebalanceData() { "rebalance protocol events")); } - private StreamsRebalanceListener streamsGroupRebalanceCallbacks() { - return streamsGroupRebalanceCallbacks.orElseThrow( + private StreamsRebalanceListener streamsRebalanceListener() { + return streamsRebalanceListener.orElseThrow( () -> new IllegalStateException("Background event processor was not created to be used with Streams " + "rebalance protocol events")); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java index 284707a812b53..62d1fe3a86638 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java @@ -727,7 +727,7 @@ public NetworkClientDelegate.UnsentRequest toUnsentRequest() { lastEpochSentOnCommit = Optional.empty(); } - OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data); + OffsetCommitRequest.Builder builder = OffsetCommitRequest.Builder.forTopicNames(data); return buildRequestWithResponseHandling(builder); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 01fc605ea7982..1cba10ef15d97 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -1327,7 +1327,7 @@ RequestFuture sendOffsetCommitRequest(final Map ShareConsumerDelegate create(final ConsumerConfig config, try { LogContext logContext = new LogContext(); Logger log = logContext.logger(getClass()); - log.warn("Share groups and KafkaShareConsumer are part of the early access of KIP-932 and MUST NOT be used in production."); + log.warn("Share groups and KafkaShareConsumer are part of a preview feature introduced by KIP-932, and are not recommended for use in production."); return new ShareConsumerImpl<>(config, keyDeserializer, valueDeserializer); } catch (KafkaException e) { throw e; @@ -66,7 +66,7 @@ public ShareConsumerDelegate create(final LogContext logContext, final ConsumerMetadata metadata) { try { Logger log = logContext.logger(getClass()); - log.warn("Share groups and KafkaShareConsumer are part of the early access of KIP-932 and MUST NOT be used in production."); + log.warn("Share groups and KafkaShareConsumer are part of a preview feature introduced by KIP-932, and are not recommended for use in production."); return new ShareConsumerImpl<>( logContext, clientId, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java index 5012aba5a32b0..b86c6d0498c35 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManager.java @@ -521,11 +521,14 @@ private void onSuccessResponse(final StreamsGroupHeartbeatResponse response, fin } List statuses = data.status(); - if (statuses != null && !statuses.isEmpty()) { - String statusDetails = statuses.stream() - .map(status -> "(" + status.statusCode() + ") " + status.statusDetail()) - .collect(Collectors.joining(", ")); - logger.warn("Membership is in the following statuses: {}", statusDetails); + if (statuses != null) { + streamsRebalanceData.setStatuses(statuses); + if (!statuses.isEmpty()) { + String statusDetails = statuses.stream() + .map(status -> "(" + status.statusCode() + ") " + status.statusDetail()) + .collect(Collectors.joining(", ")); + logger.warn("Membership is in the following statuses: {}", statusDetails); + } } membershipManager.onHeartbeatSuccess(response); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java index eb006ac8dda5a..d562fc754f43a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsMembershipManager.java @@ -52,6 +52,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static java.util.Collections.unmodifiableList; + /** * Tracks the state of a single member in relationship to a group: *

@@ -1305,4 +1307,9 @@ public void onAllTasksLostCallbackCompleted(final StreamsOnAllTasksLostCallbackC future.complete(null); } } + + // visible for testing + List stateListeners() { + return unmodifiableList(stateUpdatesListeners); + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java index 6157b66cf16ed..0158370a50999 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.consumer.internals; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; import java.util.Collection; import java.util.Collections; @@ -300,6 +301,8 @@ public String toString() { private final AtomicBoolean shutdownRequested = new AtomicBoolean(false); + private final AtomicReference> statuses = new AtomicReference<>(List.of()); + public StreamsRebalanceData(final UUID processId, final Optional endpoint, final Map subtopologies, @@ -346,11 +349,24 @@ public Map> partitionsByHost() { return partitionsByHost.get(); } + /** For the current stream thread to request a shutdown of all Streams clients belonging to the same application. */ public void requestShutdown() { shutdownRequested.set(true); } + /** True if the current stream thread requested a shutdown of all Streams clients belonging to the same application. */ public boolean shutdownRequested() { return shutdownRequested.get(); } + + /** Updated whenever the status of the streams group is updated. */ + public void setStatuses(final List s) { + statuses.set(s); + } + + /** For communicating the current status of the group to the stream thread */ + public List statuses() { + return statuses.get(); + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 64a3af83e6571..d46a2204beb40 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -75,7 +75,6 @@ import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.telemetry.internals.ClientTelemetryUtils; import org.apache.kafka.common.utils.AppInfoParser; -import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; @@ -256,7 +255,7 @@ public class KafkaProducer implements Producer { private final ProducerMetadata metadata; private final RecordAccumulator accumulator; private final Sender sender; - private final Thread ioThread; + private final Sender.SenderThread ioThread; private final Compression compression; private final Sensor errors; private final Time time; @@ -454,7 +453,7 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali this.errors = this.metrics.sensor("errors"); this.sender = newSender(logContext, kafkaClient, this.metadata); String ioThreadName = NETWORK_THREAD_PREFIX + " | " + clientId; - this.ioThread = new KafkaThread(ioThreadName, this.sender, true); + this.ioThread = new Sender.SenderThread(ioThreadName, this.sender, true); this.ioThread.start(); config.logUnused(); AppInfoParser.registerAppInfo(JMX_PREFIX, clientId, metrics, time.milliseconds()); @@ -480,7 +479,7 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali ProducerInterceptors interceptors, Partitioner partitioner, Time time, - KafkaThread ioThread, + Sender.SenderThread ioThread, Optional clientTelemetryReporter) { this.producerConfig = config; this.time = time; @@ -596,14 +595,17 @@ private TransactionManager configureTransactionState(ProducerConfig config, if (config.getBoolean(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG)) { final String transactionalId = config.getString(ProducerConfig.TRANSACTIONAL_ID_CONFIG); + final boolean enable2PC = config.getBoolean(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG); final int transactionTimeoutMs = config.getInt(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); final long retryBackoffMs = config.getLong(ProducerConfig.RETRY_BACKOFF_MS_CONFIG); + transactionManager = new TransactionManager( logContext, transactionalId, transactionTimeoutMs, retryBackoffMs, - apiVersions + apiVersions, + enable2PC ); if (transactionManager.isTransactional()) @@ -618,8 +620,13 @@ private TransactionManager configureTransactionState(ProducerConfig config, } /** + * Initialize the transactional state for this producer, similar to {@link #initTransactions()} but + * with additional capabilities to keep a previously prepared transaction. + * * Needs to be called before any other methods when the {@code transactional.id} is set in the configuration. - * This method does the following: + * + * When {@code keepPreparedTxn} is {@code false}, this behaves like the standard transactional + * initialization where the method does the following: *

    *
  1. Ensures any transactions initiated by previous instances of the producer with the same * {@code transactional.id} are completed. If the previous instance had failed with a transaction in @@ -628,26 +635,38 @@ private TransactionManager configureTransactionState(ProducerConfig config, *
  2. Gets the internal producer id and epoch, used in all future transactional * messages issued by the producer.
  3. *
+ * + *

+ * When {@code keepPreparedTxn} is set to {@code true}, the producer does not automatically abort existing + * transactions. Instead, it enters a recovery mode allowing only finalization of those previously + * prepared transactions. + * This behavior is especially crucial for 2PC scenarios, where transactions should remain intact + * until the external transaction manager decides whether to commit or abort. + *

+ * + * @param keepPreparedTxn true to retain any in-flight prepared transactions (necessary for 2PC + * recovery), false to abort existing transactions and behave like + * the standard initTransactions. + * * Note that this method will raise {@link TimeoutException} if the transactional state cannot * be initialized before expiration of {@code max.block.ms}. Additionally, it will raise {@link InterruptException} * if interrupted. It is safe to retry in either case, but once the transactional state has been successfully * initialized, this method should no longer be used. * - * @throws IllegalStateException if no {@code transactional.id} has been configured - * @throws org.apache.kafka.common.errors.UnsupportedVersionException fatal error indicating the broker - * does not support transactions (i.e. if its version is lower than 0.11.0.0) - * @throws org.apache.kafka.common.errors.AuthorizationException error indicating that the configured - * transactional.id is not authorized, or the idempotent producer id is unavailable. See the exception for - * more details. User may retry this function call after fixing the permission. - * @throws KafkaException if the producer has encountered a previous fatal error or for any other unexpected error + * @throws IllegalStateException if no {@code transactional.id} is configured + * @throws org.apache.kafka.common.errors.UnsupportedVersionException if the broker does not + * support transactions (i.e. if its version is lower than 0.11.0.0) + * @throws org.apache.kafka.common.errors.TransactionalIdAuthorizationException if the configured + * {@code transactional.id} is unauthorized either for normal transaction writes or 2PC. + * @throws KafkaException if the producer encounters a fatal error or any other unexpected error * @throws TimeoutException if the time taken for initialize the transaction has surpassed max.block.ms. * @throws InterruptException if the thread is interrupted while blocked */ - public void initTransactions() { + public void initTransactions(boolean keepPreparedTxn) { throwIfNoTransactionManager(); throwIfProducerClosed(); long now = time.nanoseconds(); - TransactionalRequestResult result = transactionManager.initializeTransactions(); + TransactionalRequestResult result = transactionManager.initializeTransactions(keepPreparedTxn); sender.wakeup(); result.await(maxBlockTimeMs, TimeUnit.MILLISECONDS); producerMetrics.recordInit(time.nanoseconds() - now); @@ -1546,6 +1565,7 @@ private class AppendCallbacks implements RecordAccumulator.AppendCallbacks { private final String recordLogString; private volatile int partition = RecordMetadata.UNKNOWN_PARTITION; private volatile TopicPartition topicPartition; + private final Headers headers; private AppendCallbacks(Callback userCallback, ProducerInterceptors interceptors, ProducerRecord record) { this.userCallback = userCallback; @@ -1554,6 +1574,12 @@ private AppendCallbacks(Callback userCallback, ProducerInterceptors interc // whole lifetime of the batch. // We don't want to have an NPE here, because the interceptors would not be notified (see .doSend). topic = record != null ? record.topic() : null; + if (record != null) { + headers = record.headers(); + } else { + headers = new RecordHeaders(); + ((RecordHeaders) headers).setReadOnly(); + } recordPartition = record != null ? record.partition() : null; recordLogString = log.isTraceEnabled() && record != null ? record.toString() : ""; } @@ -1563,7 +1589,7 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { if (metadata == null) { metadata = new RecordMetadata(topicPartition(), -1, -1, RecordBatch.NO_TIMESTAMP, -1, -1); } - this.interceptors.onAcknowledgement(metadata, exception); + this.interceptors.onAcknowledgement(metadata, exception, headers); if (this.userCallback != null) this.userCallback.onCompletion(metadata, exception); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index a4aac86df09fc..e3c5a23ca5195 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -142,7 +142,7 @@ public MockProducer() { } @Override - public void initTransactions() { + public void initTransactions(boolean keepPreparedTxn) { verifyNotClosed(); verifyNotFenced(); if (this.transactionInitialized) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java index 798034dda6de2..a5cd92295ff96 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java @@ -42,7 +42,14 @@ public interface Producer extends Closeable { /** * See {@link KafkaProducer#initTransactions()} */ - void initTransactions(); + default void initTransactions() { + initTransactions(false); + } + + /** + * See {@link KafkaProducer#initTransactions(boolean)} + */ + void initTransactions(boolean keepPreparedTxn); /** * See {@link KafkaProducer#beginTransaction()} diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 949c6c167ba8e..362d205e8c1aa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -355,6 +355,11 @@ public class ProducerConfig extends AbstractConfig { "By default the TransactionId is not configured, which means transactions cannot be used. " + "Note that, by default, transactions require a cluster of at least three brokers which is the recommended setting for production; for development you can change this, by adjusting broker setting transaction.state.log.replication.factor."; + /** transaction.two.phase.commit.enable */ + public static final String TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG = "transaction.two.phase.commit.enable"; + private static final String TRANSACTION_TWO_PHASE_COMMIT_ENABLE_DOC = "If set to true, then the broker is informed that the client is participating in " + + "two phase commit protocol and transactions that this client starts never expire."; + /** * security.providers */ @@ -526,6 +531,11 @@ public class ProducerConfig extends AbstractConfig { new ConfigDef.NonEmptyString(), Importance.LOW, TRANSACTIONAL_ID_DOC) + .define(TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, + Type.BOOLEAN, + false, + Importance.LOW, + TRANSACTION_TWO_PHASE_COMMIT_ENABLE_DOC) .define(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, Type.STRING, CommonClientConfigs.DEFAULT_METADATA_RECOVERY_STRATEGY, @@ -609,6 +619,20 @@ private void postProcessAndValidateIdempotenceConfigs(final Map if (!idempotenceEnabled && userConfiguredTransactions) { throw new ConfigException("Cannot set a " + ProducerConfig.TRANSACTIONAL_ID_CONFIG + " without also enabling idempotence."); } + + // Validate that transaction.timeout.ms is not set when transaction.two.phase.commit.enable is true + // In standard Kafka transactions, the broker enforces transaction.timeout.ms and aborts any + // transaction that isn't completed in time. With two-phase commit (2PC), an external coordinator + // decides when to finalize, so broker-side timeouts don't apply. Disallow using both. + boolean enable2PC = this.getBoolean(TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG); + boolean userConfiguredTransactionTimeout = originalConfigs.containsKey(TRANSACTION_TIMEOUT_CONFIG); + if (enable2PC && userConfiguredTransactionTimeout) { + throw new ConfigException( + "Cannot set " + ProducerConfig.TRANSACTION_TIMEOUT_CONFIG + + " when " + ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG + + " is set to true. Transactions will not expire with two-phase commit enabled." + ); + } } private static String parseAcks(String acksString) { diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java index 5bc4b2c2c8520..4a813dc96babc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.producer; import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.header.Headers; /** * A plugin interface that allows you to intercept (and possibly mutate) the records received by the producer before @@ -83,12 +84,37 @@ public interface ProducerInterceptor extends Configurable, AutoCloseable { * @param metadata The metadata for the record that was sent (i.e. the partition and offset). * If an error occurred, metadata will contain only valid topic and maybe * partition. If partition is not given in ProducerRecord and an error occurs - * before partition gets assigned, then partition will be set to RecordMetadata.NO_PARTITION. + * before partition gets assigned, then partition will be set to {@link RecordMetadata#UNKNOWN_PARTITION}. * The metadata may be null if the client passed null record to * {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord)}. * @param exception The exception thrown during processing of this record. Null if no error occurred. */ - void onAcknowledgement(RecordMetadata metadata, Exception exception); + default void onAcknowledgement(RecordMetadata metadata, Exception exception) {} + + /** + * This method is called when the record sent to the server has been acknowledged, or when sending the record fails before + * it gets sent to the server. + *

+ * This method is generally called just before the user callback is called, and in additional cases when KafkaProducer.send() + * throws an exception. + *

+ * Any exception thrown by this method will be ignored by the caller. + *

+ * This method will generally execute in the background I/O thread, so the implementation should be reasonably fast. + * Otherwise, sending of messages from other threads could be delayed. + * + * @param metadata The metadata for the record that was sent (i.e. the partition and offset). + * If an error occurred, metadata will contain only valid topic and maybe + * partition. If partition is not given in ProducerRecord and an error occurs + * before partition gets assigned, then partition will be set to {@link RecordMetadata#UNKNOWN_PARTITION}. + * The metadata may be null if the client passed null record to + * {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord)}. + * @param exception The exception thrown during processing of this record. Null if no error occurred. + * @param headers The headers for the record that was sent. It is read-only. + */ + default void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + onAcknowledgement(metadata, exception); + } /** * This is called when interceptor is closed diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java index 9936eef760940..71d3839cedd12 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java @@ -22,6 +22,8 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.internals.Plugin; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.RecordBatch; @@ -77,7 +79,7 @@ public ProducerRecord onSend(ProducerRecord record) { /** * This method is called when the record sent to the server has been acknowledged, or when sending the record fails before - * it gets sent to the server. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception)} + * it gets sent to the server. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception, Headers)} * method for each interceptor. * * This method does not throw exceptions. Exceptions thrown by any of interceptor methods are caught and ignored. @@ -85,11 +87,12 @@ public ProducerRecord onSend(ProducerRecord record) { * @param metadata The metadata for the record that was sent (i.e. the partition and offset). * If an error occurred, metadata will only contain valid topic and maybe partition. * @param exception The exception thrown during processing of this record. Null if no error occurred. + * @param headers The headers for the record that was sent */ - public void onAcknowledgement(RecordMetadata metadata, Exception exception) { + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { for (Plugin> interceptorPlugin : this.interceptorPlugins) { try { - interceptorPlugin.get().onAcknowledgement(metadata, exception); + interceptorPlugin.get().onAcknowledgement(metadata, exception, headers); } catch (Exception e) { // do not propagate interceptor exceptions, just log log.warn("Error executing interceptor onAcknowledgement callback", e); @@ -99,7 +102,7 @@ public void onAcknowledgement(RecordMetadata metadata, Exception exception) { /** * This method is called when sending the record fails in {@link ProducerInterceptor#onSend - * (ProducerRecord)} method. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception)} + * (ProducerRecord)} method. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception, Headers)} * method for each interceptor * * @param record The record from client @@ -110,14 +113,22 @@ public void onAcknowledgement(RecordMetadata metadata, Exception exception) { public void onSendError(ProducerRecord record, TopicPartition interceptTopicPartition, Exception exception) { for (Plugin> interceptorPlugin : this.interceptorPlugins) { try { + Headers headers = record != null ? record.headers() : new RecordHeaders(); + if (headers instanceof RecordHeaders && !((RecordHeaders) headers).isReadOnly()) { + // make a copy of the headers to make sure we don't change the state of origin record's headers. + // original headers are still writable because client might want to mutate them before retrying. + RecordHeaders recordHeaders = (RecordHeaders) headers; + headers = new RecordHeaders(recordHeaders); + ((RecordHeaders) headers).setReadOnly(); + } if (record == null && interceptTopicPartition == null) { - interceptorPlugin.get().onAcknowledgement(null, exception); + interceptorPlugin.get().onAcknowledgement(null, exception, headers); } else { if (interceptTopicPartition == null) { interceptTopicPartition = extractTopicPartition(record); } interceptorPlugin.get().onAcknowledgement(new RecordMetadata(interceptTopicPartition, -1, -1, - RecordBatch.NO_TIMESTAMP, -1, -1), exception); + RecordBatch.NO_TIMESTAMP, -1, -1), exception, headers); } } catch (Exception e) { // do not propagate interceptor exceptions, just log diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 614fe562d873e..6739facfc3415 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -52,6 +52,7 @@ import org.apache.kafka.common.requests.ProduceRequest; import org.apache.kafka.common.requests.ProduceResponse; import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -234,9 +235,6 @@ private boolean hasPendingTransactionalRequests() { public void run() { log.debug("Starting Kafka producer I/O thread."); - if (transactionManager != null) - transactionManager.setPoisonStateOnInvalidTransition(true); - // main loop, runs until close is called while (running) { try { @@ -1072,4 +1070,10 @@ void recordBatchSplit() { } } + public static class SenderThread extends KafkaThread { + + public SenderThread(final String name, Runnable runnable, boolean daemon) { + super(name, runnable, daemon); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java index c78134c72ecf2..e17dc15d239fe 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java @@ -120,58 +120,6 @@ public class TransactionManager { private final Set newPartitionsInTransaction; private final Set pendingPartitionsInTransaction; private final Set partitionsInTransaction; - - /** - * During its normal course of operations, the transaction manager transitions through different internal - * states (i.e. by updating {@link #currentState}) to one of those defined in {@link State}. These state transitions - * result from actions on one of the following classes of threads: - * - *

    - *
  • Application threads that invokes {@link Producer} API calls
  • - *
  • {@link Sender} thread operations
  • - *
- * - * When an invalid state transition is detected during execution on an application thread, the - * {@link #currentState} is not updated and an {@link IllegalStateException} is thrown. This gives the - * application the opportunity to fix the issue without permanently poisoning the state of the - * transaction manager. The {@link Producer} API calls that perform a state transition include: - * - *
    - *
  • {@link Producer#initTransactions()} calls {@link #initializeTransactions()}
  • - *
  • {@link Producer#beginTransaction()} calls {@link #beginTransaction()}
  • - *
  • {@link Producer#commitTransaction()}} calls {@link #beginCommit()}
  • - *
  • {@link Producer#abortTransaction()} calls {@link #beginAbort()} - *
  • - *
  • {@link Producer#sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} calls - * {@link #sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} - *
  • - *
  • {@link Producer#send(ProducerRecord)} (and its variants) calls - * {@link #maybeAddPartition(TopicPartition)} and - * {@link #maybeTransitionToErrorState(RuntimeException)} - *
  • - *
- * - *

- * - * The {@link Producer} is implemented such that much of its work delegated to and performed asynchronously on the - * {@link Sender} thread. This includes record batching, network I/O, broker response handlers, etc. If an - * invalid state transition is detected in the {@link Sender} thread, in addition to throwing an - * {@link IllegalStateException}, the transaction manager intentionally "poisons" itself by setting its - * {@link #currentState} to {@link State#FATAL_ERROR}, a state from which it cannot recover. - * - *

- * - * It's important to prevent possible corruption when the transaction manager has determined that it is in a - * fatal state. Subsequent transaction operations attempted via either the application or the - * {@link Sender} thread should fail. This is achieved when these operations invoke the - * {@link #maybeFailWithError()} method, as it causes a {@link KafkaException} to be thrown, ensuring the stated - * transactional guarantees are not violated. - * - *

- * - * See KAFKA-14831 for more detail. - */ - private final ThreadLocal shouldPoisonStateOnInvalidTransition; private PendingStateTransition pendingTransition; // This is used by the TxnRequestHandlers to control how long to back off before a given request is retried. @@ -195,6 +143,7 @@ public class TransactionManager { private volatile boolean clientSideEpochBumpRequired = false; private volatile long latestFinalizedFeaturesEpoch = -1; private volatile boolean isTransactionV2Enabled = false; + private final boolean enable2PC; private enum State { UNINITIALIZED, @@ -255,7 +204,8 @@ public TransactionManager(final LogContext logContext, final String transactionalId, final int transactionTimeoutMs, final long retryBackoffMs, - final ApiVersions apiVersions) { + final ApiVersions apiVersions, + final boolean enable2PC) { this.producerIdAndEpoch = ProducerIdAndEpoch.NONE; this.transactionalId = transactionalId; this.log = logContext.logger(TransactionManager.class); @@ -265,7 +215,6 @@ public TransactionManager(final LogContext logContext, this.newPartitionsInTransaction = new HashSet<>(); this.pendingPartitionsInTransaction = new HashSet<>(); this.partitionsInTransaction = new HashSet<>(); - this.shouldPoisonStateOnInvalidTransition = ThreadLocal.withInitial(() -> false); this.pendingRequests = new PriorityQueue<>(10, Comparator.comparingInt(o -> o.priority().priority)); this.pendingTxnOffsetCommits = new HashMap<>(); this.partitionsWithUnresolvedSequences = new HashMap<>(); @@ -273,17 +222,78 @@ public TransactionManager(final LogContext logContext, this.retryBackoffMs = retryBackoffMs; this.txnPartitionMap = new TxnPartitionMap(logContext); this.apiVersions = apiVersions; + this.enable2PC = enable2PC; } - void setPoisonStateOnInvalidTransition(boolean shouldPoisonState) { - shouldPoisonStateOnInvalidTransition.set(shouldPoisonState); + /** + * During its normal course of operations, the transaction manager transitions through different internal + * states (i.e. by updating {@link #currentState}) to one of those defined in {@link State}. These state transitions + * result from actions on one of the following classes of threads: + * + *

    + *
  • Application threads that invokes {@link Producer} API calls
  • + *
  • {@link Sender} thread operations
  • + *
+ * + * When an invalid state transition is detected during execution on an application thread, the + * {@link #currentState} is not updated and an {@link IllegalStateException} is thrown. This gives the + * application the opportunity to fix the issue without permanently poisoning the state of the + * transaction manager. The {@link Producer} API calls that perform a state transition include: + * + *
    + *
  • {@link Producer#initTransactions()} calls {@link #initializeTransactions()}
  • + *
  • {@link Producer#beginTransaction()} calls {@link #beginTransaction()}
  • + *
  • {@link Producer#commitTransaction()}} calls {@link #beginCommit()}
  • + *
  • {@link Producer#abortTransaction()} calls {@link #beginAbort()} + *
  • + *
  • {@link Producer#sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} calls + * {@link #sendOffsetsToTransaction(Map, ConsumerGroupMetadata)} + *
  • + *
  • {@link Producer#send(ProducerRecord)} (and its variants) calls + * {@link #maybeAddPartition(TopicPartition)} and + * {@link #maybeTransitionToErrorState(RuntimeException)} + *
  • + *
+ * + *

+ * + * The {@link Producer} is implemented such that much of its work delegated to and performed asynchronously on the + * {@link Sender} thread. This includes record batching, network I/O, broker response handlers, etc. If an + * invalid state transition is detected in the {@link Sender} thread, in addition to throwing an + * {@link IllegalStateException}, the transaction manager intentionally "poisons" itself by setting its + * {@link #currentState} to {@link State#FATAL_ERROR}, a state from which it cannot recover. + * + *

+ * + * It's important to prevent possible corruption when the transaction manager has determined that it is in a + * fatal state. Subsequent transaction operations attempted via either the application or the + * {@link Sender} thread should fail. This is achieved when these operations invoke the + * {@link #maybeFailWithError()} method, as it causes a {@link KafkaException} to be thrown, ensuring the stated + * transactional guarantees are not violated. + * + *

+ * + * See KAFKA-14831 for more detail. + * + * @return {@code true} to set state to {@link State#FATAL_ERROR} before throwing an exception, + * {@code false} to throw an exception without first changing the state + */ + protected boolean shouldPoisonStateOnInvalidTransition() { + return Thread.currentThread() instanceof Sender.SenderThread; } - public synchronized TransactionalRequestResult initializeTransactions() { - return initializeTransactions(ProducerIdAndEpoch.NONE); + synchronized TransactionalRequestResult initializeTransactions(ProducerIdAndEpoch producerIdAndEpoch) { + return initializeTransactions(producerIdAndEpoch, false); } - synchronized TransactionalRequestResult initializeTransactions(ProducerIdAndEpoch producerIdAndEpoch) { + public synchronized TransactionalRequestResult initializeTransactions(boolean keepPreparedTxn) { + return initializeTransactions(ProducerIdAndEpoch.NONE, keepPreparedTxn); + } + + synchronized TransactionalRequestResult initializeTransactions( + ProducerIdAndEpoch producerIdAndEpoch, + boolean keepPreparedTxn + ) { maybeFailWithError(); boolean isEpochBump = producerIdAndEpoch != ProducerIdAndEpoch.NONE; @@ -292,6 +302,9 @@ synchronized TransactionalRequestResult initializeTransactions(ProducerIdAndEpoc if (!isEpochBump) { transitionTo(State.INITIALIZING); log.info("Invoking InitProducerId for the first time in order to acquire a producer ID"); + if (keepPreparedTxn) { + log.info("Invoking InitProducerId with keepPreparedTxn set to true for 2PC transactions"); + } } else { log.info("Invoking InitProducerId with current producer ID and epoch {} in order to bump the epoch", producerIdAndEpoch); } @@ -299,7 +312,10 @@ synchronized TransactionalRequestResult initializeTransactions(ProducerIdAndEpoc .setTransactionalId(transactionalId) .setTransactionTimeoutMs(transactionTimeoutMs) .setProducerId(producerIdAndEpoch.producerId) - .setProducerEpoch(producerIdAndEpoch.epoch); + .setProducerEpoch(producerIdAndEpoch.epoch) + .setEnable2Pc(enable2PC) + .setKeepPreparedTxn(keepPreparedTxn); + InitProducerIdHandler handler = new InitProducerIdHandler(new InitProducerIdRequest.Builder(requestData), isEpochBump); enqueueRequest(handler); @@ -1063,7 +1079,7 @@ private void transitionTo(State target, RuntimeException error) { String message = idString + "Invalid transition attempted from state " + currentState.name() + " to state " + target.name(); - if (shouldPoisonStateOnInvalidTransition.get()) { + if (shouldPoisonStateOnInvalidTransition()) { currentState = State.FATAL_ERROR; lastError = new IllegalStateException(message); throw lastError; diff --git a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java index 52863c6c0b564..6311f6ca34825 100644 --- a/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java +++ b/clients/src/main/java/org/apache/kafka/common/header/internals/RecordHeaders.java @@ -108,6 +108,10 @@ public void setReadOnly() { this.isReadOnly = true; } + public boolean isReadOnly() { + return isReadOnly; + } + public Header[] toArray() { return headers.isEmpty() ? Record.EMPTY_HEADERS : headers.toArray(new Header[0]); } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java b/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java index f643f5b5779b1..c3e2886e656a7 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ByteBufferAccessor.java @@ -145,6 +145,11 @@ public int remaining() { return buf.remaining(); } + @Override + public Readable slice() { + return new ByteBufferAccessor(buf.slice()); + } + public void flip() { buf.flip(); } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java index 80bee86748269..a00a7dae596c3 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Readable.java @@ -39,6 +39,15 @@ public interface Readable { long readVarlong(); int remaining(); + /** + * Returns a new Readable object whose content will be shared with this object. + *
+ * The content of the new Readable object will start at this Readable's current + * position. The two Readable position will be independent, so read from one will + * not impact the other. + */ + Readable slice(); + default String readString(int length) { byte[] arr = readArray(length); return new String(arr, StandardCharsets.UTF_8); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java index 5bf6186d049ea..01cc03c12b625 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java @@ -18,8 +18,10 @@ import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.SendBuilder; import java.nio.ByteBuffer; @@ -51,8 +53,8 @@ final ByteBuffer serializeWithHeader(ResponseHeader header, short version) { } // Visible for testing - final ByteBuffer serialize(short version) { - return MessageUtil.toByteBufferAccessor(data(), version).buffer(); + final ByteBufferAccessor serialize(short version) { + return MessageUtil.toByteBufferAccessor(data(), version); } /** @@ -106,189 +108,189 @@ public static AbstractResponse parseResponse(ByteBuffer buffer, RequestHeader re requestHeader.correlationId(), responseHeader.correlationId()); } - return AbstractResponse.parseResponse(apiKey, buffer, apiVersion); + return AbstractResponse.parseResponse(apiKey, new ByteBufferAccessor(buffer), apiVersion); } - public static AbstractResponse parseResponse(ApiKeys apiKey, ByteBuffer responseBuffer, short version) { + public static AbstractResponse parseResponse(ApiKeys apiKey, Readable readable, short version) { switch (apiKey) { case PRODUCE: - return ProduceResponse.parse(responseBuffer, version); + return ProduceResponse.parse(readable, version); case FETCH: - return FetchResponse.parse(responseBuffer, version); + return FetchResponse.parse(readable, version); case LIST_OFFSETS: - return ListOffsetsResponse.parse(responseBuffer, version); + return ListOffsetsResponse.parse(readable, version); case METADATA: - return MetadataResponse.parse(responseBuffer, version); + return MetadataResponse.parse(readable, version); case OFFSET_COMMIT: - return OffsetCommitResponse.parse(responseBuffer, version); + return OffsetCommitResponse.parse(readable, version); case OFFSET_FETCH: - return OffsetFetchResponse.parse(responseBuffer, version); + return OffsetFetchResponse.parse(readable, version); case FIND_COORDINATOR: - return FindCoordinatorResponse.parse(responseBuffer, version); + return FindCoordinatorResponse.parse(readable, version); case JOIN_GROUP: - return JoinGroupResponse.parse(responseBuffer, version); + return JoinGroupResponse.parse(readable, version); case HEARTBEAT: - return HeartbeatResponse.parse(responseBuffer, version); + return HeartbeatResponse.parse(readable, version); case LEAVE_GROUP: - return LeaveGroupResponse.parse(responseBuffer, version); + return LeaveGroupResponse.parse(readable, version); case SYNC_GROUP: - return SyncGroupResponse.parse(responseBuffer, version); + return SyncGroupResponse.parse(readable, version); case DESCRIBE_GROUPS: - return DescribeGroupsResponse.parse(responseBuffer, version); + return DescribeGroupsResponse.parse(readable, version); case LIST_GROUPS: - return ListGroupsResponse.parse(responseBuffer, version); + return ListGroupsResponse.parse(readable, version); case SASL_HANDSHAKE: - return SaslHandshakeResponse.parse(responseBuffer, version); + return SaslHandshakeResponse.parse(readable, version); case API_VERSIONS: - return ApiVersionsResponse.parse(responseBuffer, version); + return ApiVersionsResponse.parse(readable, version); case CREATE_TOPICS: - return CreateTopicsResponse.parse(responseBuffer, version); + return CreateTopicsResponse.parse(readable, version); case DELETE_TOPICS: - return DeleteTopicsResponse.parse(responseBuffer, version); + return DeleteTopicsResponse.parse(readable, version); case DELETE_RECORDS: - return DeleteRecordsResponse.parse(responseBuffer, version); + return DeleteRecordsResponse.parse(readable, version); case INIT_PRODUCER_ID: - return InitProducerIdResponse.parse(responseBuffer, version); + return InitProducerIdResponse.parse(readable, version); case OFFSET_FOR_LEADER_EPOCH: - return OffsetsForLeaderEpochResponse.parse(responseBuffer, version); + return OffsetsForLeaderEpochResponse.parse(readable, version); case ADD_PARTITIONS_TO_TXN: - return AddPartitionsToTxnResponse.parse(responseBuffer, version); + return AddPartitionsToTxnResponse.parse(readable, version); case ADD_OFFSETS_TO_TXN: - return AddOffsetsToTxnResponse.parse(responseBuffer, version); + return AddOffsetsToTxnResponse.parse(readable, version); case END_TXN: - return EndTxnResponse.parse(responseBuffer, version); + return EndTxnResponse.parse(readable, version); case WRITE_TXN_MARKERS: - return WriteTxnMarkersResponse.parse(responseBuffer, version); + return WriteTxnMarkersResponse.parse(readable, version); case TXN_OFFSET_COMMIT: - return TxnOffsetCommitResponse.parse(responseBuffer, version); + return TxnOffsetCommitResponse.parse(readable, version); case DESCRIBE_ACLS: - return DescribeAclsResponse.parse(responseBuffer, version); + return DescribeAclsResponse.parse(readable, version); case CREATE_ACLS: - return CreateAclsResponse.parse(responseBuffer, version); + return CreateAclsResponse.parse(readable, version); case DELETE_ACLS: - return DeleteAclsResponse.parse(responseBuffer, version); + return DeleteAclsResponse.parse(readable, version); case DESCRIBE_CONFIGS: - return DescribeConfigsResponse.parse(responseBuffer, version); + return DescribeConfigsResponse.parse(readable, version); case ALTER_CONFIGS: - return AlterConfigsResponse.parse(responseBuffer, version); + return AlterConfigsResponse.parse(readable, version); case ALTER_REPLICA_LOG_DIRS: - return AlterReplicaLogDirsResponse.parse(responseBuffer, version); + return AlterReplicaLogDirsResponse.parse(readable, version); case DESCRIBE_LOG_DIRS: - return DescribeLogDirsResponse.parse(responseBuffer, version); + return DescribeLogDirsResponse.parse(readable, version); case SASL_AUTHENTICATE: - return SaslAuthenticateResponse.parse(responseBuffer, version); + return SaslAuthenticateResponse.parse(readable, version); case CREATE_PARTITIONS: - return CreatePartitionsResponse.parse(responseBuffer, version); + return CreatePartitionsResponse.parse(readable, version); case CREATE_DELEGATION_TOKEN: - return CreateDelegationTokenResponse.parse(responseBuffer, version); + return CreateDelegationTokenResponse.parse(readable, version); case RENEW_DELEGATION_TOKEN: - return RenewDelegationTokenResponse.parse(responseBuffer, version); + return RenewDelegationTokenResponse.parse(readable, version); case EXPIRE_DELEGATION_TOKEN: - return ExpireDelegationTokenResponse.parse(responseBuffer, version); + return ExpireDelegationTokenResponse.parse(readable, version); case DESCRIBE_DELEGATION_TOKEN: - return DescribeDelegationTokenResponse.parse(responseBuffer, version); + return DescribeDelegationTokenResponse.parse(readable, version); case DELETE_GROUPS: - return DeleteGroupsResponse.parse(responseBuffer, version); + return DeleteGroupsResponse.parse(readable, version); case ELECT_LEADERS: - return ElectLeadersResponse.parse(responseBuffer, version); + return ElectLeadersResponse.parse(readable, version); case INCREMENTAL_ALTER_CONFIGS: - return IncrementalAlterConfigsResponse.parse(responseBuffer, version); + return IncrementalAlterConfigsResponse.parse(readable, version); case ALTER_PARTITION_REASSIGNMENTS: - return AlterPartitionReassignmentsResponse.parse(responseBuffer, version); + return AlterPartitionReassignmentsResponse.parse(readable, version); case LIST_PARTITION_REASSIGNMENTS: - return ListPartitionReassignmentsResponse.parse(responseBuffer, version); + return ListPartitionReassignmentsResponse.parse(readable, version); case OFFSET_DELETE: - return OffsetDeleteResponse.parse(responseBuffer, version); + return OffsetDeleteResponse.parse(readable, version); case DESCRIBE_CLIENT_QUOTAS: - return DescribeClientQuotasResponse.parse(responseBuffer, version); + return DescribeClientQuotasResponse.parse(readable, version); case ALTER_CLIENT_QUOTAS: - return AlterClientQuotasResponse.parse(responseBuffer, version); + return AlterClientQuotasResponse.parse(readable, version); case DESCRIBE_USER_SCRAM_CREDENTIALS: - return DescribeUserScramCredentialsResponse.parse(responseBuffer, version); + return DescribeUserScramCredentialsResponse.parse(readable, version); case ALTER_USER_SCRAM_CREDENTIALS: - return AlterUserScramCredentialsResponse.parse(responseBuffer, version); + return AlterUserScramCredentialsResponse.parse(readable, version); case VOTE: - return VoteResponse.parse(responseBuffer, version); + return VoteResponse.parse(readable, version); case BEGIN_QUORUM_EPOCH: - return BeginQuorumEpochResponse.parse(responseBuffer, version); + return BeginQuorumEpochResponse.parse(readable, version); case END_QUORUM_EPOCH: - return EndQuorumEpochResponse.parse(responseBuffer, version); + return EndQuorumEpochResponse.parse(readable, version); case DESCRIBE_QUORUM: - return DescribeQuorumResponse.parse(responseBuffer, version); + return DescribeQuorumResponse.parse(readable, version); case ALTER_PARTITION: - return AlterPartitionResponse.parse(responseBuffer, version); + return AlterPartitionResponse.parse(readable, version); case UPDATE_FEATURES: - return UpdateFeaturesResponse.parse(responseBuffer, version); + return UpdateFeaturesResponse.parse(readable, version); case ENVELOPE: - return EnvelopeResponse.parse(responseBuffer, version); + return EnvelopeResponse.parse(readable, version); case FETCH_SNAPSHOT: - return FetchSnapshotResponse.parse(responseBuffer, version); + return FetchSnapshotResponse.parse(readable, version); case DESCRIBE_CLUSTER: - return DescribeClusterResponse.parse(responseBuffer, version); + return DescribeClusterResponse.parse(readable, version); case DESCRIBE_PRODUCERS: - return DescribeProducersResponse.parse(responseBuffer, version); + return DescribeProducersResponse.parse(readable, version); case BROKER_REGISTRATION: - return BrokerRegistrationResponse.parse(responseBuffer, version); + return BrokerRegistrationResponse.parse(readable, version); case BROKER_HEARTBEAT: - return BrokerHeartbeatResponse.parse(responseBuffer, version); + return BrokerHeartbeatResponse.parse(readable, version); case UNREGISTER_BROKER: - return UnregisterBrokerResponse.parse(responseBuffer, version); + return UnregisterBrokerResponse.parse(readable, version); case DESCRIBE_TRANSACTIONS: - return DescribeTransactionsResponse.parse(responseBuffer, version); + return DescribeTransactionsResponse.parse(readable, version); case LIST_TRANSACTIONS: - return ListTransactionsResponse.parse(responseBuffer, version); + return ListTransactionsResponse.parse(readable, version); case ALLOCATE_PRODUCER_IDS: - return AllocateProducerIdsResponse.parse(responseBuffer, version); + return AllocateProducerIdsResponse.parse(readable, version); case CONSUMER_GROUP_HEARTBEAT: - return ConsumerGroupHeartbeatResponse.parse(responseBuffer, version); + return ConsumerGroupHeartbeatResponse.parse(readable, version); case CONSUMER_GROUP_DESCRIBE: - return ConsumerGroupDescribeResponse.parse(responseBuffer, version); + return ConsumerGroupDescribeResponse.parse(readable, version); case CONTROLLER_REGISTRATION: - return ControllerRegistrationResponse.parse(responseBuffer, version); + return ControllerRegistrationResponse.parse(readable, version); case GET_TELEMETRY_SUBSCRIPTIONS: - return GetTelemetrySubscriptionsResponse.parse(responseBuffer, version); + return GetTelemetrySubscriptionsResponse.parse(readable, version); case PUSH_TELEMETRY: - return PushTelemetryResponse.parse(responseBuffer, version); + return PushTelemetryResponse.parse(readable, version); case ASSIGN_REPLICAS_TO_DIRS: - return AssignReplicasToDirsResponse.parse(responseBuffer, version); + return AssignReplicasToDirsResponse.parse(readable, version); case LIST_CLIENT_METRICS_RESOURCES: - return ListClientMetricsResourcesResponse.parse(responseBuffer, version); + return ListClientMetricsResourcesResponse.parse(readable, version); case DESCRIBE_TOPIC_PARTITIONS: - return DescribeTopicPartitionsResponse.parse(responseBuffer, version); + return DescribeTopicPartitionsResponse.parse(readable, version); case SHARE_GROUP_HEARTBEAT: - return ShareGroupHeartbeatResponse.parse(responseBuffer, version); + return ShareGroupHeartbeatResponse.parse(readable, version); case SHARE_GROUP_DESCRIBE: - return ShareGroupDescribeResponse.parse(responseBuffer, version); + return ShareGroupDescribeResponse.parse(readable, version); case SHARE_FETCH: - return ShareFetchResponse.parse(responseBuffer, version); + return ShareFetchResponse.parse(readable, version); case SHARE_ACKNOWLEDGE: - return ShareAcknowledgeResponse.parse(responseBuffer, version); + return ShareAcknowledgeResponse.parse(readable, version); case ADD_RAFT_VOTER: - return AddRaftVoterResponse.parse(responseBuffer, version); + return AddRaftVoterResponse.parse(readable, version); case REMOVE_RAFT_VOTER: - return RemoveRaftVoterResponse.parse(responseBuffer, version); + return RemoveRaftVoterResponse.parse(readable, version); case UPDATE_RAFT_VOTER: - return UpdateRaftVoterResponse.parse(responseBuffer, version); + return UpdateRaftVoterResponse.parse(readable, version); case INITIALIZE_SHARE_GROUP_STATE: - return InitializeShareGroupStateResponse.parse(responseBuffer, version); + return InitializeShareGroupStateResponse.parse(readable, version); case READ_SHARE_GROUP_STATE: - return ReadShareGroupStateResponse.parse(responseBuffer, version); + return ReadShareGroupStateResponse.parse(readable, version); case WRITE_SHARE_GROUP_STATE: - return WriteShareGroupStateResponse.parse(responseBuffer, version); + return WriteShareGroupStateResponse.parse(readable, version); case DELETE_SHARE_GROUP_STATE: - return DeleteShareGroupStateResponse.parse(responseBuffer, version); + return DeleteShareGroupStateResponse.parse(readable, version); case READ_SHARE_GROUP_STATE_SUMMARY: - return ReadShareGroupStateSummaryResponse.parse(responseBuffer, version); + return ReadShareGroupStateSummaryResponse.parse(readable, version); case STREAMS_GROUP_HEARTBEAT: - return StreamsGroupHeartbeatResponse.parse(responseBuffer, version); + return StreamsGroupHeartbeatResponse.parse(readable, version); case STREAMS_GROUP_DESCRIBE: - return StreamsGroupDescribeResponse.parse(responseBuffer, version); + return StreamsGroupDescribeResponse.parse(readable, version); case DESCRIBE_SHARE_GROUP_OFFSETS: - return DescribeShareGroupOffsetsResponse.parse(responseBuffer, version); + return DescribeShareGroupOffsetsResponse.parse(readable, version); case ALTER_SHARE_GROUP_OFFSETS: - return AlterShareGroupOffsetsResponse.parse(responseBuffer, version); + return AlterShareGroupOffsetsResponse.parse(readable, version); case DELETE_SHARE_GROUP_OFFSETS: - return DeleteShareGroupOffsetsResponse.parse(responseBuffer, version); + return DeleteShareGroupOffsetsResponse.parse(readable, version); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `parseResponse`, the " + "code should be updated to do so.", apiKey)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java index d90afd04ddcde..ad3946b432b6f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; /** @@ -66,8 +65,8 @@ public AddOffsetsToTxnResponseData data() { return data; } - public static AddOffsetsToTxnResponse parse(ByteBuffer buffer, short version) { - return new AddOffsetsToTxnResponse(new AddOffsetsToTxnResponseData(new ByteBufferAccessor(buffer), version)); + public static AddOffsetsToTxnResponse parse(Readable readable, short version) { + return new AddOffsetsToTxnResponse(new AddOffsetsToTxnResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java index 0abf85bf0239a..645fd667186b8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java @@ -24,10 +24,9 @@ import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnTopicResult; import org.apache.kafka.common.message.AddPartitionsToTxnResponseData.AddPartitionsToTxnTopicResultCollection; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -150,8 +149,8 @@ public AddPartitionsToTxnResponseData data() { return data; } - public static AddPartitionsToTxnResponse parse(ByteBuffer buffer, short version) { - return new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData(new ByteBufferAccessor(buffer), version)); + public static AddPartitionsToTxnResponse parse(Readable readable, short version) { + return new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddRaftVoterResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddRaftVoterResponse.java index ab0600b618406..52a0cb05feb76 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddRaftVoterResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddRaftVoterResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.AddRaftVoterResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -58,8 +57,8 @@ public Map errorCounts() { } } - public static AddRaftVoterResponse parse(ByteBuffer buffer, short version) { + public static AddRaftVoterResponse parse(Readable readable, short version) { return new AddRaftVoterResponse( - new AddRaftVoterResponseData(new ByteBufferAccessor(buffer), version)); + new AddRaftVoterResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java index 2511e2b2db320..4c47651193188 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AllocateProducerIdsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.AllocateProducerIdsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -65,8 +64,8 @@ public Errors error() { return Errors.forCode(data.errorCode()); } - public static AllocateProducerIdsResponse parse(ByteBuffer buffer, short version) { + public static AllocateProducerIdsResponse parse(Readable readable, short version) { return new AllocateProducerIdsResponse(new AllocateProducerIdsResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java index 9987b47be1d2d..9c4990dd3c719 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterClientQuotasResponse.java @@ -21,11 +21,10 @@ import org.apache.kafka.common.message.AlterClientQuotasResponseData.EntityData; import org.apache.kafka.common.message.AlterClientQuotasResponseData.EntryData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.quota.ClientQuotaEntity; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.EnumMap; import java.util.HashMap; @@ -97,8 +96,8 @@ private static List toEntityData(ClientQuotaEntity entity) { return entityData; } - public static AlterClientQuotasResponse parse(ByteBuffer buffer, short version) { - return new AlterClientQuotasResponse(new AlterClientQuotasResponseData(new ByteBufferAccessor(buffer), version)); + public static AlterClientQuotasResponse parse(Readable readable, short version) { + return new AlterClientQuotasResponse(new AlterClientQuotasResponseData(readable, version)); } public static AlterClientQuotasResponse fromQuotaEntities(Map result, int throttleTimeMs) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java index 1668c2446bc77..5f7b9421cb8e9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterConfigsResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.AlterConfigsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; import java.util.stream.Collectors; @@ -65,8 +64,8 @@ public AlterConfigsResponseData data() { return data; } - public static AlterConfigsResponse parse(ByteBuffer buffer, short version) { - return new AlterConfigsResponse(new AlterConfigsResponseData(new ByteBufferAccessor(buffer), version)); + public static AlterConfigsResponse parse(Readable readable, short version) { + return new AlterConfigsResponse(new AlterConfigsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java index ba9a875616d7c..691a399761dec 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionReassignmentsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.AlterPartitionReassignmentsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -35,9 +34,9 @@ public AlterPartitionReassignmentsResponse(AlterPartitionReassignmentsResponseDa this.data = data; } - public static AlterPartitionReassignmentsResponse parse(ByteBuffer buffer, short version) { + public static AlterPartitionReassignmentsResponse parse(Readable readable, short version) { return new AlterPartitionReassignmentsResponse( - new AlterPartitionReassignmentsResponseData(new ByteBufferAccessor(buffer), version)); + new AlterPartitionReassignmentsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java index a6f9809a6efbc..580be92b8999e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.AlterPartitionResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -60,7 +59,7 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static AlterPartitionResponse parse(ByteBuffer buffer, short version) { - return new AlterPartitionResponse(new AlterPartitionResponseData(new ByteBufferAccessor(buffer), version)); + public static AlterPartitionResponse parse(Readable readable, short version) { + return new AlterPartitionResponse(new AlterPartitionResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java index bf715ae0575c0..755fc98038a26 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterReplicaLogDirsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.AlterReplicaLogDirsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -67,8 +66,8 @@ public Map errorCounts() { return errorCounts; } - public static AlterReplicaLogDirsResponse parse(ByteBuffer buffer, short version) { - return new AlterReplicaLogDirsResponse(new AlterReplicaLogDirsResponseData(new ByteBufferAccessor(buffer), version)); + public static AlterReplicaLogDirsResponse parse(Readable readable, short version) { + return new AlterReplicaLogDirsResponse(new AlterReplicaLogDirsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AlterShareGroupOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AlterShareGroupOffsetsRequest.java index 3417c1ae9c5aa..6c663e3a40e1f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AlterShareGroupOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AlterShareGroupOffsetsRequest.java @@ -41,11 +41,7 @@ public static class Builder extends AbstractRequest.Builder errorCounts() { return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode()))); } - public static AlterUserScramCredentialsResponse parse(ByteBuffer buffer, short version) { - return new AlterUserScramCredentialsResponse(new AlterUserScramCredentialsResponseData(new ByteBufferAccessor(buffer), version)); + public static AlterUserScramCredentialsResponse parse(Readable readable, short version) { + return new AlterUserScramCredentialsResponse(new AlterUserScramCredentialsResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 324e527984d08..daaa7ba2fce7b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -29,10 +29,9 @@ import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKey; import org.apache.kafka.common.message.ApiVersionsResponseData.SupportedFeatureKeyCollection; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -152,19 +151,18 @@ public boolean zkMigrationReady() { return data.zkMigrationReady(); } - public static ApiVersionsResponse parse(ByteBuffer buffer, short version) { + public static ApiVersionsResponse parse(Readable readable, short version) { // Fallback to version 0 for ApiVersions response. If a client sends an ApiVersionsRequest // using a version higher than that supported by the broker, a version 0 response is sent // to the client indicating UNSUPPORTED_VERSION. When the client receives the response, it // falls back while parsing it which means that the version received by this // method is not necessarily the real one. It may be version 0 as well. - int prev = buffer.position(); + Readable readableCopy = readable.slice(); try { - return new ApiVersionsResponse(new ApiVersionsResponseData(new ByteBufferAccessor(buffer), version)); + return new ApiVersionsResponse(new ApiVersionsResponseData(readable, version)); } catch (RuntimeException e) { - buffer.position(prev); if (version != 0) - return new ApiVersionsResponse(new ApiVersionsResponseData(new ByteBufferAccessor(buffer), (short) 0)); + return new ApiVersionsResponse(new ApiVersionsResponseData(readableCopy, (short) 0)); else throw e; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AssignReplicasToDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AssignReplicasToDirsResponse.java index 90912956029cc..84f86d058ec65 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AssignReplicasToDirsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AssignReplicasToDirsResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.AssignReplicasToDirsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -54,8 +53,8 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static AssignReplicasToDirsResponse parse(ByteBuffer buffer, short version) { + public static AssignReplicasToDirsResponse parse(Readable readable, short version) { return new AssignReplicasToDirsResponse(new AssignReplicasToDirsResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java index 3801d49ab9336..7f77c10e93859 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BeginQuorumEpochResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -77,8 +76,8 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { // Not supported by the response schema } - public static BeginQuorumEpochResponse parse(ByteBuffer buffer, short version) { - return new BeginQuorumEpochResponse(new BeginQuorumEpochResponseData(new ByteBufferAccessor(buffer), version)); + public static BeginQuorumEpochResponse parse(Readable readable, short version) { + return new BeginQuorumEpochResponse(new BeginQuorumEpochResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java index b6521666ab57e..f46e56ca50eef 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BrokerHeartbeatResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.BrokerHeartbeatResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -56,8 +55,8 @@ public Map errorCounts() { return errorCounts; } - public static BrokerHeartbeatResponse parse(ByteBuffer buffer, short version) { - return new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData(new ByteBufferAccessor(buffer), version)); + public static BrokerHeartbeatResponse parse(Readable readable, short version) { + return new BrokerHeartbeatResponse(new BrokerHeartbeatResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java index e636b84116993..be8a2f1f50601 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/BrokerRegistrationResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.BrokerRegistrationResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -56,8 +55,8 @@ public Map errorCounts() { return errorCounts; } - public static BrokerRegistrationResponse parse(ByteBuffer buffer, short version) { - return new BrokerRegistrationResponse(new BrokerRegistrationResponseData(new ByteBufferAccessor(buffer), version)); + public static BrokerRegistrationResponse parse(Readable readable, short version) { + return new BrokerRegistrationResponse(new BrokerRegistrationResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupDescribeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupDescribeResponse.java index 0f3458af85a77..25ec8b6fbe235 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupDescribeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupDescribeResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -70,9 +69,9 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static ConsumerGroupDescribeResponse parse(ByteBuffer buffer, short version) { + public static ConsumerGroupDescribeResponse parse(Readable readable, short version) { return new ConsumerGroupDescribeResponse( - new ConsumerGroupDescribeResponseData(new ByteBufferAccessor(buffer), version) + new ConsumerGroupDescribeResponseData(readable, version) ); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupHeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupHeartbeatResponse.java index 2d8b91c3d24bc..1d6fb4e682d67 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupHeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerGroupHeartbeatResponse.java @@ -16,14 +16,18 @@ */ package org.apache.kafka.common.requests; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; /** * Possible error codes. @@ -69,8 +73,21 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static ConsumerGroupHeartbeatResponse parse(ByteBuffer buffer, short version) { + public static ConsumerGroupHeartbeatResponse parse(Readable readable, short version) { return new ConsumerGroupHeartbeatResponse(new ConsumerGroupHeartbeatResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); + } + + public static ConsumerGroupHeartbeatResponseData.Assignment createAssignment( + Map> assignment + ) { + List topicPartitions = assignment.entrySet().stream() + .map(keyValue -> new ConsumerGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(keyValue.getKey()) + .setPartitions(new ArrayList<>(keyValue.getValue()))) + .collect(Collectors.toList()); + + return new ConsumerGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(topicPartitions); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ControllerRegistrationResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ControllerRegistrationResponse.java index d44e915b5fa5a..9cc53db15a9a4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ControllerRegistrationResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ControllerRegistrationResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.ControllerRegistrationResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -54,8 +53,8 @@ public Map errorCounts() { return Collections.singletonMap(Errors.forCode(data.errorCode()), 1); } - public static ControllerRegistrationResponse parse(ByteBuffer buffer, short version) { + public static ControllerRegistrationResponse parse(Readable readable, short version) { return new ControllerRegistrationResponse( - new ControllerRegistrationResponseData(new ByteBufferAccessor(buffer), version)); + new ControllerRegistrationResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java index cef7b73ac27e9..2f2877ad510f7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateAclsResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.CreateAclsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -57,8 +56,8 @@ public Map errorCounts() { return errorCounts(results().stream().map(r -> Errors.forCode(r.errorCode()))); } - public static CreateAclsResponse parse(ByteBuffer buffer, short version) { - return new CreateAclsResponse(new CreateAclsResponseData(new ByteBufferAccessor(buffer), version)); + public static CreateAclsResponse parse(Readable readable, short version) { + return new CreateAclsResponse(new CreateAclsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java index 0a9f9a8991bdc..ce577d48d9779 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateDelegationTokenResponse.java @@ -18,8 +18,8 @@ import org.apache.kafka.common.message.CreateDelegationTokenResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.security.auth.KafkaPrincipal; import java.nio.ByteBuffer; @@ -34,9 +34,9 @@ public CreateDelegationTokenResponse(CreateDelegationTokenResponseData data) { this.data = data; } - public static CreateDelegationTokenResponse parse(ByteBuffer buffer, short version) { + public static CreateDelegationTokenResponse parse(Readable readable, short version) { return new CreateDelegationTokenResponse( - new CreateDelegationTokenResponseData(new ByteBufferAccessor(buffer), version)); + new CreateDelegationTokenResponseData(readable, version)); } public static CreateDelegationTokenResponse prepareResponse(int version, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java index 485a4a34020a4..86d8672e19b34 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreatePartitionsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.CreatePartitionsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -49,8 +48,8 @@ public Map errorCounts() { return counts; } - public static CreatePartitionsResponse parse(ByteBuffer buffer, short version) { - return new CreatePartitionsResponse(new CreatePartitionsResponseData(new ByteBufferAccessor(buffer), version)); + public static CreatePartitionsResponse parse(Readable readable, short version) { + return new CreatePartitionsResponse(new CreatePartitionsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java index baed1af16fe2f..be36ff7df5b77 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.CreateTopicsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -74,8 +73,8 @@ public Map errorCounts() { return counts; } - public static CreateTopicsResponse parse(ByteBuffer buffer, short version) { - return new CreateTopicsResponse(new CreateTopicsResponseData(new ByteBufferAccessor(buffer), version)); + public static CreateTopicsResponse parse(Readable readable, short version) { + return new CreateTopicsResponse(new CreateTopicsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java index 6b759d7cee5f5..789c00148699c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteAclsResponse.java @@ -25,8 +25,8 @@ import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsMatchingAcl; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; @@ -35,7 +35,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -75,8 +74,8 @@ public Map errorCounts() { return errorCounts(filterResults().stream().map(r -> Errors.forCode(r.errorCode()))); } - public static DeleteAclsResponse parse(ByteBuffer buffer, short version) { - return new DeleteAclsResponse(new DeleteAclsResponseData(new ByteBufferAccessor(buffer), version), version); + public static DeleteAclsResponse parse(Readable readable, short version) { + return new DeleteAclsResponse(new DeleteAclsResponseData(readable, version), version); } public String toString() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java index f41abe1a1449d..d1939581a08e0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteGroupsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.DeleteGroupsResponseData; import org.apache.kafka.common.message.DeleteGroupsResponseData.DeletableGroupResult; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.HashMap; import java.util.Map; @@ -77,8 +76,8 @@ public Map errorCounts() { return counts; } - public static DeleteGroupsResponse parse(ByteBuffer buffer, short version) { - return new DeleteGroupsResponse(new DeleteGroupsResponseData(new ByteBufferAccessor(buffer), version)); + public static DeleteGroupsResponse parse(Readable readable, short version) { + return new DeleteGroupsResponse(new DeleteGroupsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java index 4db4f4b66f565..b440dec72dc4e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteRecordsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.DeleteRecordsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -72,8 +71,8 @@ public Map errorCounts() { return errorCounts; } - public static DeleteRecordsResponse parse(ByteBuffer buffer, short version) { - return new DeleteRecordsResponse(new DeleteRecordsResponseData(new ByteBufferAccessor(buffer), version)); + public static DeleteRecordsResponse parse(Readable readable, short version) { + return new DeleteRecordsResponse(new DeleteRecordsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DeleteShareGroupOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DeleteShareGroupOffsetsRequest.java index f96bad8d17836..bec0077b9b3c5 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DeleteShareGroupOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DeleteShareGroupOffsetsRequest.java @@ -29,11 +29,7 @@ public static class Builder extends AbstractRequest.Builder errorCounts() { return counts; } - public static DeleteTopicsResponse parse(ByteBuffer buffer, short version) { - return new DeleteTopicsResponse(new DeleteTopicsResponseData(new ByteBufferAccessor(buffer), version)); + public static DeleteTopicsResponse parse(Readable readable, short version) { + return new DeleteTopicsResponse(new DeleteTopicsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java index 424ff563a4c8a..c387dbd4da3b3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeAclsResponse.java @@ -26,13 +26,12 @@ import org.apache.kafka.common.message.DescribeAclsResponseData.AclDescription; import org.apache.kafka.common.message.DescribeAclsResponseData.DescribeAclsResource; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -89,8 +88,8 @@ public final List acls() { return data.resources(); } - public static DescribeAclsResponse parse(ByteBuffer buffer, short version) { - return new DescribeAclsResponse(new DescribeAclsResponseData(new ByteBufferAccessor(buffer), version), version); + public static DescribeAclsResponse parse(Readable readable, short version) { + return new DescribeAclsResponse(new DescribeAclsResponseData(readable, version), version); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java index 3a052c9fe8eba..5d9a980ea3c36 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClientQuotasResponse.java @@ -22,11 +22,10 @@ import org.apache.kafka.common.message.DescribeClientQuotasResponseData.EntryData; import org.apache.kafka.common.message.DescribeClientQuotasResponseData.ValueData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.quota.ClientQuotaEntity; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -85,8 +84,8 @@ public Map errorCounts() { return errorCounts(Errors.forCode(data.errorCode())); } - public static DescribeClientQuotasResponse parse(ByteBuffer buffer, short version) { - return new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData(new ByteBufferAccessor(buffer), version)); + public static DescribeClientQuotasResponse parse(Readable readable, short version) { + return new DescribeClientQuotasResponse(new DescribeClientQuotasResponseData(readable, version)); } public static DescribeClientQuotasResponse fromQuotaEntities(Map> entities, diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java index 7c892874214e8..0b8724a57602b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeClusterResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.message.DescribeClusterResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; @@ -63,7 +62,7 @@ public DescribeClusterResponseData data() { return data; } - public static DescribeClusterResponse parse(ByteBuffer buffer, short version) { - return new DescribeClusterResponse(new DescribeClusterResponseData(new ByteBufferAccessor(buffer), version)); + public static DescribeClusterResponse parse(Readable readable, short version) { + return new DescribeClusterResponse(new DescribeClusterResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java index c8c3fab584c7a..f291af273b84a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeConfigsResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.message.DescribeConfigsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collection; import java.util.EnumMap; import java.util.Map; @@ -248,8 +247,8 @@ public Map errorCounts() { return errorCounts; } - public static DescribeConfigsResponse parse(ByteBuffer buffer, short version) { - return new DescribeConfigsResponse(new DescribeConfigsResponseData(new ByteBufferAccessor(buffer), version)); + public static DescribeConfigsResponse parse(Readable readable, short version) { + return new DescribeConfigsResponse(new DescribeConfigsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java index a922f056a89aa..d0476a3772caf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeDelegationTokenResponse.java @@ -20,13 +20,12 @@ import org.apache.kafka.common.message.DescribeDelegationTokenResponseData.DescribedDelegationToken; import org.apache.kafka.common.message.DescribeDelegationTokenResponseData.DescribedDelegationTokenRenewer; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.token.delegation.DelegationToken; import org.apache.kafka.common.security.token.delegation.TokenInformation; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -76,9 +75,9 @@ public DescribeDelegationTokenResponse(DescribeDelegationTokenResponseData data) this.data = data; } - public static DescribeDelegationTokenResponse parse(ByteBuffer buffer, short version) { + public static DescribeDelegationTokenResponse parse(Readable readable, short version) { return new DescribeDelegationTokenResponse(new DescribeDelegationTokenResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java index 2171493b053ab..4d59aee8758ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeGroupsResponse.java @@ -20,11 +20,10 @@ import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup; import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroupMember; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.utils.Utils; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.EnumMap; import java.util.List; @@ -145,8 +144,8 @@ public Map errorCounts() { return errorCounts; } - public static DescribeGroupsResponse parse(ByteBuffer buffer, short version) { - return new DescribeGroupsResponse(new DescribeGroupsResponseData(new ByteBufferAccessor(buffer), version)); + public static DescribeGroupsResponse parse(Readable readable, short version) { + return new DescribeGroupsResponse(new DescribeGroupsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java index bbfc54d294d15..37929c0b3d8fd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeLogDirsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.DescribeLogDirsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -64,8 +63,8 @@ public Map errorCounts() { return errorCounts; } - public static DescribeLogDirsResponse parse(ByteBuffer buffer, short version) { - return new DescribeLogDirsResponse(new DescribeLogDirsResponseData(new ByteBufferAccessor(buffer), version)); + public static DescribeLogDirsResponse parse(Readable readable, short version) { + return new DescribeLogDirsResponse(new DescribeLogDirsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java index 065a101bed6e8..1b30862c4dc5b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeProducersResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.message.DescribeProducersResponseData.PartitionResponse; import org.apache.kafka.common.message.DescribeProducersResponseData.TopicResponse; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.HashMap; import java.util.Map; @@ -51,9 +50,9 @@ public Map errorCounts() { return errorCounts; } - public static DescribeProducersResponse parse(ByteBuffer buffer, short version) { + public static DescribeProducersResponse parse(Readable readable, short version) { return new DescribeProducersResponse(new DescribeProducersResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java index 5ce367bfed29e..c3b33d48052cd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeQuorumResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.DescribeQuorumResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.EnumMap; import java.util.Map; @@ -106,7 +105,7 @@ public static DescribeQuorumResponseData singletonResponse( return res; } - public static DescribeQuorumResponse parse(ByteBuffer buffer, short version) { - return new DescribeQuorumResponse(new DescribeQuorumResponseData(new ByteBufferAccessor(buffer), version)); + public static DescribeQuorumResponse parse(Readable readable, short version) { + return new DescribeQuorumResponse(new DescribeQuorumResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeShareGroupOffsetsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeShareGroupOffsetsRequest.java index 8dfe418162f60..bc859511a7517 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeShareGroupOffsetsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeShareGroupOffsetsRequest.java @@ -33,11 +33,7 @@ public static class Builder extends AbstractRequest.Builder errorCounts() { return errorCounts; } - public static DescribeTransactionsResponse parse(ByteBuffer buffer, short version) { + public static DescribeTransactionsResponse parse(Readable readable, short version) { return new DescribeTransactionsResponse(new DescribeTransactionsResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java index 58ba4212949c6..fad733fd44f5b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/DescribeUserScramCredentialsResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.DescribeUserScramCredentialsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; public class DescribeUserScramCredentialsResponse extends AbstractResponse { @@ -58,7 +57,7 @@ public Map errorCounts() { return errorCounts(data.results().stream().map(r -> Errors.forCode(r.errorCode()))); } - public static DescribeUserScramCredentialsResponse parse(ByteBuffer buffer, short version) { - return new DescribeUserScramCredentialsResponse(new DescribeUserScramCredentialsResponseData(new ByteBufferAccessor(buffer), version)); + public static DescribeUserScramCredentialsResponse parse(Readable readable, short version) { + return new DescribeUserScramCredentialsResponse(new DescribeUserScramCredentialsResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java index d6c7466472ce0..1a1546980a8cf 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ElectLeadersResponse.java @@ -21,10 +21,9 @@ import org.apache.kafka.common.message.ElectLeadersResponseData; import org.apache.kafka.common.message.ElectLeadersResponseData.ReplicaElectionResult; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.HashMap; import java.util.List; @@ -80,8 +79,8 @@ public Map errorCounts() { return counts; } - public static ElectLeadersResponse parse(ByteBuffer buffer, short version) { - return new ElectLeadersResponse(new ElectLeadersResponseData(new ByteBufferAccessor(buffer), version)); + public static ElectLeadersResponse parse(Readable readable, short version) { + return new ElectLeadersResponse(new ElectLeadersResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java index 7750ada7969db..fe39d80087e2d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndQuorumEpochResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -76,7 +75,7 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { // Not supported by the response schema } - public static EndQuorumEpochResponse parse(ByteBuffer buffer, short version) { - return new EndQuorumEpochResponse(new EndQuorumEpochResponseData(new ByteBufferAccessor(buffer), version)); + public static EndQuorumEpochResponse parse(Readable readable, short version) { + return new EndQuorumEpochResponse(new EndQuorumEpochResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java index 0ab01bb1a3d33..21f111e93956e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.EndTxnResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; /** @@ -69,8 +68,8 @@ public EndTxnResponseData data() { return data; } - public static EndTxnResponse parse(ByteBuffer buffer, short version) { - return new EndTxnResponse(new EndTxnResponseData(new ByteBufferAccessor(buffer), version)); + public static EndTxnResponse parse(Readable readable, short version) { + return new EndTxnResponse(new EndTxnResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java index 4f534b6721f4e..9faccc417d546 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EnvelopeResponse.java @@ -18,8 +18,8 @@ import org.apache.kafka.common.message.EnvelopeResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import java.nio.ByteBuffer; import java.util.Map; @@ -72,8 +72,8 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { // Not supported by the response schema } - public static EnvelopeResponse parse(ByteBuffer buffer, short version) { - return new EnvelopeResponse(new EnvelopeResponseData(new ByteBufferAccessor(buffer), version)); + public static EnvelopeResponse parse(Readable readable, short version) { + return new EnvelopeResponse(new EnvelopeResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java index 945db7acdc09e..9fe141565c239 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ExpireDelegationTokenResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.ExpireDelegationTokenResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; public class ExpireDelegationTokenResponse extends AbstractResponse { @@ -33,8 +32,8 @@ public ExpireDelegationTokenResponse(ExpireDelegationTokenResponseData data) { this.data = data; } - public static ExpireDelegationTokenResponse parse(ByteBuffer buffer, short version) { - return new ExpireDelegationTokenResponse(new ExpireDelegationTokenResponseData(new ByteBufferAccessor(buffer), + public static ExpireDelegationTokenResponse parse(Readable readable, short version) { + return new ExpireDelegationTokenResponse(new ExpireDelegationTokenResponseData(readable, version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index d4684e07652ed..0324f591ef4c8 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -22,13 +22,12 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ObjectSerializationCache; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; @@ -145,8 +144,8 @@ public Map errorCounts() { * *

This method should only be used in client-side.

*/ - public static FetchResponse parse(ByteBuffer buffer, short version) { - return new FetchResponse(new FetchResponseData(new ByteBufferAccessor(buffer), version)); + public static FetchResponse parse(Readable readable, short version) { + return new FetchResponse(new FetchResponseData(readable, version)); } // Fetch versions 13 and above should have topic IDs for all topics. diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java index ac7ecd313bf30..77a1089abb477 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchSnapshotResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; import java.util.Optional; @@ -99,7 +98,7 @@ public static Optional forTopicPart .findAny(); } - public static FetchSnapshotResponse parse(ByteBuffer buffer, short version) { - return new FetchSnapshotResponse(new FetchSnapshotResponseData(new ByteBufferAccessor(buffer), version)); + public static FetchSnapshotResponse parse(Readable readable, short version) { + return new FetchSnapshotResponse(new FetchSnapshotResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java index 118e15abaf3a9..5bd08934b62c7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FindCoordinatorResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.message.FindCoordinatorResponseData; import org.apache.kafka.common.message.FindCoordinatorResponseData.Coordinator; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; @@ -108,8 +107,8 @@ public Map errorCounts() { } } - public static FindCoordinatorResponse parse(ByteBuffer buffer, short version) { - return new FindCoordinatorResponse(new FindCoordinatorResponseData(new ByteBufferAccessor(buffer), version)); + public static FindCoordinatorResponse parse(Readable readable, short version) { + return new FindCoordinatorResponse(new FindCoordinatorResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GetTelemetrySubscriptionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/GetTelemetrySubscriptionsResponse.java index dd7bea170a8de..fa8cdf28a91df 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/GetTelemetrySubscriptionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/GetTelemetrySubscriptionsResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.GetTelemetrySubscriptionsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -65,8 +64,8 @@ public Errors error() { return Errors.forCode(data.errorCode()); } - public static GetTelemetrySubscriptionsResponse parse(ByteBuffer buffer, short version) { + public static GetTelemetrySubscriptionsResponse parse(Readable readable, short version) { return new GetTelemetrySubscriptionsResponse(new GetTelemetrySubscriptionsResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java index aebb903e967e7..cc9d81fefa120 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.HeartbeatResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; public class HeartbeatResponse extends AbstractResponse { @@ -67,8 +66,8 @@ public HeartbeatResponseData data() { return data; } - public static HeartbeatResponse parse(ByteBuffer buffer, short version) { - return new HeartbeatResponse(new HeartbeatResponseData(new ByteBufferAccessor(buffer), version)); + public static HeartbeatResponse parse(Readable readable, short version) { + return new HeartbeatResponse(new HeartbeatResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java index 2f660f758d77f..2f487ac420d46 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/IncrementalAlterConfigsResponse.java @@ -21,10 +21,9 @@ import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData; import org.apache.kafka.common.message.IncrementalAlterConfigsResponseData.AlterConfigsResourceResponse; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.EnumMap; import java.util.HashMap; @@ -96,8 +95,8 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static IncrementalAlterConfigsResponse parse(ByteBuffer buffer, short version) { + public static IncrementalAlterConfigsResponse parse(Readable readable, short version) { return new IncrementalAlterConfigsResponse(new IncrementalAlterConfigsResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java index 96c7a4d400ced..6e864b9ec4292 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.InitProducerIdResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; /** @@ -63,8 +62,8 @@ public InitProducerIdResponseData data() { return data; } - public static InitProducerIdResponse parse(ByteBuffer buffer, short version) { - return new InitProducerIdResponse(new InitProducerIdResponseData(new ByteBufferAccessor(buffer), version)); + public static InitProducerIdResponse parse(Readable readable, short version) { + return new InitProducerIdResponse(new InitProducerIdResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitializeShareGroupStateRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/InitializeShareGroupStateRequest.java index 780a66c368103..2feaf55d8142b 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/InitializeShareGroupStateRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/InitializeShareGroupStateRequest.java @@ -33,11 +33,7 @@ public static class Builder extends AbstractRequest.Builder errorCounts() { return errorCounts(Errors.forCode(data.errorCode())); } - public static JoinGroupResponse parse(ByteBuffer buffer, short version) { - return new JoinGroupResponse(new JoinGroupResponseData(new ByteBufferAccessor(buffer), version), version); + public static JoinGroupResponse parse(Readable readable, short version) { + return new JoinGroupResponse(new JoinGroupResponseData(readable, version), version); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java index d8ee6b0336412..7cae507d4c2a7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaveGroupResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.message.LeaveGroupResponseData; import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.List; import java.util.Map; @@ -149,8 +148,8 @@ public LeaveGroupResponseData data() { return data; } - public static LeaveGroupResponse parse(ByteBuffer buffer, short version) { - return new LeaveGroupResponse(new LeaveGroupResponseData(new ByteBufferAccessor(buffer), version)); + public static LeaveGroupResponse parse(Readable readable, short version) { + return new LeaveGroupResponse(new LeaveGroupResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListClientMetricsResourcesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListClientMetricsResourcesResponse.java index 87b25a0a90444..c2b3b1601ed89 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListClientMetricsResourcesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListClientMetricsResourcesResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.clients.admin.ClientMetricsResourceListing; import org.apache.kafka.common.message.ListClientMetricsResourcesResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collection; import java.util.Map; import java.util.stream.Collectors; @@ -48,9 +47,9 @@ public Map errorCounts() { return errorCounts(Errors.forCode(data.errorCode())); } - public static ListClientMetricsResourcesResponse parse(ByteBuffer buffer, short version) { + public static ListClientMetricsResourcesResponse parse(Readable readable, short version) { return new ListClientMetricsResourcesResponse(new ListClientMetricsResourcesResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java index a12f85341d6a4..fa40a4f751067 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListGroupsResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.ListGroupsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; public class ListGroupsResponse extends AbstractResponse { @@ -53,8 +52,8 @@ public Map errorCounts() { return errorCounts(Errors.forCode(data.errorCode())); } - public static ListGroupsResponse parse(ByteBuffer buffer, short version) { - return new ListGroupsResponse(new ListGroupsResponseData(new ByteBufferAccessor(buffer), version)); + public static ListGroupsResponse parse(Readable readable, short version) { + return new ListGroupsResponse(new ListGroupsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java index 003daf55df607..cadff02033958 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetsResponse.java @@ -21,11 +21,10 @@ import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsPartitionResponse; import org.apache.kafka.common.message.ListOffsetsResponseData.ListOffsetsTopicResponse; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.record.RecordBatch; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.EnumMap; import java.util.List; @@ -89,8 +88,8 @@ public Map errorCounts() { return errorCounts; } - public static ListOffsetsResponse parse(ByteBuffer buffer, short version) { - return new ListOffsetsResponse(new ListOffsetsResponseData(new ByteBufferAccessor(buffer), version)); + public static ListOffsetsResponse parse(Readable readable, short version) { + return new ListOffsetsResponse(new ListOffsetsResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java index cbf06d4c46624..cee49055598a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListPartitionReassignmentsResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.ListPartitionReassignmentsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; public class ListPartitionReassignmentsResponse extends AbstractResponse { @@ -33,9 +32,9 @@ public ListPartitionReassignmentsResponse(ListPartitionReassignmentsResponseData this.data = responseData; } - public static ListPartitionReassignmentsResponse parse(ByteBuffer buffer, short version) { + public static ListPartitionReassignmentsResponse parse(Readable readable, short version) { return new ListPartitionReassignmentsResponse(new ListPartitionReassignmentsResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java index 6bebdb2cd0ec3..e563b111984b1 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListTransactionsResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.ListTransactionsResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -44,9 +43,9 @@ public Map errorCounts() { return errorCounts; } - public static ListTransactionsResponse parse(ByteBuffer buffer, short version) { + public static ListTransactionsResponse parse(Readable readable, short version) { return new ListTransactionsResponse(new ListTransactionsResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index ade38d74d3e62..bbcebdd484618 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -26,10 +26,9 @@ import org.apache.kafka.common.message.MetadataResponseData.MetadataResponsePartition; import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -281,8 +280,8 @@ private static boolean hasReliableLeaderEpochs(short version) { return version >= 9; } - public static MetadataResponse parse(ByteBuffer buffer, short version) { - return new MetadataResponse(new MetadataResponseData(new ByteBufferAccessor(buffer), version), + public static MetadataResponse parse(Readable readable, short version) { + return new MetadataResponse(new MetadataResponseData(readable, version), hasReliableLeaderEpochs(version)); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java index 8f6ab39d1fce4..1bd9c41f66834 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestTopic; @@ -45,20 +46,39 @@ public static class Builder extends AbstractRequest.Builder private final OffsetCommitRequestData data; - public Builder(OffsetCommitRequestData data, boolean enableUnstableLastVersion) { - super(ApiKeys.OFFSET_COMMIT, enableUnstableLastVersion); + private Builder(OffsetCommitRequestData data, short oldestAllowedVersion, short latestAllowedVersion) { + super(ApiKeys.OFFSET_COMMIT, oldestAllowedVersion, latestAllowedVersion); this.data = data; } - public Builder(OffsetCommitRequestData data) { - this(data, false); + public static Builder forTopicIdsOrNames(OffsetCommitRequestData data, boolean enableUnstableLastVersion) { + return new Builder(data, ApiKeys.OFFSET_COMMIT.oldestVersion(), ApiKeys.OFFSET_COMMIT.latestVersion(enableUnstableLastVersion)); + } + + public static Builder forTopicNames(OffsetCommitRequestData data) { + return new Builder(data, ApiKeys.OFFSET_COMMIT.oldestVersion(), (short) 9); } @Override public OffsetCommitRequest build(short version) { if (data.groupInstanceId() != null && version < 7) { - throw new UnsupportedVersionException("The broker offset commit protocol version " + - version + " does not support usage of config group.instance.id."); + throw new UnsupportedVersionException("The broker offset commit api version " + + version + " does not support usage of config group.instance.id."); + } + if (version >= 10) { + data.topics().forEach(topic -> { + if (topic.topicId() == null || topic.topicId().equals(Uuid.ZERO_UUID)) { + throw new UnsupportedVersionException("The broker offset commit api version " + + version + " does require usage of topic ids."); + } + }); + } else { + data.topics().forEach(topic -> { + if (topic.name() == null || topic.name().isEmpty()) { + throw new UnsupportedVersionException("The broker offset commit api version " + + version + " does require usage of topic names."); + } + }); } return new OffsetCommitRequest(data, version); } @@ -97,6 +117,7 @@ public static OffsetCommitResponseData getErrorResponse( OffsetCommitResponseData response = new OffsetCommitResponseData(); request.topics().forEach(topic -> { OffsetCommitResponseTopic responseTopic = new OffsetCommitResponseTopic() + .setTopicId(topic.topicId()) .setName(topic.name()); response.topics().add(responseTopic); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 2b6d00b1a47f6..521ffa1c2fdc3 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -17,14 +17,14 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.OffsetCommitResponseData; import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponsePartition; import org.apache.kafka.common.message.OffsetCommitResponseData.OffsetCommitResponseTopic; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -99,8 +99,8 @@ public Map errorCounts() { Errors.forCode(partitionResult.errorCode())))); } - public static OffsetCommitResponse parse(ByteBuffer buffer, short version) { - return new OffsetCommitResponse(new OffsetCommitResponseData(new ByteBufferAccessor(buffer), version)); + public static OffsetCommitResponse parse(Readable readable, short version) { + return new OffsetCommitResponse(new OffsetCommitResponseData(readable, version)); } @Override @@ -123,43 +123,56 @@ public boolean shouldClientThrottle(short version) { return version >= 4; } - public static class Builder { - OffsetCommitResponseData data = new OffsetCommitResponseData(); - HashMap byTopicName = new HashMap<>(); + public static boolean useTopicIds(short version) { + return version >= 10; + } - private OffsetCommitResponseTopic getOrCreateTopic( - String topicName - ) { - OffsetCommitResponseTopic topic = byTopicName.get(topicName); - if (topic == null) { - topic = new OffsetCommitResponseTopic().setName(topicName); - data.topics().add(topic); - byTopicName.put(topicName, topic); - } - return topic; + public static Builder newBuilder(boolean useTopicIds) { + if (useTopicIds) { + return new TopicIdBuilder(); + } else { + return new TopicNameBuilder(); } + } + + public abstract static class Builder { + protected OffsetCommitResponseData data = new OffsetCommitResponseData(); + + protected abstract void add( + OffsetCommitResponseTopic topic + ); + + protected abstract OffsetCommitResponseTopic get( + Uuid topicId, + String topicName + ); + + protected abstract OffsetCommitResponseTopic getOrCreate( + Uuid topicId, + String topicName + ); public Builder addPartition( + Uuid topicId, String topicName, int partitionIndex, Errors error ) { - final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName); - + final OffsetCommitResponseTopic topicResponse = getOrCreate(topicId, topicName); topicResponse.partitions().add(new OffsetCommitResponsePartition() .setPartitionIndex(partitionIndex) .setErrorCode(error.code())); - return this; } public

Builder addPartitions( + Uuid topicId, String topicName, List

partitions, Function partitionIndex, Errors error ) { - final OffsetCommitResponseTopic topicResponse = getOrCreateTopic(topicName); + final OffsetCommitResponseTopic topicResponse = getOrCreate(topicId, topicName); partitions.forEach(partition -> topicResponse.partitions().add(new OffsetCommitResponsePartition() .setPartitionIndex(partitionIndex.apply(partition)) @@ -177,11 +190,10 @@ public Builder merge( } else { // Otherwise, we have to merge them together. newData.topics().forEach(newTopic -> { - OffsetCommitResponseTopic existingTopic = byTopicName.get(newTopic.name()); + OffsetCommitResponseTopic existingTopic = get(newTopic.topicId(), newTopic.name()); if (existingTopic == null) { // If no topic exists, we can directly copy the new topic data. - data.topics().add(newTopic); - byTopicName.put(newTopic.name(), newTopic); + add(newTopic); } else { // Otherwise, we add the partitions to the existing one. Note we // expect non-overlapping partitions here as we don't verify @@ -190,7 +202,6 @@ public Builder merge( } }); } - return this; } @@ -198,4 +209,78 @@ public OffsetCommitResponse build() { return new OffsetCommitResponse(data); } } + + public static class TopicIdBuilder extends Builder { + private final HashMap byTopicId = new HashMap<>(); + + @Override + protected void add(OffsetCommitResponseTopic topic) { + throwIfTopicIdIsNull(topic.topicId()); + data.topics().add(topic); + byTopicId.put(topic.topicId(), topic); + } + + @Override + protected OffsetCommitResponseTopic get(Uuid topicId, String topicName) { + throwIfTopicIdIsNull(topicId); + return byTopicId.get(topicId); + } + + @Override + protected OffsetCommitResponseTopic getOrCreate(Uuid topicId, String topicName) { + throwIfTopicIdIsNull(topicId); + OffsetCommitResponseTopic topic = byTopicId.get(topicId); + if (topic == null) { + topic = new OffsetCommitResponseTopic() + .setName(topicName) + .setTopicId(topicId); + data.topics().add(topic); + byTopicId.put(topicId, topic); + } + return topic; + } + + private static void throwIfTopicIdIsNull(Uuid topicId) { + if (topicId == null) { + throw new IllegalArgumentException("TopicId cannot be null."); + } + } + } + + public static class TopicNameBuilder extends Builder { + private final HashMap byTopicName = new HashMap<>(); + + @Override + protected void add(OffsetCommitResponseTopic topic) { + throwIfTopicNameIsNull(topic.name()); + data.topics().add(topic); + byTopicName.put(topic.name(), topic); + } + + @Override + protected OffsetCommitResponseTopic get(Uuid topicId, String topicName) { + throwIfTopicNameIsNull(topicName); + return byTopicName.get(topicName); + } + + @Override + protected OffsetCommitResponseTopic getOrCreate(Uuid topicId, String topicName) { + throwIfTopicNameIsNull(topicName); + OffsetCommitResponseTopic topic = byTopicName.get(topicName); + if (topic == null) { + topic = new OffsetCommitResponseTopic() + .setName(topicName) + .setTopicId(topicId); + data.topics().add(topic); + byTopicName.put(topicName, topic); + } + return topic; + } + + private void throwIfTopicNameIsNull(String topicName) { + if (topicName == null) { + throw new IllegalArgumentException("TopicName cannot be null."); + } + } + } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java index a419a096e7a4b..0f3655d62c67d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetDeleteResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponsePartition; import org.apache.kafka.common.message.OffsetDeleteResponseData.OffsetDeleteResponseTopic; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.List; import java.util.Map; @@ -150,8 +149,8 @@ public Map errorCounts() { return counts; } - public static OffsetDeleteResponse parse(ByteBuffer buffer, short version) { - return new OffsetDeleteResponse(new OffsetDeleteResponseData(new ByteBufferAccessor(buffer), version)); + public static OffsetDeleteResponse parse(Readable readable, short version) { + return new OffsetDeleteResponse(new OffsetDeleteResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index d2f9d7159b1f1..7b60ba59c5873 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -25,10 +25,9 @@ import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopic; import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopics; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.EnumMap; @@ -391,8 +390,8 @@ public Map partitionDataMap(String groupId) { return buildResponseData(groupId); } - public static OffsetFetchResponse parse(ByteBuffer buffer, short version) { - return new OffsetFetchResponse(new OffsetFetchResponseData(new ByteBufferAccessor(buffer), version), version); + public static OffsetFetchResponse parse(Readable readable, short version) { + return new OffsetFetchResponse(new OffsetFetchResponseData(readable, version), version); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java index a3f4323698d33..ff879667b2655 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -73,8 +72,8 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static OffsetsForLeaderEpochResponse parse(ByteBuffer buffer, short version) { - return new OffsetsForLeaderEpochResponse(new OffsetForLeaderEpochResponseData(new ByteBufferAccessor(buffer), version)); + public static OffsetsForLeaderEpochResponse parse(Readable readable, short version) { + return new OffsetsForLeaderEpochResponse(new OffsetForLeaderEpochResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 573ec9e434d2f..33eee9bb6e905 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -21,11 +21,10 @@ import org.apache.kafka.common.message.ProduceResponseData; import org.apache.kafka.common.message.ProduceResponseData.LeaderIdAndEpoch; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.record.RecordBatch; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.EnumMap; import java.util.List; @@ -286,8 +285,8 @@ public String toString() { } } - public static ProduceResponse parse(ByteBuffer buffer, short version) { - return new ProduceResponse(new ProduceResponseData(new ByteBufferAccessor(buffer), version)); + public static ProduceResponse parse(Readable readable, short version) { + return new ProduceResponse(new ProduceResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/PushTelemetryResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/PushTelemetryResponse.java index e224eed7aebd5..dcd12b42bb68c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/PushTelemetryResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/PushTelemetryResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.PushTelemetryResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -65,8 +64,8 @@ public Errors error() { return Errors.forCode(data.errorCode()); } - public static PushTelemetryResponse parse(ByteBuffer buffer, short version) { + public static PushTelemetryResponse parse(Readable readable, short version) { return new PushTelemetryResponse(new PushTelemetryResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateRequest.java index e3cc84884e64a..920f189ce7980 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ReadShareGroupStateRequest.java @@ -33,11 +33,7 @@ public static class Builder extends AbstractRequest.Builder errorCounts() { } } - public static RemoveRaftVoterResponse parse(ByteBuffer buffer, short version) { + public static RemoveRaftVoterResponse parse(Readable readable, short version) { return new RemoveRaftVoterResponse( - new RemoveRaftVoterResponseData(new ByteBufferAccessor(buffer), version)); + new RemoveRaftVoterResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java index 2baf992283e90..6eb0dc0e0ed53 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/RenewDelegationTokenResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.RenewDelegationTokenResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; public class RenewDelegationTokenResponse extends AbstractResponse { @@ -33,9 +32,9 @@ public RenewDelegationTokenResponse(RenewDelegationTokenResponseData data) { this.data = data; } - public static RenewDelegationTokenResponse parse(ByteBuffer buffer, short version) { + public static RenewDelegationTokenResponse parse(Readable readable, short version) { return new RenewDelegationTokenResponse(new RenewDelegationTokenResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java index d6ca8c170dc45..ba0fc29a39180 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslAuthenticateResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.SaslAuthenticateResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; /** @@ -77,7 +76,7 @@ public SaslAuthenticateResponseData data() { return data; } - public static SaslAuthenticateResponse parse(ByteBuffer buffer, short version) { - return new SaslAuthenticateResponse(new SaslAuthenticateResponseData(new ByteBufferAccessor(buffer), version)); + public static SaslAuthenticateResponse parse(Readable readable, short version) { + return new SaslAuthenticateResponse(new SaslAuthenticateResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java index 5097711e73787..40de2ceff30dd 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.SaslHandshakeResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.List; import java.util.Map; @@ -71,7 +70,7 @@ public List enabledMechanisms() { return data.mechanisms(); } - public static SaslHandshakeResponse parse(ByteBuffer buffer, short version) { - return new SaslHandshakeResponse(new SaslHandshakeResponseData(new ByteBufferAccessor(buffer), version)); + public static SaslHandshakeResponse parse(Readable readable, short version) { + return new SaslHandshakeResponse(new SaslHandshakeResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java index c6be8213c0cb7..a286058496757 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeRequest.java @@ -36,11 +36,7 @@ public static class Builder extends AbstractRequest.Builder ackTopic.partitions().add(ackPartition)); }); - return new ShareAcknowledgeRequest.Builder(data, true); + return new ShareAcknowledgeRequest.Builder(data); } public ShareAcknowledgeRequestData data() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java index 994e063aa1381..33e4cbe1a0138 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareAcknowledgeResponse.java @@ -21,10 +21,9 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ShareAcknowledgeResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.EnumMap; import java.util.Iterator; @@ -84,9 +83,9 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static ShareAcknowledgeResponse parse(ByteBuffer buffer, short version) { + public static ShareAcknowledgeResponse parse(Readable readable, short version) { return new ShareAcknowledgeResponse( - new ShareAcknowledgeResponseData(new ByteBufferAccessor(buffer), version) + new ShareAcknowledgeResponseData(readable, version) ); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java index ea8d93f2a91f3..987400a7ce0df 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchRequest.java @@ -37,11 +37,7 @@ public static class Builder extends AbstractRequest.Builder { private final ShareFetchRequestData data; public Builder(ShareFetchRequestData data) { - this(data, false); - } - - public Builder(ShareFetchRequestData data, boolean enableUnstableLastVersion) { - super(ApiKeys.SHARE_FETCH, enableUnstableLastVersion); + super(ApiKeys.SHARE_FETCH); this.data = data; } @@ -104,7 +100,7 @@ public static Builder forConsumer(String groupId, ShareRequestMetadata metadata, }); } - Builder builder = new Builder(data, true); + Builder builder = new Builder(data); // And finally, forget the topic-partitions that are no longer in the session if (!forget.isEmpty()) { data.setForgottenTopicsData(new ArrayList<>()); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java index 2bab79ead9bc0..eaa6b5f16b893 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareFetchResponse.java @@ -22,13 +22,12 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ObjectSerializationCache; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; import java.util.EnumMap; @@ -110,9 +109,9 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { * *

This method should only be used in client-side.

*/ - public static ShareFetchResponse parse(ByteBuffer buffer, short version) { + public static ShareFetchResponse parse(Readable readable, short version) { return new ShareFetchResponse( - new ShareFetchResponseData(new ByteBufferAccessor(buffer), version) + new ShareFetchResponseData(readable, version) ); } @@ -209,7 +208,7 @@ public static ShareFetchResponseData.PartitionData partitionResponse(TopicIdPart return partitionResponse(topicIdPartition.topicPartition().partition(), error); } - public static ShareFetchResponseData.PartitionData partitionResponse(int partition, Errors error) { + private static ShareFetchResponseData.PartitionData partitionResponse(int partition, Errors error) { return new ShareFetchResponseData.PartitionData() .setPartitionIndex(partition) .setErrorCode(error.code()) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java index 14dd429b8a4f8..1ad411f86010f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ShareGroupDescribeRequest.java @@ -32,11 +32,7 @@ public static class Builder extends AbstractRequest.Builder> assignment + ) { + List topicPartitions = assignment.entrySet().stream() + .map(keyValue -> new ShareGroupHeartbeatResponseData.TopicPartitions() + .setTopicId(keyValue.getKey()) + .setPartitions(new ArrayList<>(keyValue.getValue()))) + .collect(Collectors.toList()); + + return new ShareGroupHeartbeatResponseData.Assignment() + .setTopicPartitions(topicPartitions); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupDescribeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupDescribeResponse.java index cf14373d43750..efee6e521f4fc 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupDescribeResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupDescribeResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.StreamsGroupDescribeResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -70,9 +69,9 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static StreamsGroupDescribeResponse parse(ByteBuffer buffer, short version) { + public static StreamsGroupDescribeResponse parse(Readable readable, short version) { return new StreamsGroupDescribeResponse( - new StreamsGroupDescribeResponseData(new ByteBufferAccessor(buffer), version) + new StreamsGroupDescribeResponseData(readable, version) ); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupHeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupHeartbeatResponse.java index 760d1e33d224c..32fe55f12cdd4 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupHeartbeatResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StreamsGroupHeartbeatResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -72,9 +71,9 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { data.setThrottleTimeMs(throttleTimeMs); } - public static StreamsGroupHeartbeatResponse parse(ByteBuffer buffer, short version) { + public static StreamsGroupHeartbeatResponse parse(Readable readable, short version) { return new StreamsGroupHeartbeatResponse(new StreamsGroupHeartbeatResponseData( - new ByteBufferAccessor(buffer), version)); + readable, version)); } public enum Status { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java index 596110242902c..c31092bdbf78a 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java @@ -18,10 +18,9 @@ import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Map; public class SyncGroupResponse extends AbstractResponse { @@ -62,8 +61,8 @@ public String toString() { return data.toString(); } - public static SyncGroupResponse parse(ByteBuffer buffer, short version) { - return new SyncGroupResponse(new SyncGroupResponseData(new ByteBufferAccessor(buffer), version)); + public static SyncGroupResponse parse(Readable readable, short version) { + return new SyncGroupResponse(new SyncGroupResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java index ce7dd9e7f1cbb..dea99cf2b0709 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java @@ -21,10 +21,9 @@ import org.apache.kafka.common.message.TxnOffsetCommitResponseData.TxnOffsetCommitResponsePartition; import org.apache.kafka.common.message.TxnOffsetCommitResponseData.TxnOffsetCommitResponseTopic; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -191,8 +190,8 @@ public Map errors() { return errorMap; } - public static TxnOffsetCommitResponse parse(ByteBuffer buffer, short version) { - return new TxnOffsetCommitResponse(new TxnOffsetCommitResponseData(new ByteBufferAccessor(buffer), version)); + public static TxnOffsetCommitResponse parse(Readable readable, short version) { + return new TxnOffsetCommitResponse(new TxnOffsetCommitResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java index 6ede317f8f9ab..a0f71a7021f94 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UnregisterBrokerResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.UnregisterBrokerResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -58,8 +57,8 @@ public Map errorCounts() { return errorCounts; } - public static UnregisterBrokerResponse parse(ByteBuffer buffer, short version) { - return new UnregisterBrokerResponse(new UnregisterBrokerResponseData(new ByteBufferAccessor(buffer), version)); + public static UnregisterBrokerResponse parse(Readable readable, short version) { + return new UnregisterBrokerResponse(new UnregisterBrokerResponseData(readable, version)); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java index 14aa2043358a8..895cfd0f85fb6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateFeaturesResponse.java @@ -20,10 +20,9 @@ import org.apache.kafka.common.message.UpdateFeaturesResponseData.UpdatableFeatureResult; import org.apache.kafka.common.message.UpdateFeaturesResponseData.UpdatableFeatureResultCollection; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; import java.util.Set; @@ -79,8 +78,8 @@ public UpdateFeaturesResponseData data() { return data; } - public static UpdateFeaturesResponse parse(ByteBuffer buffer, short version) { - return new UpdateFeaturesResponse(new UpdateFeaturesResponseData(new ByteBufferAccessor(buffer), version)); + public static UpdateFeaturesResponse parse(Readable readable, short version) { + return new UpdateFeaturesResponse(new UpdateFeaturesResponseData(readable, version)); } public static UpdateFeaturesResponse createWithErrors(ApiError topLevelError, Set updates, int throttleTimeMs) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateRaftVoterResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateRaftVoterResponse.java index 5c89caed2ef94..f52157234fa44 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateRaftVoterResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateRaftVoterResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.UpdateRaftVoterResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.Collections; import java.util.Map; @@ -58,8 +57,8 @@ public Map errorCounts() { } } - public static UpdateRaftVoterResponse parse(ByteBuffer buffer, short version) { + public static UpdateRaftVoterResponse parse(Readable readable, short version) { return new UpdateRaftVoterResponse( - new UpdateRaftVoterResponseData(new ByteBufferAccessor(buffer), version)); + new UpdateRaftVoterResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java index cb0d86c867d51..9f3797dc3c017 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/VoteResponse.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; -import java.nio.ByteBuffer; import java.util.EnumMap; import java.util.Map; @@ -76,7 +75,7 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) { // Not supported by the response schema } - public static VoteResponse parse(ByteBuffer buffer, short version) { - return new VoteResponse(new VoteResponseData(new ByteBufferAccessor(buffer), version)); + public static VoteResponse parse(Readable readable, short version) { + return new VoteResponse(new VoteResponseData(readable, version)); } } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteShareGroupStateRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteShareGroupStateRequest.java index 3422eb5d94dee..4d8417c135e56 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/WriteShareGroupStateRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteShareGroupStateRequest.java @@ -33,11 +33,7 @@ public static class Builder extends AbstractRequest.Builder errorCounts() { return errorCounts; } - public static WriteTxnMarkersResponse parse(ByteBuffer buffer, short version) { - return new WriteTxnMarkersResponse(new WriteTxnMarkersResponseData(new ByteBufferAccessor(buffer), version)); + public static WriteTxnMarkersResponse parse(Readable readable, short version) { + return new WriteTxnMarkersResponse(new WriteTxnMarkersResponseData(readable, version)); } } diff --git a/clients/src/main/resources/common/message/AlterShareGroupOffsetsRequest.json b/clients/src/main/resources/common/message/AlterShareGroupOffsetsRequest.json index 357f4083c81e2..0fc7b8f308afa 100644 --- a/clients/src/main/resources/common/message/AlterShareGroupOffsetsRequest.json +++ b/clients/src/main/resources/common/message/AlterShareGroupOffsetsRequest.json @@ -20,7 +20,6 @@ "name": "AlterShareGroupOffsetsRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", "about": "The group identifier." }, diff --git a/clients/src/main/resources/common/message/AlterShareGroupOffsetsResponse.json b/clients/src/main/resources/common/message/AlterShareGroupOffsetsResponse.json index dbcd1f9c0bcea..34bc35c72acb6 100644 --- a/clients/src/main/resources/common/message/AlterShareGroupOffsetsResponse.json +++ b/clients/src/main/resources/common/message/AlterShareGroupOffsetsResponse.json @@ -7,6 +7,7 @@ "flexibleVersions": "0+", // Supported errors: // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - TOPIC_AUTHORIZATION_FAILED (version 0+) // - NOT_COORDINATOR (version 0+) // - COORDINATOR_NOT_AVAILABLE (version 0+) // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) diff --git a/clients/src/main/resources/common/message/DeleteShareGroupOffsetsRequest.json b/clients/src/main/resources/common/message/DeleteShareGroupOffsetsRequest.json index b6a8559cdba76..be778894c1e65 100644 --- a/clients/src/main/resources/common/message/DeleteShareGroupOffsetsRequest.json +++ b/clients/src/main/resources/common/message/DeleteShareGroupOffsetsRequest.json @@ -20,7 +20,6 @@ "name": "DeleteShareGroupOffsetsRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", "about": "The group identifier." }, diff --git a/clients/src/main/resources/common/message/DeleteShareGroupStateRequest.json b/clients/src/main/resources/common/message/DeleteShareGroupStateRequest.json index 37672b92f6561..ff5ad99735845 100644 --- a/clients/src/main/resources/common/message/DeleteShareGroupStateRequest.json +++ b/clients/src/main/resources/common/message/DeleteShareGroupStateRequest.json @@ -20,16 +20,15 @@ "name": "DeleteShareGroupStateRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", - "about":"The group identifier." }, + "about": "The group identifier." }, { "name": "Topics", "type": "[]DeleteStateData", "versions": "0+", "about": "The data for the topics.", "fields": [ { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The topic identifier." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", - "about": "The data for the partitions.", "fields": [ + "about": "The data for the partitions.", "fields": [ { "name": "Partition", "type": "int32", "versions": "0+", "about": "The partition index." } ]} diff --git a/clients/src/main/resources/common/message/DescribeShareGroupOffsetsRequest.json b/clients/src/main/resources/common/message/DescribeShareGroupOffsetsRequest.json index e9093296bb669..f87c1fc394ce8 100644 --- a/clients/src/main/resources/common/message/DescribeShareGroupOffsetsRequest.json +++ b/clients/src/main/resources/common/message/DescribeShareGroupOffsetsRequest.json @@ -20,7 +20,6 @@ "name": "DescribeShareGroupOffsetsRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "Groups", "type": "[]DescribeShareGroupOffsetsRequestGroup", "versions": "0+", "about": "The groups to describe offsets for.", "fields": [ diff --git a/clients/src/main/resources/common/message/InitializeShareGroupStateRequest.json b/clients/src/main/resources/common/message/InitializeShareGroupStateRequest.json index 76de19b48fb0d..7512ce6e51335 100644 --- a/clients/src/main/resources/common/message/InitializeShareGroupStateRequest.json +++ b/clients/src/main/resources/common/message/InitializeShareGroupStateRequest.json @@ -20,7 +20,6 @@ "name": "InitializeShareGroupStateRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "about": "The group identifier." }, @@ -29,7 +28,7 @@ { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The topic identifier." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", - "about": "The data for the partitions.", "fields": [ + "about": "The data for the partitions.", "fields": [ { "name": "Partition", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "StateEpoch", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/OffsetCommitRequest.json b/clients/src/main/resources/common/message/OffsetCommitRequest.json index 348ed2b90c5c8..ba3c12f0e2b47 100644 --- a/clients/src/main/resources/common/message/OffsetCommitRequest.json +++ b/clients/src/main/resources/common/message/OffsetCommitRequest.json @@ -36,8 +36,11 @@ // // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The // request is the same as version 8. - "validVersions": "2-9", + // + // Version 10 adds support for topic ids and removes support for topic names (KIP-848). + "validVersions": "2-10", "flexibleVersions": "8+", + "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", "about": "The unique group identifier." }, @@ -52,8 +55,10 @@ "about": "The time period in ms to retain the offset." }, { "name": "Topics", "type": "[]OffsetCommitRequestTopic", "versions": "0+", "about": "The topics to commit offsets for.", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Name", "type": "string", "versions": "0-9", "entityType": "topicName", "ignorable": true, "about": "The topic name." }, + { "name": "TopicId", "type": "uuid", "versions": "10+", "ignorable": true, + "about": "The topic ID." }, { "name": "Partitions", "type": "[]OffsetCommitRequestPartition", "versions": "0+", "about": "Each partition to commit offsets for.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/OffsetCommitResponse.json b/clients/src/main/resources/common/message/OffsetCommitResponse.json index 0cccd64816c47..0228733ce6bb0 100644 --- a/clients/src/main/resources/common/message/OffsetCommitResponse.json +++ b/clients/src/main/resources/common/message/OffsetCommitResponse.json @@ -34,7 +34,9 @@ // Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The response is // the same as version 8 but can return STALE_MEMBER_EPOCH when the new consumer group protocol is used and // GROUP_ID_NOT_FOUND when the group does not exist for both protocols. - "validVersions": "2-9", + // + // Version 10 adds support for topic ids and removes support for topic names (KIP-848). + "validVersions": "2-10", "flexibleVersions": "8+", // Supported errors: // - GROUP_AUTHORIZATION_FAILED (version 0+) @@ -47,13 +49,16 @@ // - FENCED_MEMBER_EPOCH (version 7+) // - GROUP_ID_NOT_FOUND (version 9+) // - STALE_MEMBER_EPOCH (version 9+) + // - UNKNOWN_TOPIC_ID (version 10+) "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "3+", "ignorable": true, "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, { "name": "Topics", "type": "[]OffsetCommitResponseTopic", "versions": "0+", "about": "The responses for each topic.", "fields": [ - { "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName", + { "name": "Name", "type": "string", "versions": "0-9", "entityType": "topicName", "ignorable": true, "about": "The topic name." }, + { "name": "TopicId", "type": "uuid", "versions": "10+", "ignorable": true, + "about": "The topic ID." }, { "name": "Partitions", "type": "[]OffsetCommitResponsePartition", "versions": "0+", "about": "The responses for each partition in the topic.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/ReadShareGroupStateRequest.json b/clients/src/main/resources/common/message/ReadShareGroupStateRequest.json index d32b99e0f59f0..a19913dfde847 100644 --- a/clients/src/main/resources/common/message/ReadShareGroupStateRequest.json +++ b/clients/src/main/resources/common/message/ReadShareGroupStateRequest.json @@ -20,16 +20,15 @@ "name": "ReadShareGroupStateRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", - "about":"The group identifier." }, + "about": "The group identifier." }, { "name": "Topics", "type": "[]ReadStateData", "versions": "0+", "about": "The data for the topics.", "fields": [ { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The topic identifier." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", - "about": "The data for the partitions.", "fields": [ + "about": "The data for the partitions.", "fields": [ { "name": "Partition", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "LeaderEpoch", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/ReadShareGroupStateResponse.json b/clients/src/main/resources/common/message/ReadShareGroupStateResponse.json index 7815f7b50c7d1..734a0fefdc4d3 100644 --- a/clients/src/main/resources/common/message/ReadShareGroupStateResponse.json +++ b/clients/src/main/resources/common/message/ReadShareGroupStateResponse.json @@ -24,6 +24,7 @@ // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) // - GROUP_ID_NOT_FOUND (version 0+) // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) + // - FENCED_LEADER_EPOCH (version 0+) // - INVALID_REQUEST (version 0+) "fields": [ { "name": "Results", "type": "[]ReadStateResult", "versions": "0+", @@ -39,17 +40,17 @@ { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "about": "The error message, or null if there was no error." }, { "name": "StateEpoch", "type": "int32", "versions": "0+", - "about": "The state epoch for this share-partition." }, + "about": "The state epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", "about": "The share-partition start offset, which can be -1 if it is not yet initialized." }, { "name": "StateBatches", "type": "[]StateBatch", "versions": "0+", "about": "The state batches for this share-partition.", "fields":[ { "name": "FirstOffset", "type": "int64", "versions": "0+", - "about": "The base offset of this state batch." }, + "about": "The first offset of this state batch." }, { "name": "LastOffset", "type": "int64", "versions": "0+", "about": "The last offset of this state batch." }, { "name": "DeliveryState", "type": "int8", "versions": "0+", - "about": "The state - 0:Available,2:Acked,4:Archived." }, + "about": "The delivery state - 0:Available,2:Acked,4:Archived." }, { "name": "DeliveryCount", "type": "int16", "versions": "0+", "about": "The delivery count." } ]} diff --git a/clients/src/main/resources/common/message/ReadShareGroupStateSummaryRequest.json b/clients/src/main/resources/common/message/ReadShareGroupStateSummaryRequest.json index 870f01f3fd494..cdbad63bfa22b 100644 --- a/clients/src/main/resources/common/message/ReadShareGroupStateSummaryRequest.json +++ b/clients/src/main/resources/common/message/ReadShareGroupStateSummaryRequest.json @@ -20,16 +20,15 @@ "name": "ReadShareGroupStateSummaryRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", - "about":"The group identifier." }, + "about": "The group identifier." }, { "name": "Topics", "type": "[]ReadStateSummaryData", "versions": "0+", "about": "The data for the topics.", "fields": [ { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The topic identifier." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", - "about": "The data for the partitions.", "fields": [ + "about": "The data for the partitions.", "fields": [ { "name": "Partition", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "LeaderEpoch", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json b/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json index db534cb4c1c13..7c6f5897cd5dc 100644 --- a/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json +++ b/clients/src/main/resources/common/message/ShareAcknowledgeRequest.json @@ -18,12 +18,11 @@ "type": "request", "listeners": ["broker"], "name": "ShareAcknowledgeRequest", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", - // The ShareAcknowledgeRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", "about": "The group identifier." }, @@ -33,7 +32,7 @@ "about": "The current share session epoch: 0 to open a share session; -1 to close it; otherwise increments for consecutive requests." }, { "name": "Topics", "type": "[]AcknowledgeTopic", "versions": "0+", "about": "The topics containing records to acknowledge.", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The unique topic ID."}, + { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The unique topic ID." }, { "name": "Partitions", "type": "[]AcknowledgePartition", "versions": "0+", "about": "The partitions containing records to acknowledge.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", @@ -41,11 +40,11 @@ { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", "about": "Record batches to acknowledge.", "fields": [ { "name": "FirstOffset", "type": "int64", "versions": "0+", - "about": "First offset of batch of records to acknowledge."}, + "about": "First offset of batch of records to acknowledge." }, { "name": "LastOffset", "type": "int64", "versions": "0+", - "about": "Last offset (inclusive) of batch of records to acknowledge."}, + "about": "Last offset (inclusive) of batch of records to acknowledge." }, { "name": "AcknowledgeTypes", "type": "[]int8", "versions": "0+", - "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject."} + "about": "Array of acknowledge types - 0:Gap,1:Accept,2:Release,3:Reject." } ]} ]} ]} diff --git a/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json b/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json index 1f726a0c7d6a4..d5950243c6b32 100644 --- a/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json +++ b/clients/src/main/resources/common/message/ShareAcknowledgeResponse.json @@ -17,7 +17,10 @@ "apiKey": 79, "type": "response", "name": "ShareAcknowledgeResponse", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", // Supported errors: // - GROUP_AUTHORIZATION_FAILED (version 0+) @@ -32,16 +35,16 @@ // - INVALID_REQUEST (version 0+) // - UNKNOWN_SERVER_ERROR (version 0+) "fields": [ - { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "ignorable": true, + { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, - { "name": "ErrorCode", "type": "int16", "versions": "0+", "ignorable": true, + { "name": "ErrorCode", "type": "int16", "versions": "0+", "about": "The top level response error code." }, { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "about": "The top-level error message, or null if there was no error." }, { "name": "Responses", "type": "[]ShareAcknowledgeTopicResponse", "versions": "0+", "about": "The response topics.", "fields": [ - { "name": "TopicId", "type": "uuid", "versions": "0+", "ignorable": true, - "about": "The unique topic ID."}, + { "name": "TopicId", "type": "uuid", "versions": "0+", + "about": "The unique topic ID." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", "about": "The topic partitions.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", diff --git a/clients/src/main/resources/common/message/ShareFetchRequest.json b/clients/src/main/resources/common/message/ShareFetchRequest.json index 82ac3093db21e..1d898a8232676 100644 --- a/clients/src/main/resources/common/message/ShareFetchRequest.json +++ b/clients/src/main/resources/common/message/ShareFetchRequest.json @@ -18,12 +18,11 @@ "type": "request", "listeners": ["broker"], "name": "ShareFetchRequest", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", - // The ShareFetchRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "groupId", "about": "The group identifier." }, @@ -37,9 +36,9 @@ "about": "The minimum bytes to accumulate in the response." }, { "name": "MaxBytes", "type": "int32", "versions": "0+", "default": "0x7fffffff", "about": "The maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored." }, - { "name": "MaxRecords", "type": "int32", "versions": "0+", + { "name": "MaxRecords", "type": "int32", "versions": "1+", "about": "The maximum number of records to fetch. This limit can be exceeded for alignment of batch boundaries." }, - { "name": "BatchSize", "type": "int32", "versions": "0+", + { "name": "BatchSize", "type": "int32", "versions": "1+", "about": "The optimal number of records for batches of acquired records and acknowledgements." }, { "name": "Topics", "type": "[]FetchTopic", "versions": "0+", "about": "The topics to fetch.", "fields": [ @@ -48,6 +47,8 @@ "about": "The partitions to fetch.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", "about": "The partition index." }, + { "name": "PartitionMaxBytes", "type": "int32", "versions": "0", + "about": "The maximum bytes to fetch from this partition. 0 when only acknowledgement with no fetching is required. See KIP-74 for cases where this limit may not be honored." }, { "name": "AcknowledgementBatches", "type": "[]AcknowledgementBatch", "versions": "0+", "about": "Record batches to acknowledge.", "fields": [ { "name": "FirstOffset", "type": "int64", "versions": "0+", diff --git a/clients/src/main/resources/common/message/ShareFetchResponse.json b/clients/src/main/resources/common/message/ShareFetchResponse.json index ed459f304af13..28cc13ee939bf 100644 --- a/clients/src/main/resources/common/message/ShareFetchResponse.json +++ b/clients/src/main/resources/common/message/ShareFetchResponse.json @@ -17,7 +17,10 @@ "apiKey": 78, "type": "response", "name": "ShareFetchResponse", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", // Supported errors for ErrorCode and AcknowledgeErrorCode: // - GROUP_AUTHORIZATION_FAILED (version 0+) @@ -39,12 +42,12 @@ "about": "The top-level response error code." }, { "name": "ErrorMessage", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "about": "The top-level error message, or null if there was no error." }, - { "name": "AcquisitionLockTimeoutMs", "type": "int32", "versions": "0+", + { "name": "AcquisitionLockTimeoutMs", "type": "int32", "versions": "1+", "about": "The time in milliseconds for which the acquired records are locked." }, { "name": "Responses", "type": "[]ShareFetchableTopicResponse", "versions": "0+", "about": "The response topics.", "fields": [ { "name": "TopicId", "type": "uuid", "versions": "0+", - "about": "The unique topic ID."}, + "about": "The unique topic ID." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", "about": "The topic partitions.", "fields": [ { "name": "PartitionIndex", "type": "int32", "versions": "0+", @@ -64,11 +67,11 @@ { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "about": "The latest known leader epoch." } ]}, - { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0+", "about": "The record data."}, + { "name": "Records", "type": "records", "versions": "0+", "nullableVersions": "0", "about": "The record data." }, { "name": "AcquiredRecords", "type": "[]AcquiredRecords", "versions": "0+", "about": "The acquired records.", "fields": [ - {"name": "FirstOffset", "type": "int64", "versions": "0+", "about": "The earliest offset in this batch of acquired records."}, - {"name": "LastOffset", "type": "int64", "versions": "0+", "about": "The last offset of this batch of acquired records."}, - {"name": "DeliveryCount", "type": "int16", "versions": "0+", "about": "The delivery count of this batch of acquired records."} + { "name": "FirstOffset", "type": "int64", "versions": "0+", "about": "The earliest offset in this batch of acquired records." }, + { "name": "LastOffset", "type": "int64", "versions": "0+", "about": "The last offset of this batch of acquired records." }, + { "name": "DeliveryCount", "type": "int16", "versions": "0+", "about": "The delivery count of this batch of acquired records." } ]} ]} ]}, diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json b/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json index 5efd435939db1..897b8bc7b2a94 100644 --- a/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json +++ b/clients/src/main/resources/common/message/ShareGroupDescribeRequest.json @@ -18,12 +18,11 @@ "type": "request", "listeners": ["broker"], "name": "ShareGroupDescribeRequest", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", - // The ShareGroupDescribeRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, "fields": [ { "name": "GroupIds", "type": "[]string", "versions": "0+", "entityType": "groupId", "about": "The ids of the groups to describe." }, diff --git a/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json b/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json index e90e431f64e43..57595c1b51c9b 100644 --- a/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json +++ b/clients/src/main/resources/common/message/ShareGroupDescribeResponse.json @@ -17,17 +17,20 @@ "apiKey": 77, "type": "response", "name": "ShareGroupDescribeResponse", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", // Supported errors: // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - TOPIC_AUTHORIZATION_FAILED (version 1+) // - NOT_COORDINATOR (version 0+) // - COORDINATOR_NOT_AVAILABLE (version 0+) // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) - // - INVALID_REQUEST (version 0+) // - INVALID_GROUP_ID (version 0+) // - GROUP_ID_NOT_FOUND (version 0+) - // - TOPIC_AUTHORIZATION_FAILED (version 0+) + // - INVALID_REQUEST (version 0+) "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json index e49e022d45864..37d5d04efb2a4 100644 --- a/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json +++ b/clients/src/main/resources/common/message/ShareGroupHeartbeatRequest.json @@ -18,12 +18,11 @@ "type": "request", "listeners": ["broker"], "name": "ShareGroupHeartbeatRequest", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", - // The ShareGroupHeartbeatRequest API is added as part of KIP-932 and is still under - // development. Hence, the API is not exposed by default by brokers unless - // explicitly enabled. - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId", "about": "The group identifier." }, @@ -31,7 +30,7 @@ "about": "The member id generated by the consumer. The member id must be kept during the entire lifetime of the consumer process." }, { "name": "MemberEpoch", "type": "int32", "versions": "0+", "about": "The current member epoch; 0 to join the group; -1 to leave the group." }, - { "name": "RackId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", + { "name": "RackId", "type": "string", "versions": "0+", "nullableVersions": "0+", "default": "null", "about": "null if not provided or if it didn't change since the last heartbeat; the rack ID of consumer otherwise." }, { "name": "SubscribedTopicNames", "type": "[]string", "versions": "0+", "nullableVersions": "0+", "default": "null", "entityType": "topicName", "about": "null if it didn't change since the last heartbeat; the subscribed topic names otherwise." } diff --git a/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json b/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json index 75aa62b76f4fc..c12eb3dca2021 100644 --- a/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json +++ b/clients/src/main/resources/common/message/ShareGroupHeartbeatResponse.json @@ -17,17 +17,20 @@ "apiKey": 76, "type": "response", "name": "ShareGroupHeartbeatResponse", - "validVersions": "0", + // Version 0 was used for early access of KIP-932 in Apache Kafka 4.0 but removed in Apacke Kafka 4.1. + // + // Version 1 is the initial stable version (KIP-932). + "validVersions": "1", "flexibleVersions": "0+", // Supported errors: // - GROUP_AUTHORIZATION_FAILED (version 0+) + // - TOPIC_AUTHORIZATION_FAILED (version 1+) // - NOT_COORDINATOR (version 0+) // - COORDINATOR_NOT_AVAILABLE (version 0+) // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) - // - INVALID_REQUEST (version 0+) // - UNKNOWN_MEMBER_ID (version 0+) // - GROUP_MAX_SIZE_REACHED (version 0+) - // - TOPIC_AUTHORIZATION_FAILED (version 0+) + // - INVALID_REQUEST (version 0+) "fields": [ { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+", "about": "The duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." }, diff --git a/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json b/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json index c0584542739ea..9ebe169c8d6ac 100644 --- a/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json +++ b/clients/src/main/resources/common/message/WriteShareGroupStateRequest.json @@ -20,20 +20,19 @@ "name": "WriteShareGroupStateRequest", "validVersions": "0", "flexibleVersions": "0+", - "latestVersionUnstable": true, "fields": [ { "name": "GroupId", "type": "string", "versions": "0+", - "about":"The group identifier." }, + "about": "The group identifier." }, { "name": "Topics", "type": "[]WriteStateData", "versions": "0+", "about": "The data for the topics.", "fields": [ { "name": "TopicId", "type": "uuid", "versions": "0+", "about": "The topic identifier." }, { "name": "Partitions", "type": "[]PartitionData", "versions": "0+", - "about": "The data for the partitions.", "fields": [ + "about": "The data for the partitions.", "fields": [ { "name": "Partition", "type": "int32", "versions": "0+", "about": "The partition index." }, { "name": "StateEpoch", "type": "int32", "versions": "0+", - "about": "The state epoch for this share-partition." }, + "about": "The state epoch of the share-partition." }, { "name": "LeaderEpoch", "type": "int32", "versions": "0+", "about": "The leader epoch of the share-partition." }, { "name": "StartOffset", "type": "int64", "versions": "0+", @@ -41,11 +40,11 @@ { "name": "StateBatches", "type": "[]StateBatch", "versions": "0+", "about": "The state batches for the share-partition.", "fields": [ { "name": "FirstOffset", "type": "int64", "versions": "0+", - "about": "The base offset of this state batch." }, + "about": "The first offset of this state batch." }, { "name": "LastOffset", "type": "int64", "versions": "0+", "about": "The last offset of this state batch." }, { "name": "DeliveryState", "type": "int8", "versions": "0+", - "about": "The state - 0:Available,2:Acked,4:Archived." }, + "about": "The delivery state - 0:Available,2:Acked,4:Archived." }, { "name": "DeliveryCount", "type": "int16", "versions": "0+", "about": "The delivery count." } ]} diff --git a/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json b/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json index e529126c44b77..8d4050476519c 100644 --- a/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json +++ b/clients/src/main/resources/common/message/WriteShareGroupStateResponse.json @@ -24,6 +24,7 @@ // - COORDINATOR_LOAD_IN_PROGRESS (version 0+) // - GROUP_ID_NOT_FOUND (version 0+) // - UNKNOWN_TOPIC_OR_PARTITION (version 0+) + // - FENCED_LEADER_EPOCH (version 0+) // - FENCED_STATE_EPOCH (version 0+) // - INVALID_REQUEST (version 0+) "fields": [ diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index cb70207e3d97b..f15c46748967b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.requests.RequestTestUtils; @@ -43,7 +44,6 @@ import org.junit.jupiter.api.Test; import java.net.InetSocketAddress; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -209,8 +209,8 @@ public void testIgnoreLeaderEpochInOlderMetadataResponse() { .setBrokers(new MetadataResponseBrokerCollection()); for (short version = ApiKeys.METADATA.oldestVersion(); version < 9; version++) { - ByteBuffer buffer = MessageUtil.toByteBufferAccessor(data, version).buffer(); - MetadataResponse response = MetadataResponse.parse(buffer, version); + Readable readable = MessageUtil.toByteBufferAccessor(data, version); + MetadataResponse response = MetadataResponse.parse(readable, version); assertFalse(response.hasReliableLeaderEpochs()); metadata.updateWithCurrentRequestVersion(response, false, 100); assertTrue(metadata.partitionMetadataIfCurrent(tp).isPresent()); @@ -219,8 +219,8 @@ public void testIgnoreLeaderEpochInOlderMetadataResponse() { } for (short version = 9; version <= ApiKeys.METADATA.latestVersion(); version++) { - ByteBuffer buffer = MessageUtil.toByteBufferAccessor(data, version).buffer(); - MetadataResponse response = MetadataResponse.parse(buffer, version); + Readable readable = MessageUtil.toByteBufferAccessor(data, version); + MetadataResponse response = MetadataResponse.parse(readable, version); assertTrue(response.hasReliableLeaderEpochs()); metadata.updateWithCurrentRequestVersion(response, false, 100); assertTrue(metadata.partitionMetadataIfCurrent(tp).isPresent()); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java index 074fe7ef4a7e6..9a12ce75a6c97 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumerTest.java @@ -117,6 +117,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; @@ -205,6 +206,13 @@ private AsyncKafkaConsumer newConsumerWithoutGroupId() { } private AsyncKafkaConsumer newConsumer(Properties props) { + return newConsumerWithStreamRebalanceData(props, null); + } + + private AsyncKafkaConsumer newConsumerWithStreamRebalanceData( + Properties props, + StreamsRebalanceData streamsRebalanceData + ) { // disable auto-commit by default, so we don't need to handle SyncCommitEvent for each case if (!props.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); @@ -215,12 +223,12 @@ private AsyncKafkaConsumer newConsumer(Properties props) { new StringDeserializer(), new StringDeserializer(), time, - (a, b, c, d, e, f, g, h) -> applicationEventHandler, - a -> backgroundEventReaper, - (a, b, c, d, e, f, g) -> fetchCollector, - (a, b, c, d) -> metadata, + (logContext, time, applicationEventBlockingQueue, completableEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics) -> applicationEventHandler, + logContext -> backgroundEventReaper, + (logContext, consumerMetadata, subscriptionState, fetchConfig, deserializers, fetchMetricsManager, time) -> fetchCollector, + (consumerConfig, subscriptionState, logContext, clusterResourceListeners) -> metadata, backgroundEventQueue, - Optional.empty() + Optional.ofNullable(streamsRebalanceData) ); } @@ -230,10 +238,10 @@ private AsyncKafkaConsumer newConsumer(ConsumerConfig config) { new StringDeserializer(), new StringDeserializer(), time, - (a, b, c, d, e, f, g, h) -> applicationEventHandler, - a -> backgroundEventReaper, - (a, b, c, d, e, f, g) -> fetchCollector, - (a, b, c, d) -> metadata, + (logContext, time, applicationEventBlockingQueue, completableEventReaper, applicationEventProcessorSupplier, networkClientDelegateSupplier, requestManagersSupplier, asyncConsumerMetrics) -> applicationEventHandler, + logContext -> backgroundEventReaper, + (logContext, consumerMetadata, subscriptionState, fetchConfig, deserializers, fetchMetricsManager, time) -> fetchCollector, + (consumerConfig, subscriptionState, logContext, clusterResourceListeners) -> metadata, backgroundEventQueue, Optional.empty() ); @@ -1371,6 +1379,51 @@ public void testGroupMetadataIsResetAfterUnsubscribe() { assertEquals(groupMetadataAfterUnsubscribe, consumer.groupMetadata()); } + private Optional captureStreamRebalanceData(final MockedStatic requestManagers) { + ArgumentCaptor> streamRebalanceData = ArgumentCaptor.forClass(Optional.class); + requestManagers.verify(() -> RequestManagers.supplier( + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + any(), + streamRebalanceData.capture() + )); + return streamRebalanceData.getValue(); + } + + @Test + public void testEmptyStreamRebalanceData() { + final String groupId = "consumerGroupA"; + try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { + consumer = newConsumer(requiredConsumerConfigAndGroupId(groupId)); + final Optional groupMetadataUpdateListener = captureStreamRebalanceData(requestManagers); + assertTrue(groupMetadataUpdateListener.isEmpty()); + } + } + + @Test + public void testStreamRebalanceData() { + final String groupId = "consumerGroupA"; + try (final MockedStatic requestManagers = mockStatic(RequestManagers.class)) { + StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of()); + consumer = newConsumerWithStreamRebalanceData(requiredConsumerConfigAndGroupId(groupId), streamsRebalanceData); + final Optional groupMetadataUpdateListener = captureStreamRebalanceData(requestManagers); + assertTrue(groupMetadataUpdateListener.isPresent()); + assertEquals(streamsRebalanceData, groupMetadataUpdateListener.get()); + } + } + /** * Tests that the consumer correctly invokes the callbacks for {@link ConsumerRebalanceListener} that was * specified. We don't go through the full effort to emulate heartbeats and correct group management here. We're diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 5c9e06ff90d62..a34f2f1633778 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -682,7 +682,7 @@ public void testCoordinatorUnknownInUnsentCallbacksAfterCoordinatorDead() { ) ); - consumerClient.send(coordinator.checkAndGetCoordinator(), new OffsetCommitRequest.Builder(offsetCommitRequestData)) + consumerClient.send(coordinator.checkAndGetCoordinator(), OffsetCommitRequest.Builder.forTopicNames(offsetCommitRequestData)) .compose(new RequestFutureAdapter() { @Override public void onSuccess(ClientResponse value, RequestFuture future) {} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java index 499c7f8f39da5..67628c513406a 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/RequestManagersTest.java @@ -26,10 +26,13 @@ import org.junit.jupiter.api.Test; +import java.util.Map; import java.util.Optional; import java.util.Properties; +import java.util.UUID; import static org.apache.kafka.test.TestUtils.requiredConsumerConfig; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; @@ -65,8 +68,53 @@ public void testMemberStateListenerRegistered() { listener, Optional.empty() ).get(); - requestManagers.consumerMembershipManager.ifPresent( - membershipManager -> assertTrue(membershipManager.stateListeners().contains(listener)) + assertTrue(requestManagers.consumerMembershipManager.isPresent()); + assertTrue(requestManagers.streamsMembershipManager.isEmpty()); + assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isEmpty()); + + assertEquals(2, requestManagers.consumerMembershipManager.get().stateListeners().size()); + assertTrue(requestManagers.consumerMembershipManager.get().stateListeners().stream() + .anyMatch(m -> m instanceof CommitRequestManager)); + assertTrue(requestManagers.consumerMembershipManager.get().stateListeners().contains(listener)); + } + + @Test + public void testStreamMemberStateListenerRegistered() { + + final MemberStateListener listener = (memberEpoch, memberId) -> { }; + + final Properties properties = requiredConsumerConfig(); + properties.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "consumerGroup"); + final ConsumerConfig config = new ConsumerConfig(properties); + final GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( + config, + GroupRebalanceConfig.ProtocolType.CONSUMER ); + final RequestManagers requestManagers = RequestManagers.supplier( + new MockTime(), + new LogContext(), + mock(BackgroundEventHandler.class), + mock(ConsumerMetadata.class), + mock(SubscriptionState.class), + mock(FetchBuffer.class), + config, + groupRebalanceConfig, + mock(ApiVersions.class), + mock(FetchMetricsManager.class), + () -> mock(NetworkClientDelegate.class), + Optional.empty(), + new Metrics(), + mock(OffsetCommitCallbackInvoker.class), + listener, + Optional.of(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())) + ).get(); + assertTrue(requestManagers.streamsMembershipManager.isPresent()); + assertTrue(requestManagers.streamsGroupHeartbeatRequestManager.isPresent()); + assertTrue(requestManagers.consumerMembershipManager.isEmpty()); + + assertEquals(2, requestManagers.streamsMembershipManager.get().stateListeners().size()); + assertTrue(requestManagers.streamsMembershipManager.get().stateListeners().stream() + .anyMatch(m -> m instanceof CommitRequestManager)); + assertTrue(requestManagers.streamsMembershipManager.get().stateListeners().contains(listener)); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java index d43bcfc7891fb..5e0c6652ef63f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java @@ -1520,6 +1520,36 @@ public void testMaximumTimeToWaitSelectingMinimumWaitTime(final long remainingMs } } + @Test + public void testResetPollTimer() { + try (final MockedConstruction pollTimerMockedConstruction = mockConstruction(Timer.class)) { + final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager(); + final Timer pollTimer = pollTimerMockedConstruction.constructed().get(1); + + heartbeatRequestManager.resetPollTimer(time.milliseconds()); + verify(pollTimer).update(time.milliseconds()); + verify(pollTimer).isExpired(); + verify(pollTimer).reset(DEFAULT_MAX_POLL_INTERVAL_MS); + } + } + + @Test + public void testResetPollTimerWhenExpired() { + try (final MockedConstruction pollTimerMockedConstruction = mockConstruction(Timer.class)) { + final StreamsGroupHeartbeatRequestManager heartbeatRequestManager = createStreamsGroupHeartbeatRequestManager(); + final Timer pollTimer = pollTimerMockedConstruction.constructed().get(1); + + when(pollTimer.isExpired()).thenReturn(true); + heartbeatRequestManager.resetPollTimer(time.milliseconds()); + verify(pollTimer).update(time.milliseconds()); + verify(pollTimer).isExpired(); + verify(pollTimer).isExpiredBy(); + verify(membershipManager).memberId(); + verify(membershipManager).maybeRejoinStaleMember(); + verify(pollTimer).reset(DEFAULT_MAX_POLL_INTERVAL_MS); + } + } + private static ConsumerConfig config() { Properties prop = new Properties(); prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java index 3caa298ee164b..a83b0ae2d23df 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceDataTest.java @@ -420,4 +420,21 @@ public void streamsRebalanceDataShouldBeConstructedWithShutDownRequestedSetFalse assertFalse(streamsRebalanceData.shutdownRequested()); } + + @Test + public void streamsRebalanceDataShouldBeConstructedWithEmptyStatuses() { + final UUID processId = UUID.randomUUID(); + final Optional endpoint = Optional.of(new StreamsRebalanceData.HostInfo("localhost", 9090)); + final Map subtopologies = Map.of(); + final Map clientTags = Map.of("clientTag1", "clientTagValue1"); + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData( + processId, + endpoint, + subtopologies, + clientTags + ); + + assertTrue(streamsRebalanceData.statuses().isEmpty()); + } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java index 9cd306a9be157..dde3f567132fc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessorTest.java @@ -31,15 +31,19 @@ import org.apache.kafka.clients.consumer.internals.NetworkClientDelegate; import org.apache.kafka.clients.consumer.internals.OffsetsRequestManager; import org.apache.kafka.clients.consumer.internals.RequestManagers; +import org.apache.kafka.clients.consumer.internals.StreamsGroupHeartbeatRequestManager; +import org.apache.kafka.clients.consumer.internals.StreamsMembershipManager; import org.apache.kafka.clients.consumer.internals.SubscriptionState; import org.apache.kafka.clients.consumer.internals.TopicMetadataRequestManager; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.logging.log4j.Level; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -86,6 +90,8 @@ public class ApplicationEventProcessorTest { private final OffsetsRequestManager offsetsRequestManager = mock(OffsetsRequestManager.class); private SubscriptionState subscriptionState = mock(SubscriptionState.class); private final ConsumerMetadata metadata = mock(ConsumerMetadata.class); + private final StreamsGroupHeartbeatRequestManager streamsGroupHeartbeatRequestManager = mock(StreamsGroupHeartbeatRequestManager.class); + private final StreamsMembershipManager streamsMembershipManager = mock(StreamsMembershipManager.class); private ApplicationEventProcessor processor; private void setupProcessor(boolean withGroupId) { @@ -109,6 +115,27 @@ private void setupProcessor(boolean withGroupId) { ); } + private void setupStreamProcessor(boolean withGroupId) { + RequestManagers requestManagers = new RequestManagers( + new LogContext(), + offsetsRequestManager, + mock(TopicMetadataRequestManager.class), + mock(FetchRequestManager.class), + withGroupId ? Optional.of(mock(CoordinatorRequestManager.class)) : Optional.empty(), + withGroupId ? Optional.of(commitRequestManager) : Optional.empty(), + withGroupId ? Optional.of(heartbeatRequestManager) : Optional.empty(), + Optional.empty(), + withGroupId ? Optional.of(streamsGroupHeartbeatRequestManager) : Optional.empty(), + withGroupId ? Optional.of(streamsMembershipManager) : Optional.empty() + ); + processor = new ApplicationEventProcessor( + new LogContext(), + requestManagers, + metadata, + subscriptionState + ); + } + @Test public void testPrepClosingCommitEvents() { setupProcessor(true); @@ -556,6 +583,78 @@ public void testAsyncCommitEventWithException() { assertFutureThrows(IllegalStateException.class, event.future()); } + @Test + public void testStreamsOnTasksRevokedCallbackCompletedEvent() { + setupStreamProcessor(true); + StreamsOnTasksRevokedCallbackCompletedEvent event = + new StreamsOnTasksRevokedCallbackCompletedEvent(new CompletableFuture<>(), Optional.empty()); + processor.process(event); + verify(streamsMembershipManager).onTasksRevokedCallbackCompleted(event); + } + + @Test + public void testStreamsOnTasksRevokedCallbackCompletedEventWithoutStreamsMembershipManager() { + setupStreamProcessor(false); + StreamsOnTasksRevokedCallbackCompletedEvent event = + new StreamsOnTasksRevokedCallbackCompletedEvent(new CompletableFuture<>(), Optional.empty()); + try (final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister()) { + logAppender.setClassLogger(ApplicationEventProcessor.class, Level.WARN); + processor.process(event); + assertTrue(logAppender.getMessages().stream().anyMatch(e -> + e.contains("An internal error occurred; the Streams membership manager was not present, so the notification " + + "of the onTasksRevoked callback execution could not be sent"))); + verify(streamsMembershipManager, never()).onTasksRevokedCallbackCompleted(event); + } + } + + @Test + public void testStreamsOnTasksAssignedCallbackCompletedEvent() { + setupStreamProcessor(true); + StreamsOnTasksAssignedCallbackCompletedEvent event = + new StreamsOnTasksAssignedCallbackCompletedEvent(new CompletableFuture<>(), Optional.empty()); + processor.process(event); + verify(streamsMembershipManager).onTasksAssignedCallbackCompleted(event); + } + + @Test + public void testStreamsOnTasksAssignedCallbackCompletedEventWithoutStreamsMembershipManager() { + setupStreamProcessor(false); + StreamsOnTasksAssignedCallbackCompletedEvent event = + new StreamsOnTasksAssignedCallbackCompletedEvent(new CompletableFuture<>(), Optional.empty()); + try (final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister()) { + logAppender.setClassLogger(ApplicationEventProcessor.class, Level.WARN); + processor.process(event); + assertTrue(logAppender.getMessages().stream().anyMatch(e -> + e.contains("An internal error occurred; the Streams membership manager was not present, so the notification " + + "of the onTasksAssigned callback execution could not be sent"))); + verify(streamsMembershipManager, never()).onTasksAssignedCallbackCompleted(event); + } + } + + @Test + public void testStreamsOnAllTasksLostCallbackCompletedEvent() { + setupStreamProcessor(true); + StreamsOnAllTasksLostCallbackCompletedEvent event = + new StreamsOnAllTasksLostCallbackCompletedEvent(new CompletableFuture<>(), Optional.empty()); + processor.process(event); + verify(streamsMembershipManager).onAllTasksLostCallbackCompleted(event); + } + + @Test + public void testStreamsOnAllTasksLostCallbackCompletedEventWithoutStreamsMembershipManager() { + setupStreamProcessor(false); + StreamsOnAllTasksLostCallbackCompletedEvent event = + new StreamsOnAllTasksLostCallbackCompletedEvent(new CompletableFuture<>(), Optional.empty()); + try (final LogCaptureAppender logAppender = LogCaptureAppender.createAndRegister()) { + logAppender.setClassLogger(ApplicationEventProcessor.class, Level.WARN); + processor.process(event); + assertTrue(logAppender.getMessages().stream().anyMatch(e -> + e.contains("An internal error occurred; the Streams membership manager was not present, so the notification " + + "of the onAllTasksLost callback execution could not be sent"))); + verify(streamsMembershipManager, never()).onAllTasksLostCallbackCompleted(event); + } + } + private List mockCommitResults() { return Collections.singletonList(mock(NetworkClientDelegate.UnsentRequest.class)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index fbb3484a03f7f..d8cfe6578c0ac 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -48,7 +48,9 @@ import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.message.AddOffsetsToTxnResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData; @@ -73,6 +75,7 @@ import org.apache.kafka.common.requests.EndTxnResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; import org.apache.kafka.common.requests.FindCoordinatorResponse; +import org.apache.kafka.common.requests.InitProducerIdRequest; import org.apache.kafka.common.requests.InitProducerIdResponse; import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.MetadataResponse; @@ -85,7 +88,6 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.telemetry.internals.ClientTelemetryReporter; import org.apache.kafka.common.telemetry.internals.ClientTelemetrySender; -import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; @@ -102,6 +104,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.MockedStatic; import org.mockito.Mockito; @@ -1084,13 +1087,14 @@ public void testTopicExpiryInMetadata() throws InterruptedException { @SuppressWarnings("unchecked") @Test - public void testHeaders() { + public void testHeadersSuccess() { doTestHeaders(Serializer.class); } private > void doTestHeaders(Class serializerClassToMock) { Map configs = new HashMap<>(); configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + configs.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, ProducerInterceptorForHeaders.class.getName()); Serializer keySerializer = mock(serializerClassToMock); Serializer valueSerializer = mock(serializerClassToMock); @@ -1119,7 +1123,9 @@ private > void doTestHeaders(Class serializerCla producer.send(record, null); //ensure headers are closed and cannot be mutated post send - assertThrows(IllegalStateException.class, () -> record.headers().add(new RecordHeader("test", "test".getBytes()))); + RecordHeaders recordHeaders = (RecordHeaders) record.headers(); + assertTrue(recordHeaders.isReadOnly()); + assertThrows(IllegalStateException.class, () -> recordHeaders.add(new RecordHeader("test", "test".getBytes()))); //ensure existing headers are not changed, and last header for key is still original value assertArrayEquals(record.headers().lastHeader("test").value(), "header2".getBytes()); @@ -1130,6 +1136,28 @@ private > void doTestHeaders(Class serializerCla producer.close(Duration.ofMillis(0)); } + @Test + public void testHeadersFailure() { + Properties props = new Properties(); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 5); + props.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, ProducerInterceptorForHeaders.class.getName()); + Serializer keySerializer = mock(StringSerializer.class); + Serializer valueSerializer = mock(StringSerializer.class); + + KafkaProducer producer = new KafkaProducer<>(props, keySerializer, valueSerializer); + ProducerRecord record = new ProducerRecord<>("topic", "key", "value"); + Future future = producer.send(record, (recordMetadata, exception) -> { }); + try { + TestUtils.assertFutureThrows(TimeoutException.class, future); + //ensure headers are writable if send failure + RecordHeaders recordHeaders = (RecordHeaders) record.headers(); + assertFalse(recordHeaders.isReadOnly()); + } finally { + producer.close(Duration.ofMillis(0)); + } + } + @Test public void closeShouldBeIdempotent() { Properties producerProps = new Properties(); @@ -1289,7 +1317,7 @@ public void testInitTransactionsResponseAfterTimeout() throws Exception { ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), FindCoordinatorResponse.prepareResponse(Errors.NONE, "bad-transaction", NODE)); - Future future = executor.submit(producer::initTransactions); + Future future = executor.submit(() -> producer.initTransactions()); TestUtils.waitForCondition(client::hasInFlightRequests, "Timed out while waiting for expected `InitProducerId` request to be sent"); @@ -1364,6 +1392,59 @@ public void testInitTransactionWhileThrottled() { } } + @ParameterizedTest + @CsvSource({ + "true, false", + "true, true", + "false, true" + }) + public void testInitTransactionsWithKeepPreparedTxnAndTwoPhaseCommit(boolean keepPreparedTxn, boolean enable2PC) { + Map configs = new HashMap<>(); + configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "test-txn-id"); + configs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, 10000); + configs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + if (enable2PC) { + configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, true); + } + + Time time = new MockTime(1); + MetadataResponse initialUpdateResponse = RequestTestUtils.metadataUpdateWith(1, singletonMap("topic", 1)); + ProducerMetadata metadata = newMetadata(0, 0, Long.MAX_VALUE); + MockClient client = new MockClient(time, metadata); + client.updateMetadata(initialUpdateResponse); + + // Capture flags from the InitProducerIdRequest + boolean[] requestFlags = new boolean[2]; // [keepPreparedTxn, enable2Pc] + + client.prepareResponse( + request -> request instanceof FindCoordinatorRequest && + ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), + FindCoordinatorResponse.prepareResponse(Errors.NONE, "test-txn-id", NODE)); + + client.prepareResponse( + request -> { + if (request instanceof InitProducerIdRequest) { + InitProducerIdRequest initRequest = (InitProducerIdRequest) request; + requestFlags[0] = initRequest.data().keepPreparedTxn(); + requestFlags[1] = initRequest.data().enable2Pc(); + return true; + } + return false; + }, + initProducerIdResponse(1L, (short) 5, Errors.NONE)); + + try (Producer producer = kafkaProducer(configs, new StringSerializer(), + new StringSerializer(), metadata, client, null, time)) { + producer.initTransactions(keepPreparedTxn); + + // Verify request flags match expected values + assertEquals(keepPreparedTxn, requestFlags[0], + "keepPreparedTxn flag should match input parameter"); + assertEquals(enable2PC, requestFlags[1], + "enable2Pc flag should match producer configuration"); + } + } + @Test public void testClusterAuthorizationFailure() throws Exception { int maxBlockMs = 500; @@ -2500,6 +2581,29 @@ public void configure(Map configs) { } } + public static class ProducerInterceptorForHeaders implements ProducerInterceptor { + + @Override + public ProducerRecord onSend(ProducerRecord record) { + return record; + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + RecordHeaders recordHeaders = (RecordHeaders) headers; + // Ensure that the headers are read-only, no matter send success or send failure + assertTrue(recordHeaders.isReadOnly()); + } + + @Override + public void close() { + } + + @Override + public void configure(Map configs) { + } + } + public static class ProducerInterceptorForClientId implements ProducerInterceptor { @Override @@ -2542,7 +2646,7 @@ private static class KafkaProducerTestContext { private final Map configs; private final Serializer serializer; private final Partitioner partitioner = mock(Partitioner.class); - private final KafkaThread ioThread = mock(KafkaThread.class); + private final Sender.SenderThread senderThread = mock(Sender.SenderThread.class); private final List> interceptors = new ArrayList<>(); private ProducerMetadata metadata = mock(ProducerMetadata.class); private RecordAccumulator accumulator = mock(RecordAccumulator.class); @@ -2623,7 +2727,7 @@ public KafkaProducer newKafkaProducer() { interceptors, partitioner, time, - ioThread, + senderThread, Optional.empty() ); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java index 830711c0e5449..207bac6476fc1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java @@ -145,4 +145,27 @@ void testUpperboundCheckOfEnableIdempotence() { configs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "5"); assertDoesNotThrow(() -> new ProducerConfig(configs)); } + + @Test + void testTwoPhaseCommitIncompatibleWithTransactionTimeout() { + Map configs = new HashMap<>(); + configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); + configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); + configs.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, true); + configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "test-txn-id"); + configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, true); + configs.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 60000); + + ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); + assertTrue(ce.getMessage().contains(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG)); + assertTrue(ce.getMessage().contains(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG)); + + // Verify that setting one but not the other is valid + configs.remove(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); + assertDoesNotThrow(() -> new ProducerConfig(configs)); + + configs.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 60000); + configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, false); + assertDoesNotThrow(() -> new ProducerConfig(configs)); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java index 853b27b255114..74f5db740628e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.header.Headers; import org.junit.jupiter.api.Test; @@ -95,13 +96,68 @@ public void injectOnAcknowledgementError(boolean on) { } } + private class AppendNewProducerInterceptor implements ProducerInterceptor { + private final String appendStr; + private boolean throwExceptionOnSend = false; + private boolean throwExceptionOnAck = false; + + public AppendNewProducerInterceptor(String appendStr) { + this.appendStr = appendStr; + } + + @Override + public void configure(Map configs) { + } + + @Override + public ProducerRecord onSend(ProducerRecord record) { + onSendCount++; + if (throwExceptionOnSend) + throw new KafkaException("Injected exception in AppendNewProducerInterceptor.onSend"); + + return new ProducerRecord<>( + record.topic(), record.partition(), record.key(), record.value().concat(appendStr)); + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + onAckCount++; + if (exception != null) { + onErrorAckCount++; + // the length check is just to call topic() method and let it throw an exception + // if RecordMetadata.TopicPartition is null + if (metadata != null && metadata.topic().length() >= 0) { + onErrorAckWithTopicSetCount++; + if (metadata.partition() >= 0) + onErrorAckWithTopicPartitionSetCount++; + } + } + if (throwExceptionOnAck) + throw new KafkaException("Injected exception in AppendNewProducerInterceptor.onAcknowledgement"); + } + + @Override + public void close() { + } + + // if 'on' is true, onSend will always throw an exception + public void injectOnSendError(boolean on) { + throwExceptionOnSend = on; + } + + // if 'on' is true, onAcknowledgement will always throw an exception + public void injectOnAcknowledgementError(boolean on) { + throwExceptionOnAck = on; + } + } + @Test public void testOnSendChain() { List> interceptorList = new ArrayList<>(); // we are testing two different interceptors by configuring the same interceptor differently, which is not // how it would be done in KafkaProducer, but ok for testing interceptor callbacks AppendProducerInterceptor interceptor1 = new AppendProducerInterceptor("One"); - AppendProducerInterceptor interceptor2 = new AppendProducerInterceptor("Two"); + AppendNewProducerInterceptor interceptor2 = new AppendNewProducerInterceptor("Two"); interceptorList.add(interceptor1); interceptorList.add(interceptor2); ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList, null); @@ -139,23 +195,23 @@ public void testOnAcknowledgementChain() { // we are testing two different interceptors by configuring the same interceptor differently, which is not // how it would be done in KafkaProducer, but ok for testing interceptor callbacks AppendProducerInterceptor interceptor1 = new AppendProducerInterceptor("One"); - AppendProducerInterceptor interceptor2 = new AppendProducerInterceptor("Two"); + AppendNewProducerInterceptor interceptor2 = new AppendNewProducerInterceptor("Two"); interceptorList.add(interceptor1); interceptorList.add(interceptor2); ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList, null); // verify onAck is called on all interceptors RecordMetadata meta = new RecordMetadata(tp, 0, 0, 0, 0, 0); - interceptors.onAcknowledgement(meta, null); + interceptors.onAcknowledgement(meta, null, null); assertEquals(2, onAckCount); // verify that onAcknowledgement exceptions do not propagate interceptor1.injectOnAcknowledgementError(true); - interceptors.onAcknowledgement(meta, null); + interceptors.onAcknowledgement(meta, null, null); assertEquals(4, onAckCount); interceptor2.injectOnAcknowledgementError(true); - interceptors.onAcknowledgement(meta, null); + interceptors.onAcknowledgement(meta, null, null); assertEquals(6, onAckCount); interceptors.close(); @@ -165,27 +221,29 @@ public void testOnAcknowledgementChain() { public void testOnAcknowledgementWithErrorChain() { List> interceptorList = new ArrayList<>(); AppendProducerInterceptor interceptor1 = new AppendProducerInterceptor("One"); + AppendNewProducerInterceptor interceptor2 = new AppendNewProducerInterceptor("Two"); interceptorList.add(interceptor1); + interceptorList.add(interceptor2); ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList, null); // verify that metadata contains both topic and partition interceptors.onSendError(producerRecord, new TopicPartition(producerRecord.topic(), producerRecord.partition()), new KafkaException("Test")); - assertEquals(1, onErrorAckCount); - assertEquals(1, onErrorAckWithTopicPartitionSetCount); + assertEquals(2, onErrorAckCount); + assertEquals(2, onErrorAckWithTopicPartitionSetCount); // verify that metadata contains both topic and partition (because record already contains partition) interceptors.onSendError(producerRecord, null, new KafkaException("Test")); - assertEquals(2, onErrorAckCount); - assertEquals(2, onErrorAckWithTopicPartitionSetCount); + assertEquals(4, onErrorAckCount); + assertEquals(4, onErrorAckWithTopicPartitionSetCount); // if producer record does not contain partition, interceptor should get partition == -1 ProducerRecord record2 = new ProducerRecord<>("test2", null, 1, "value"); interceptors.onSendError(record2, null, new KafkaException("Test")); - assertEquals(3, onErrorAckCount); - assertEquals(3, onErrorAckWithTopicSetCount); - assertEquals(2, onErrorAckWithTopicPartitionSetCount); + assertEquals(6, onErrorAckCount); + assertEquals(6, onErrorAckWithTopicSetCount); + assertEquals(4, onErrorAckWithTopicPartitionSetCount); // if producer record does not contain partition, but topic/partition is passed to // onSendError, then interceptor should get valid partition @@ -193,15 +251,15 @@ public void testOnAcknowledgementWithErrorChain() { interceptors.onSendError(record2, new TopicPartition(record2.topic(), reassignedPartition), new KafkaException("Test")); - assertEquals(4, onErrorAckCount); - assertEquals(4, onErrorAckWithTopicSetCount); - assertEquals(3, onErrorAckWithTopicPartitionSetCount); + assertEquals(8, onErrorAckCount); + assertEquals(8, onErrorAckWithTopicSetCount); + assertEquals(6, onErrorAckWithTopicPartitionSetCount); // if both record and topic/partition are null, interceptor should not receive metadata interceptors.onSendError(null, null, new KafkaException("Test")); - assertEquals(5, onErrorAckCount); - assertEquals(4, onErrorAckWithTopicSetCount); - assertEquals(3, onErrorAckWithTopicPartitionSetCount); + assertEquals(10, onErrorAckCount); + assertEquals(8, onErrorAckWithTopicSetCount); + assertEquals(6, onErrorAckWithTopicPartitionSetCount); interceptors.close(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index e67be76eb9baf..74a5642024637 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -490,7 +490,7 @@ public void senderThreadShouldNotGetStuckWhenThrottledAndAddingPartitionsToTxn() ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); - TransactionManager txnManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -616,10 +616,10 @@ public void testInitProducerIdWithMaxInFlightOne() { // Initialize transaction manager. InitProducerId will be queued up until metadata response // is processed and FindCoordinator can be sent to `leastLoadedNode`. TransactionManager transactionManager = new TransactionManager(new LogContext(), "testInitProducerIdWithPendingMetadataRequest", - 60000, 100L, new ApiVersions()); + 60000, 100L, new ApiVersions(), false); setupWithTransactionState(transactionManager, false, null, false); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId, (short) 0); - transactionManager.initializeTransactions(); + transactionManager.initializeTransactions(false); sender.runOnce(); // Process metadata response, prepare FindCoordinator and InitProducerId responses. @@ -668,10 +668,10 @@ public void testNodeNotReady() { client = new MockClient(time, metadata); TransactionManager transactionManager = new TransactionManager(new LogContext(), "testNodeNotReady", - 60000, 100L, new ApiVersions()); + 60000, 100L, new ApiVersions(), false); setupWithTransactionState(transactionManager, false, null, true); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(producerId, (short) 0); - transactionManager.initializeTransactions(); + transactionManager.initializeTransactions(false); sender.runOnce(); Node node = metadata.fetch().nodes().get(0); @@ -1510,7 +1510,7 @@ public void testExpiryOfFirstBatchShouldCauseEpochBumpIfFutureBatchesFail() thro public void testUnresolvedSequencesAreNotFatal() throws Exception { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); - TransactionManager txnManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -1795,7 +1795,7 @@ public void testCorrectHandlingOfDuplicateSequenceError() throws Exception { @Test public void testTransactionalUnknownProducerHandlingWhenRetentionLimitReached() throws Exception { final long producerId = 343434L; - TransactionManager transactionManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions); + TransactionManager transactionManager = new TransactionManager(logContext, "testUnresolvedSeq", 60000, 100, apiVersions, false); setupWithTransactionState(transactionManager); doInitTransactions(transactionManager, new ProducerIdAndEpoch(producerId, (short) 0)); @@ -2352,7 +2352,7 @@ public void testIdempotentSplitBatchAndSend() throws Exception { public void testTransactionalSplitBatchAndSend() throws Exception { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); TopicPartition tp = new TopicPartition("testSplitBatchAndSend", 1); - TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -2694,7 +2694,7 @@ public void testTransactionalRequestsSentOnShutdown() { Metrics m = new Metrics(); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); try { - TransactionManager txnManager = new TransactionManager(logContext, "testTransactionalRequestsSentOnShutdown", 6000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testTransactionalRequestsSentOnShutdown", 6000, 100, apiVersions, false); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, txnManager); @@ -2727,7 +2727,7 @@ public void testRecordsFlushedImmediatelyOnTransactionCompletion() throws Except int lingerMs = 50; SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); - TransactionManager txnManager = new TransactionManager(logContext, "txnId", 6000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "txnId", 6000, 100, apiVersions, false); setupWithTransactionState(txnManager, lingerMs); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, @@ -2784,7 +2784,7 @@ public void testAwaitPendingRecordsBeforeCommittingTransaction() throws Exceptio try (Metrics m = new Metrics()) { SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); - TransactionManager txnManager = new TransactionManager(logContext, "txnId", 6000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "txnId", 6000, 100, apiVersions, false); setupWithTransactionState(txnManager); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, @@ -2855,7 +2855,7 @@ public void testIncompleteTransactionAbortOnShutdown() { Metrics m = new Metrics(); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); try { - TransactionManager txnManager = new TransactionManager(logContext, "testIncompleteTransactionAbortOnShutdown", 6000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testIncompleteTransactionAbortOnShutdown", 6000, 100, apiVersions, false); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, txnManager); @@ -2889,7 +2889,7 @@ public void testForceShutdownWithIncompleteTransaction() { Metrics m = new Metrics(); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); try { - TransactionManager txnManager = new TransactionManager(logContext, "testForceShutdownWithIncompleteTransaction", 6000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testForceShutdownWithIncompleteTransaction", 6000, 100, apiVersions, false); Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, senderMetrics, time, REQUEST_TIMEOUT, RETRY_BACKOFF_MS, txnManager); @@ -2919,7 +2919,7 @@ public void testForceShutdownWithIncompleteTransaction() { @Test public void testTransactionAbortedExceptionOnAbortWithoutError() throws InterruptedException { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); - TransactionManager txnManager = new TransactionManager(logContext, "testTransactionAbortedExceptionOnAbortWithoutError", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testTransactionAbortedExceptionOnAbortWithoutError", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager, false, null); doInitTransactions(txnManager, producerIdAndEpoch); @@ -2945,7 +2945,7 @@ public void testTransactionAbortedExceptionOnAbortWithoutError() throws Interrup public void testDoNotPollWhenNoRequestSent() { client = spy(new MockClient(time, metadata)); - TransactionManager txnManager = new TransactionManager(logContext, "testDoNotPollWhenNoRequestSent", 6000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testDoNotPollWhenNoRequestSent", 6000, 100, apiVersions, false); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -2957,7 +2957,7 @@ public void testDoNotPollWhenNoRequestSent() { @Test public void testTooLargeBatchesAreSafelyRemoved() throws InterruptedException { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); - TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testSplitBatchAndSend", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager, false, null); doInitTransactions(txnManager, producerIdAndEpoch); @@ -3026,7 +3026,7 @@ public void testSenderShouldRetryWithBackoffOnRetriableError() { public void testReceiveFailedBatchTwiceWithTransactions() throws Exception { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); - TransactionManager txnManager = new TransactionManager(logContext, "testFailTwice", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testFailTwice", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -3076,7 +3076,7 @@ public void testReceiveFailedBatchTwiceWithTransactions() throws Exception { public void testInvalidTxnStateIsAnAbortableError() throws Exception { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); - TransactionManager txnManager = new TransactionManager(logContext, "testInvalidTxnState", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "testInvalidTxnState", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -3115,7 +3115,7 @@ public void testInvalidTxnStateIsAnAbortableError() throws Exception { public void testTransactionAbortableExceptionIsAnAbortableError() throws Exception { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(123456L, (short) 0); apiVersions.update("0", NodeApiVersions.create(ApiKeys.INIT_PRODUCER_ID.id, (short) 0, (short) 3)); - TransactionManager txnManager = new TransactionManager(logContext, "textTransactionAbortableException", 60000, 100, apiVersions); + TransactionManager txnManager = new TransactionManager(logContext, "textTransactionAbortableException", 60000, 100, apiVersions, false); setupWithTransactionState(txnManager); doInitTransactions(txnManager, producerIdAndEpoch); @@ -3617,7 +3617,7 @@ private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors e } private TransactionManager createTransactionManager() { - return new TransactionManager(new LogContext(), null, 0, RETRY_BACKOFF_MS, new ApiVersions()); + return new TransactionManager(new LogContext(), null, 0, RETRY_BACKOFF_MS, new ApiVersions(), false); } private void setupWithTransactionState(TransactionManager transactionManager) { @@ -3719,7 +3719,7 @@ private InitProducerIdResponse initProducerIdResponse(long producerId, short pro } private void doInitTransactions(TransactionManager transactionManager, ProducerIdAndEpoch producerIdAndEpoch) { - TransactionalRequestResult result = transactionManager.initializeTransactions(); + TransactionalRequestResult result = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, transactionManager.transactionalId()); sender.runOnce(); sender.runOnce(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java index 8b4decfb9598c..a19e4977314fd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java @@ -149,7 +149,7 @@ public class TransactionManagerTest { private RecordAccumulator accumulator = null; private Sender sender = null; - private TransactionManager transactionManager = null; + private TestableTransactionManager transactionManager = null; private Node brokerNode = null; private long finalizedFeaturesEpoch = 0; @@ -159,17 +159,28 @@ public void setup() { this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, singletonMap("test", 2))); this.brokerNode = new Node(0, "localhost", 2211); - initializeTransactionManager(Optional.of(transactionalId), false); + initializeTransactionManager(Optional.of(transactionalId), false, false); } - private void initializeTransactionManager(Optional transactionalId, boolean transactionV2Enabled) { + private void initializeTransactionManager( + Optional transactionalId, + boolean transactionV2Enabled + ) { + initializeTransactionManager(transactionalId, transactionV2Enabled, false); + } + + private void initializeTransactionManager( + Optional transactionalId, + boolean transactionV2Enabled, + boolean enable2pc + ) { Metrics metrics = new Metrics(time); apiVersions.update("0", new NodeApiVersions(Arrays.asList( new ApiVersion() .setApiKey(ApiKeys.INIT_PRODUCER_ID.id) .setMinVersion((short) 0) - .setMaxVersion((short) 3), + .setMaxVersion((short) 6), new ApiVersion() .setApiKey(ApiKeys.PRODUCE.id) .setMinVersion((short) 0) @@ -188,8 +199,9 @@ private void initializeTransactionManager(Optional transactionalId, bool .setMinVersionLevel(transactionV2Enabled ? (short) 2 : (short) 1)), finalizedFeaturesEpoch)); finalizedFeaturesEpoch += 1; - this.transactionManager = new TransactionManager(logContext, transactionalId.orElse(null), - transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, apiVersions); + this.transactionManager = new TestableTransactionManager(logContext, transactionalId.orElse(null), + transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, apiVersions, enable2pc); + int batchSize = 16 * 1024; int deliveryTimeoutMs = 3000; @@ -1038,8 +1050,8 @@ public void testTransactionManagerDisablesV2() { .setMaxVersionLevel((short) 1) .setMinVersionLevel((short) 1)), 0)); - this.transactionManager = new TransactionManager(logContext, transactionalId, - transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, apiVersions); + this.transactionManager = new TestableTransactionManager(logContext, transactionalId, + transactionTimeoutMs, DEFAULT_RETRY_BACKOFF_MS, apiVersions, false); int batchSize = 16 * 1024; int deliveryTimeoutMs = 3000; @@ -1063,7 +1075,7 @@ MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, new SenderMetricsRegistry(metrics), thi public void testDisconnectAndRetry() { // This is called from the initTransactions method in the producer as the first order of business. // It finds the coordinator and then gets a PID. - transactionManager.initializeTransactions(); + transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, true, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) == null); @@ -1076,12 +1088,12 @@ public void testDisconnectAndRetry() { public void testInitializeTransactionsTwiceRaisesError() { doInitTransactions(producerId, epoch); assertTrue(transactionManager.hasProducerId()); - assertThrows(IllegalStateException.class, () -> transactionManager.initializeTransactions()); + assertThrows(IllegalStateException.class, () -> transactionManager.initializeTransactions(false)); } @Test public void testUnsupportedFindCoordinator() { - transactionManager.initializeTransactions(); + transactionManager.initializeTransactions(false); client.prepareUnsupportedVersionResponse(body -> { FindCoordinatorRequest findCoordinatorRequest = (FindCoordinatorRequest) body; assertEquals(CoordinatorType.forId(findCoordinatorRequest.data().keyType()), CoordinatorType.TRANSACTION); @@ -1098,7 +1110,7 @@ public void testUnsupportedFindCoordinator() { @Test public void testUnsupportedInitTransactions() { - transactionManager.initializeTransactions(); + transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertFalse(transactionManager.hasError()); @@ -1243,7 +1255,7 @@ public void testIllegalGenerationInTxnOffsetCommitByGroupMetadata() { public void testLookupCoordinatorOnDisconnectAfterSend() { // This is called from the initTransactions method in the producer as the first order of business. // It finds the coordinator and then gets a PID. - TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -1275,7 +1287,7 @@ public void testLookupCoordinatorOnDisconnectAfterSend() { public void testLookupCoordinatorOnDisconnectBeforeSend() { // This is called from the initTransactions method in the producer as the first order of business. // It finds the coordinator and then gets a PID. - TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -1306,7 +1318,7 @@ public void testLookupCoordinatorOnDisconnectBeforeSend() { public void testLookupCoordinatorOnNotCoordinatorError() { // This is called from the initTransactions method in the producer as the first order of business. // It finds the coordinator and then gets a PID. - TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -1331,7 +1343,7 @@ public void testLookupCoordinatorOnNotCoordinatorError() { @Test public void testTransactionalIdAuthorizationFailureInFindCoordinator() { - TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, false, CoordinatorType.TRANSACTION, transactionalId); @@ -1346,7 +1358,7 @@ public void testTransactionalIdAuthorizationFailureInFindCoordinator() { @Test public void testTransactionalIdAuthorizationFailureInInitProducerId() { - TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -1646,7 +1658,7 @@ public void testRetryAbortTransactionAfterTimeout() throws Exception { assertFalse(result.isAcked()); assertFalse(transactionManager.hasOngoingTransaction()); - assertThrows(IllegalStateException.class, transactionManager::initializeTransactions); + assertThrows(IllegalStateException.class, () -> transactionManager.initializeTransactions(false)); assertThrows(IllegalStateException.class, transactionManager::beginTransaction); assertThrows(IllegalStateException.class, transactionManager::beginCommit); assertThrows(IllegalStateException.class, () -> transactionManager.maybeAddPartition(tp0)); @@ -1680,7 +1692,7 @@ public void testRetryCommitTransactionAfterTimeout() throws Exception { assertFalse(result.isAcked()); assertFalse(transactionManager.hasOngoingTransaction()); - assertThrows(IllegalStateException.class, transactionManager::initializeTransactions); + assertThrows(IllegalStateException.class, () -> transactionManager.initializeTransactions(false)); assertThrows(IllegalStateException.class, transactionManager::beginTransaction); assertThrows(IllegalStateException.class, transactionManager::beginAbort); assertThrows(IllegalStateException.class, () -> transactionManager.maybeAddPartition(tp0)); @@ -1694,7 +1706,7 @@ public void testRetryCommitTransactionAfterTimeout() throws Exception { @Test public void testRetryInitTransactionsAfterTimeout() { - TransactionalRequestResult result = transactionManager.initializeTransactions(); + TransactionalRequestResult result = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -1715,10 +1727,10 @@ public void testRetryInitTransactionsAfterTimeout() { assertThrows(IllegalStateException.class, transactionManager::beginCommit); assertThrows(IllegalStateException.class, () -> transactionManager.maybeAddPartition(tp0)); - assertSame(result, transactionManager.initializeTransactions()); + assertSame(result, transactionManager.initializeTransactions(false)); result.await(); assertTrue(result.isAcked()); - assertThrows(IllegalStateException.class, transactionManager::initializeTransactions); + assertThrows(IllegalStateException.class, () -> transactionManager.initializeTransactions(false)); transactionManager.beginTransaction(); assertTrue(transactionManager.hasOngoingTransaction()); @@ -1960,7 +1972,7 @@ public void testMultipleAddPartitionsPerForOneProduce() throws InterruptedExcept }) public void testRetriableErrors(Errors error) { // Ensure FindCoordinator retries. - TransactionalRequestResult result = transactionManager.initializeTransactions(); + TransactionalRequestResult result = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(error, false, CoordinatorType.TRANSACTION, transactionalId); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); @@ -1994,7 +2006,7 @@ public void testRetriableErrors(Errors error) { @Test public void testCoordinatorNotAvailable() { // Ensure FindCoordinator with COORDINATOR_NOT_AVAILABLE error retries. - TransactionalRequestResult result = transactionManager.initializeTransactions(); + TransactionalRequestResult result = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.COORDINATOR_NOT_AVAILABLE, false, CoordinatorType.TRANSACTION, transactionalId); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); @@ -2017,7 +2029,7 @@ public void testInvalidProducerEpochConvertToProducerFencedInInitProducerId() { } private void verifyProducerFencedForInitProducerId(Errors error) { - TransactionalRequestResult result = transactionManager.initializeTransactions(); + TransactionalRequestResult result = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -3802,7 +3814,7 @@ public void testBackgroundInvalidStateTransitionIsFatal() { doInitTransactions(); assertTrue(transactionManager.isTransactional()); - transactionManager.setPoisonStateOnInvalidTransition(true); + transactionManager.setShouldPoisonStateOnInvalidTransitionOverride(true); // Intentionally perform an operation that will cause an invalid state transition. The detection of this // will result in a poisoning of the transaction manager for all subsequent transactional operations since @@ -3815,7 +3827,7 @@ public void testBackgroundInvalidStateTransitionIsFatal() { assertThrows(IllegalStateException.class, () -> transactionManager.beginAbort()); assertThrows(IllegalStateException.class, () -> transactionManager.beginCommit()); assertThrows(IllegalStateException.class, () -> transactionManager.maybeAddPartition(tp0)); - assertThrows(IllegalStateException.class, () -> transactionManager.initializeTransactions()); + assertThrows(IllegalStateException.class, () -> transactionManager.initializeTransactions(false)); assertThrows(IllegalStateException.class, () -> transactionManager.sendOffsetsToTransaction(Collections.emptyMap(), new ConsumerGroupMetadata("fake-group-id"))); } @@ -3852,7 +3864,7 @@ public void testForegroundInvalidStateTransitionIsRecoverable() { @Test public void testTransactionAbortableExceptionInInitProducerId() { - TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -4035,16 +4047,39 @@ private void prepareFindCoordinatorResponse(Errors error, boolean shouldDisconne }, FindCoordinatorResponse.prepareResponse(error, coordinatorKey, brokerNode), shouldDisconnect); } - private void prepareInitPidResponse(Errors error, boolean shouldDisconnect, long producerId, short producerEpoch) { + private void prepareInitPidResponse( + Errors error, + boolean shouldDisconnect, + long producerId, + short producerEpoch + ) { + prepareInitPidResponse(error, shouldDisconnect, producerId, producerEpoch, false, false, (long) -1, (short) -1); + } + + private void prepareInitPidResponse( + Errors error, + boolean shouldDisconnect, + long producerId, + short producerEpoch, + boolean keepPreparedTxn, + boolean enable2Pc, + long ongoingProducerId, + short ongoingProducerEpoch + ) { InitProducerIdResponseData responseData = new InitProducerIdResponseData() - .setErrorCode(error.code()) - .setProducerEpoch(producerEpoch) - .setProducerId(producerId) - .setThrottleTimeMs(0); + .setErrorCode(error.code()) + .setProducerEpoch(producerEpoch) + .setProducerId(producerId) + .setThrottleTimeMs(0) + .setOngoingTxnProducerId(ongoingProducerId) + .setOngoingTxnProducerEpoch(ongoingProducerEpoch); + client.prepareResponse(body -> { InitProducerIdRequest initProducerIdRequest = (InitProducerIdRequest) body; assertEquals(transactionalId, initProducerIdRequest.data().transactionalId()); assertEquals(transactionTimeoutMs, initProducerIdRequest.data().transactionTimeoutMs()); + assertEquals(keepPreparedTxn, initProducerIdRequest.data().keepPreparedTxn()); + assertEquals(enable2Pc, initProducerIdRequest.data().enable2Pc()); return true; }, new InitProducerIdResponse(responseData), shouldDisconnect); } @@ -4309,7 +4344,7 @@ private void doInitTransactions() { } private void doInitTransactions(long producerId, short epoch) { - TransactionalRequestResult result = transactionManager.initializeTransactions(); + TransactionalRequestResult result = transactionManager.initializeTransactions(false); prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); runUntil(() -> transactionManager.coordinator(CoordinatorType.TRANSACTION) != null); assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); @@ -4373,4 +4408,65 @@ private void runUntil(Supplier condition) { ProducerTestUtils.runUntil(sender, condition); } + @Test + public void testInitializeTransactionsWithKeepPreparedTxn() { + initializeTransactionManager(Optional.of(transactionalId), true, true); + + client.prepareResponse( + FindCoordinatorResponse.prepareResponse(Errors.NONE, transactionalId, brokerNode) + ); + + // Simulate an ongoing prepared transaction (ongoingProducerId != -1). + long ongoingProducerId = 999L; + short ongoingEpoch = 10; + short bumpedEpoch = 11; + + prepareInitPidResponse( + Errors.NONE, + false, + ongoingProducerId, + bumpedEpoch, + true, + true, + ongoingProducerId, + ongoingEpoch + ); + + transactionManager.initializeTransactions(true); + runUntil(transactionManager::hasProducerId); + + assertTrue(transactionManager.hasProducerId()); + assertFalse(transactionManager.hasOngoingTransaction()); + assertEquals(ongoingProducerId, transactionManager.producerIdAndEpoch().producerId); + assertEquals(bumpedEpoch, transactionManager.producerIdAndEpoch().epoch); + } + + /** + * This subclass exists only to optionally change the default behavior related to poisoning the state + * on invalid state transition attempts. + */ + private static class TestableTransactionManager extends TransactionManager { + + private Optional shouldPoisonStateOnInvalidTransitionOverride; + + public TestableTransactionManager(LogContext logContext, + String transactionalId, + int transactionTimeoutMs, + long retryBackoffMs, + ApiVersions apiVersions, + boolean enable2Pc) { + super(logContext, transactionalId, transactionTimeoutMs, retryBackoffMs, apiVersions, enable2Pc); + this.shouldPoisonStateOnInvalidTransitionOverride = Optional.empty(); + } + + private void setShouldPoisonStateOnInvalidTransitionOverride(boolean override) { + shouldPoisonStateOnInvalidTransitionOverride = Optional.of(override); + } + + @Override + protected boolean shouldPoisonStateOnInvalidTransition() { + // If there's an override, use it, otherwise invoke the default (i.e. super class) logic. + return shouldPoisonStateOnInvalidTransitionOverride.orElseGet(super::shouldPoisonStateOnInvalidTransition); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java index b28b8274f581f..4674bf2013e3d 100644 --- a/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java +++ b/clients/src/test/java/org/apache/kafka/common/message/MessageTest.java @@ -56,11 +56,13 @@ import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.common.protocol.types.RawTaggedField; +import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; import com.fasterxml.jackson.databind.JsonNode; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.params.ParameterizedTest; import java.lang.reflect.Method; import java.nio.ByteBuffer; @@ -409,90 +411,49 @@ public void testOffsetForLeaderEpochVersions() throws Exception { new OffsetForLeaderEpochRequestData().setReplicaId(-2)); } - @Test - public void testOffsetCommitRequestVersions() throws Exception { - String groupId = "groupId"; - String topicName = "topic"; - String metadata = "metadata"; - int partition = 2; - int offset = 100; - - testAllMessageRoundTrips(new OffsetCommitRequestData() - .setGroupId(groupId) - .setTopics(Collections.singletonList( - new OffsetCommitRequestTopic() - .setName(topicName) - .setPartitions(Collections.singletonList( - new OffsetCommitRequestPartition() - .setPartitionIndex(partition) - .setCommittedMetadata(metadata) - .setCommittedOffset(offset) - ))))); - - Supplier request = - () -> new OffsetCommitRequestData() - .setGroupId(groupId) - .setMemberId("memberId") - .setGroupInstanceId("instanceId") - .setTopics(Collections.singletonList( - new OffsetCommitRequestTopic() - .setName(topicName) - .setPartitions(Collections.singletonList( - new OffsetCommitRequestPartition() - .setPartitionIndex(partition) - .setCommittedLeaderEpoch(10) - .setCommittedMetadata(metadata) - .setCommittedOffset(offset) - )))) - .setRetentionTimeMs(20); - - for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) { - OffsetCommitRequestData requestData = request.get(); - - if (version > 4) { - requestData.setRetentionTimeMs(-1); - } - - if (version < 6) { - requestData.topics().get(0).partitions().get(0).setCommittedLeaderEpoch(-1); - } - - if (version < 7) { - requestData.setGroupInstanceId(null); - } - - if (version >= 2 && version <= 4) { - testAllMessageRoundTripsBetweenVersions(version, (short) 5, requestData, requestData); - } else { - testAllMessageRoundTripsFromVersion(version, requestData); - } - } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) + public void testOffsetCommitRequestVersions(short version) throws Exception { + OffsetCommitRequestData request = new OffsetCommitRequestData() + .setGroupId("groupId") + .setMemberId("memberId") + .setGenerationIdOrMemberEpoch(version >= 1 ? 10 : -1) + .setGroupInstanceId(version >= 7 ? "instanceId" : null) + .setRetentionTimeMs((version >= 2 && version <= 4) ? 20 : -1) + .setTopics(singletonList( + new OffsetCommitRequestTopic() + .setTopicId(version >= 10 ? Uuid.randomUuid() : Uuid.ZERO_UUID) + .setName(version < 10 ? "topic" : "") + .setPartitions(singletonList( + new OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedMetadata("metadata") + .setCommittedOffset(100) + .setCommittedLeaderEpoch(version >= 6 ? 10 : -1) + + )) + )); + + testMessageRoundTrip(version, request, request); } - @Test - public void testOffsetCommitResponseVersions() throws Exception { - Supplier response = - () -> new OffsetCommitResponseData() - .setTopics( - singletonList( - new OffsetCommitResponseTopic() - .setName("topic") - .setPartitions(singletonList( - new OffsetCommitResponsePartition() - .setPartitionIndex(1) - .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) - )) - ) - ) - .setThrottleTimeMs(20); - - for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) { - OffsetCommitResponseData responseData = response.get(); - if (version < 3) { - responseData.setThrottleTimeMs(0); - } - testAllMessageRoundTripsFromVersion(version, responseData); - } + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) + public void testOffsetCommitResponseVersions(short version) throws Exception { + OffsetCommitResponseData response = new OffsetCommitResponseData() + .setThrottleTimeMs(version >= 3 ? 20 : 0) + .setTopics(singletonList( + new OffsetCommitResponseTopic() + .setTopicId(version >= 10 ? Uuid.randomUuid() : Uuid.ZERO_UUID) + .setName(version < 10 ? "topic" : "") + .setPartitions(singletonList( + new OffsetCommitResponsePartition() + .setPartitionIndex(1) + .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) + )) + )); + + testMessageRoundTrip(version, response, response); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java index a4abf88c83703..bf199275db868 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DeleteAclsResponseTest.java @@ -22,13 +22,12 @@ import org.apache.kafka.common.message.DeleteAclsResponseData; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsFilterResult; import org.apache.kafka.common.message.DeleteAclsResponseData.DeleteAclsMatchingAcl; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourceType; import org.junit.jupiter.api.Test; -import java.nio.ByteBuffer; - import static java.util.Arrays.asList; import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -98,9 +97,9 @@ public void shouldRoundTripV1() { .setThrottleTimeMs(10) .setFilterResults(asList(LITERAL_RESPONSE, PREFIXED_RESPONSE)), V1); - final ByteBuffer buffer = original.serialize(V1); + final Readable readable = original.serialize(V1); - final DeleteAclsResponse result = DeleteAclsResponse.parse(buffer, V1); + final DeleteAclsResponse result = DeleteAclsResponse.parse(readable, V1); assertEquals(original.filterResults(), result.filterResults()); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java index 243b3a80e6f29..f2925b45828f5 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/DescribeAclsResponseTest.java @@ -25,13 +25,13 @@ import org.apache.kafka.common.message.DescribeAclsResponseData.AclDescription; import org.apache.kafka.common.message.DescribeAclsResponseData.DescribeAclsResource; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.junit.jupiter.api.Test; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -90,9 +90,9 @@ public void shouldThrowIfUnknown() { public void shouldRoundTripV1() { List resources = Arrays.asList(LITERAL_ACL1, PREFIXED_ACL1); final DescribeAclsResponse original = buildResponse(100, Errors.NONE, resources); - final ByteBuffer buffer = original.serialize(V1); + final Readable readable = original.serialize(V1); - final DescribeAclsResponse result = DescribeAclsResponse.parse(buffer, V1); + final DescribeAclsResponse result = DescribeAclsResponse.parse(readable, V1); assertResponseEquals(original, result); final DescribeAclsResponse result2 = buildResponse(100, Errors.NONE, DescribeAclsResponse.aclsResources( diff --git a/clients/src/test/java/org/apache/kafka/common/requests/LeaveGroupResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/LeaveGroupResponseTest.java index 0f098b462a21a..4d73c042c5bb7 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/LeaveGroupResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/LeaveGroupResponseTest.java @@ -22,13 +22,13 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.EnumMap; @@ -111,9 +111,9 @@ public void testEqualityWithSerialization() { .setThrottleTimeMs(throttleTimeMs); for (short version : ApiKeys.LEAVE_GROUP.allVersions()) { LeaveGroupResponse primaryResponse = LeaveGroupResponse.parse( - MessageUtil.toByteBufferAccessor(responseData, version).buffer(), version); + MessageUtil.toByteBufferAccessor(responseData, version), version); LeaveGroupResponse secondaryResponse = LeaveGroupResponse.parse( - MessageUtil.toByteBufferAccessor(responseData, version).buffer(), version); + MessageUtil.toByteBufferAccessor(responseData, version), version); assertEquals(primaryResponse, primaryResponse); assertEquals(primaryResponse, secondaryResponse); @@ -130,7 +130,7 @@ public void testParse() { .setThrottleTimeMs(throttleTimeMs); for (short version : ApiKeys.LEAVE_GROUP.allVersions()) { - ByteBuffer buffer = MessageUtil.toByteBufferAccessor(data, version).buffer(); + Readable buffer = MessageUtil.toByteBufferAccessor(data, version); LeaveGroupResponse leaveGroupResponse = LeaveGroupResponse.parse(buffer, version); assertEquals(expectedErrorCounts, leaveGroupResponse.errorCounts()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java b/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java index 161a4dd5f1192..9cd95cfec769e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitRequestTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.OffsetCommitRequestData; import org.apache.kafka.common.message.OffsetCommitRequestData.OffsetCommitRequestPartition; @@ -45,6 +46,8 @@ public class OffsetCommitRequestTest { protected static String groupId = "groupId"; protected static String memberId = "consumerId"; protected static String groupInstanceId = "groupInstanceId"; + protected static Uuid topicIdOne = Uuid.randomUuid(); + protected static Uuid topicIdTwo = Uuid.randomUuid(); protected static String topicOne = "topicOne"; protected static String topicTwo = "topicTwo"; protected static int partitionOne = 1; @@ -61,6 +64,7 @@ public class OffsetCommitRequestTest { public void setUp() { List topics = Arrays.asList( new OffsetCommitRequestTopic() + .setTopicId(topicIdOne) .setName(topicOne) .setPartitions(Collections.singletonList( new OffsetCommitRequestPartition() @@ -70,6 +74,7 @@ public void setUp() { .setCommittedMetadata(metadata) )), new OffsetCommitRequestTopic() + .setTopicId(topicIdTwo) .setName(topicTwo) .setPartitions(Collections.singletonList( new OffsetCommitRequestPartition() @@ -90,7 +95,7 @@ public void testConstructor() { expectedOffsets.put(new TopicPartition(topicOne, partitionOne), offset); expectedOffsets.put(new TopicPartition(topicTwo, partitionTwo), offset); - OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder(data); + OffsetCommitRequest.Builder builder = OffsetCommitRequest.Builder.forTopicNames(data); for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) { OffsetCommitRequest request = builder.build(version); @@ -105,7 +110,7 @@ public void testConstructor() { @Test public void testVersionSupportForGroupInstanceId() { - OffsetCommitRequest.Builder builder = new OffsetCommitRequest.Builder( + OffsetCommitRequest.Builder builder = OffsetCommitRequest.Builder.forTopicNames( new OffsetCommitRequestData() .setGroupId(groupId) .setMemberId(memberId) @@ -127,12 +132,14 @@ public void testGetErrorResponse() { OffsetCommitResponseData expectedResponse = new OffsetCommitResponseData() .setTopics(Arrays.asList( new OffsetCommitResponseTopic() + .setTopicId(topicIdOne) .setName(topicOne) .setPartitions(Collections.singletonList( new OffsetCommitResponsePartition() .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code()) .setPartitionIndex(partitionOne))), new OffsetCommitResponseTopic() + .setTopicId(topicIdTwo) .setName(topicTwo) .setPartitions(Collections.singletonList( new OffsetCommitResponsePartition() diff --git a/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitResponseTest.java index 7351db55b544e..dcfb988116797 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/OffsetCommitResponseTest.java @@ -23,11 +23,11 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.MessageUtil; +import org.apache.kafka.common.protocol.Readable; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.EnumMap; @@ -88,8 +88,8 @@ public void testParse() { .setThrottleTimeMs(throttleTimeMs); for (short version : ApiKeys.OFFSET_COMMIT.allVersions()) { - ByteBuffer buffer = MessageUtil.toByteBufferAccessor(data, version).buffer(); - OffsetCommitResponse response = OffsetCommitResponse.parse(buffer, version); + Readable readable = MessageUtil.toByteBufferAccessor(data, version); + OffsetCommitResponse response = OffsetCommitResponse.parse(readable, version); assertEquals(expectedErrorCounts, response.errorCounts()); if (version >= 3) { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java index d0ef79b4479e8..2d8c6ef43f94a 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/OffsetFetchResponseTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopic; import org.apache.kafka.common.message.OffsetFetchResponseData.OffsetFetchResponseTopics; import org.apache.kafka.common.protocol.ApiKeys; -import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData; @@ -197,7 +196,7 @@ public void testStructBuild() { OffsetFetchResponse latestResponse = new OffsetFetchResponse(throttleTimeMs, Errors.NONE, partitionDataMap); OffsetFetchResponseData data = new OffsetFetchResponseData( - new ByteBufferAccessor(latestResponse.serialize(version)), version); + latestResponse.serialize(version), version); OffsetFetchResponse oldResponse = new OffsetFetchResponse(data, version); @@ -252,7 +251,7 @@ public void testStructBuild() { Collections.singletonMap(groupOne, Errors.NONE), Collections.singletonMap(groupOne, partitionDataMap)); OffsetFetchResponseData data = new OffsetFetchResponseData( - new ByteBufferAccessor(latestResponse.serialize(version)), version); + latestResponse.serialize(version), version); OffsetFetchResponse oldResponse = new OffsetFetchResponse(data, version); assertEquals(Errors.NONE.code(), data.groups().get(0).errorCode()); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java index aad3be459a682..887fccae61560 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestContextTest.java @@ -84,7 +84,7 @@ public void testSerdeUnsupportedApiVersionRequest() throws Exception { assertEquals(correlationId, responseHeader.correlationId()); ApiVersionsResponse response = (ApiVersionsResponse) AbstractResponse.parseResponse(ApiKeys.API_VERSIONS, - responseBuffer, (short) 0); + new ByteBufferAccessor(responseBuffer), (short) 0); assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data().errorCode()); assertTrue(response.data().apiKeys().isEmpty()); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b33dec17d9a0f..2179eb4d27f4d 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -329,6 +329,8 @@ // This class performs tests requests and responses for all API keys public class RequestResponseTest { + private static final Uuid TOPIC_ID = Uuid.randomUuid(); + // Exception includes a message that we verify is not included in error responses private final UnknownServerException unknownServerException = new UnknownServerException("secret"); @@ -663,7 +665,7 @@ private void verifyFetchResponseFullWrite(short version, FetchResponse fetchResp ResponseHeader responseHeader = ResponseHeader.parse(channel.buffer(), responseHeaderVersion); assertEquals(correlationId, responseHeader.correlationId()); - assertEquals(fetchResponse.serialize(version), buf); + assertEquals(fetchResponse.serialize(version).buffer(), buf); FetchResponseData deserialized = new FetchResponseData(new ByteBufferAccessor(buf), version); ObjectSerializationCache serializationCache = new ObjectSerializationCache(); assertEquals(size, responseHeader.size() + deserialized.size(serializationCache, version)); @@ -915,8 +917,8 @@ private ApiVersionsResponse defaultApiVersionsResponse() { @Test public void testApiVersionResponseParsingFallback() { for (short version : API_VERSIONS.allVersions()) { - ByteBuffer buffer = defaultApiVersionsResponse().serialize((short) 0); - ApiVersionsResponse response = ApiVersionsResponse.parse(buffer, version); + ByteBufferAccessor readable = defaultApiVersionsResponse().serialize((short) 0); + ApiVersionsResponse response = ApiVersionsResponse.parse(readable, version); assertEquals(Errors.NONE.code(), response.data().errorCode()); } } @@ -924,15 +926,16 @@ public void testApiVersionResponseParsingFallback() { @Test public void testApiVersionResponseParsingFallbackException() { for (final short version : API_VERSIONS.allVersions()) { - assertThrows(BufferUnderflowException.class, () -> ApiVersionsResponse.parse(ByteBuffer.allocate(0), version)); + assertThrows(BufferUnderflowException.class, + () -> ApiVersionsResponse.parse(new ByteBufferAccessor(ByteBuffer.allocate(0)), version)); } } @Test public void testApiVersionResponseParsing() { for (short version : API_VERSIONS.allVersions()) { - ByteBuffer buffer = defaultApiVersionsResponse().serialize(version); - ApiVersionsResponse response = ApiVersionsResponse.parse(buffer, version); + ByteBufferAccessor readable = defaultApiVersionsResponse().serialize(version); + ApiVersionsResponse response = ApiVersionsResponse.parse(readable, version); assertEquals(Errors.NONE.code(), response.data().errorCode()); } } @@ -1998,9 +2001,10 @@ private void checkResponse(AbstractResponse response, short version) { // Check for equality and hashCode of the Struct only if indicated (it is likely to fail if any of the fields // in the response is a HashMap with multiple elements since ordering of the elements may vary) try { - ByteBuffer serializedBytes = response.serialize(version); - AbstractResponse deserialized = AbstractResponse.parseResponse(response.apiKey(), serializedBytes, version); - ByteBuffer serializedBytes2 = deserialized.serialize(version); + ByteBufferAccessor readable = response.serialize(version); + ByteBuffer serializedBytes = readable.buffer(); + AbstractResponse deserialized = AbstractResponse.parseResponse(response.apiKey(), readable, version); + ByteBuffer serializedBytes2 = deserialized.serialize(version).buffer(); serializedBytes.rewind(); assertEquals(serializedBytes, serializedBytes2, "Response " + response + "failed equality test"); } catch (Exception e) { @@ -2401,7 +2405,7 @@ private MetadataResponse createMetadataResponse() { } private OffsetCommitRequest createOffsetCommitRequest(short version) { - return new OffsetCommitRequest.Builder(new OffsetCommitRequestData() + return OffsetCommitRequest.Builder.forTopicNames(new OffsetCommitRequestData() .setGroupId("group1") .setMemberId("consumer1") .setGroupInstanceId(null) @@ -2409,6 +2413,7 @@ private OffsetCommitRequest createOffsetCommitRequest(short version) { .setTopics(singletonList( new OffsetCommitRequestData.OffsetCommitRequestTopic() .setName("test") + .setTopicId(TOPIC_ID) .setPartitions(asList( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) @@ -2430,6 +2435,7 @@ private OffsetCommitResponse createOffsetCommitResponse() { .setTopics(singletonList( new OffsetCommitResponseData.OffsetCommitResponseTopic() .setName("test") + .setTopicId(TOPIC_ID) .setPartitions(singletonList( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) diff --git a/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitResponseTest.java index a0bea9095424f..bd6f98ed33937 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/TxnOffsetCommitResponseTest.java @@ -57,7 +57,7 @@ public void testParse() { for (short version : ApiKeys.TXN_OFFSET_COMMIT.allVersions()) { TxnOffsetCommitResponse response = TxnOffsetCommitResponse.parse( - MessageUtil.toByteBufferAccessor(data, version).buffer(), version); + MessageUtil.toByteBufferAccessor(data, version), version); assertEquals(expectedErrorCounts, response.errorCounts()); assertEquals(throttleTimeMs, response.throttleTimeMs()); assertEquals(version >= 1, response.shouldClientThrottle(version)); diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 8261c90014cf3..c21b5d1102305 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -51,6 +51,7 @@ import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.network.TransportLayer; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.requests.AbstractRequest; @@ -763,7 +764,7 @@ public void testApiVersionsRequestWithServerUnsupportedVersion() throws Exceptio selector.send(new NetworkSend(node, request.toSend(header))); ByteBuffer responseBuffer = waitForResponse(); ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion((short) 0)); - ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, (short) 0); + ApiVersionsResponse response = ApiVersionsResponse.parse(new ByteBufferAccessor(responseBuffer), (short) 0); assertEquals(Errors.UNSUPPORTED_VERSION.code(), response.data().errorCode()); ApiVersion apiVersion = response.data().apiKeys().find(ApiKeys.API_VERSIONS.id); @@ -822,7 +823,7 @@ public void testInvalidApiVersionsRequest() throws Exception { ByteBuffer responseBuffer = waitForResponse(); ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion(version)); ApiVersionsResponse response = - ApiVersionsResponse.parse(responseBuffer, version); + ApiVersionsResponse.parse(new ByteBufferAccessor(responseBuffer), version); assertEquals(Errors.INVALID_REQUEST.code(), response.data().errorCode()); // Send ApiVersionsRequest with a supported version. This should succeed. @@ -861,7 +862,7 @@ public void testValidApiVersionsRequest() throws Exception { selector.send(new NetworkSend(node, request.toSend(header))); ByteBuffer responseBuffer = waitForResponse(); ResponseHeader.parse(responseBuffer, ApiKeys.API_VERSIONS.responseHeaderVersion(version)); - ApiVersionsResponse response = ApiVersionsResponse.parse(responseBuffer, version); + ApiVersionsResponse response = ApiVersionsResponse.parse(new ByteBufferAccessor(responseBuffer), version); assertEquals(Errors.NONE.code(), response.data().errorCode()); // Test that client can authenticate successfully diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java index c45db777e4ca9..f9734dabb7a8c 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticatorTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.TransportLayer; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.ApiVersionsRequest; import org.apache.kafka.common.requests.ApiVersionsResponse; @@ -198,7 +199,7 @@ public void testSessionExpiresAtTokenExpiryDespiteNoReauthIsSet() throws IOExcep ByteBuffer secondResponseSent = getResponses(transportLayer).get(1); consumeSizeAndHeader(secondResponseSent); - SaslAuthenticateResponse response = SaslAuthenticateResponse.parse(secondResponseSent, (short) 2); + SaslAuthenticateResponse response = SaslAuthenticateResponse.parse(new ByteBufferAccessor(secondResponseSent), (short) 2); assertEquals(tokenExpirationDuration.toMillis(), response.sessionLifetimeMs()); } } @@ -231,7 +232,7 @@ public void testSessionExpiresAtMaxReauthTime() throws IOException { ByteBuffer secondResponseSent = getResponses(transportLayer).get(1); consumeSizeAndHeader(secondResponseSent); - SaslAuthenticateResponse response = SaslAuthenticateResponse.parse(secondResponseSent, (short) 2); + SaslAuthenticateResponse response = SaslAuthenticateResponse.parse(new ByteBufferAccessor(secondResponseSent), (short) 2); assertEquals(maxReauthMs, response.sessionLifetimeMs()); } } @@ -264,7 +265,7 @@ public void testSessionExpiresAtTokenExpiry() throws IOException { ByteBuffer secondResponseSent = getResponses(transportLayer).get(1); consumeSizeAndHeader(secondResponseSent); - SaslAuthenticateResponse response = SaslAuthenticateResponse.parse(secondResponseSent, (short) 2); + SaslAuthenticateResponse response = SaslAuthenticateResponse.parse(new ByteBufferAccessor(secondResponseSent), (short) 2); assertEquals(tokenExpiryShorterThanMaxReauth.toMillis(), response.sessionLifetimeMs()); } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java b/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java index ea510414f0481..6c80d2b5df5f1 100644 --- a/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/annotation/ApiKeyVersionsProvider.java @@ -52,7 +52,7 @@ public void accept(ApiKeyVersionsSource source) { if (toVersion > latestVersion) { throw new IllegalArgumentException(String.format("The toVersion %s is newer than the latest version %s", - fromVersion, latestVersion)); + toVersion, latestVersion)); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index 5a787909925da..bce671293882a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -203,83 +203,91 @@ public boolean isReady() { @Override public void onStartup(String connector) { statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.RUNNING, - workerId, generation())); + workerId, generation(), worker.connectorVersion(connector))); } @Override public void onStop(String connector) { statusBackingStore.put(new ConnectorStatus(connector, AbstractStatus.State.STOPPED, - workerId, generation())); + workerId, generation(), worker.connectorVersion(connector))); } @Override public void onPause(String connector) { statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.PAUSED, - workerId, generation())); + workerId, generation(), worker.connectorVersion(connector))); } @Override public void onResume(String connector) { statusBackingStore.put(new ConnectorStatus(connector, TaskStatus.State.RUNNING, - workerId, generation())); + workerId, generation(), worker.connectorVersion(connector))); } @Override public void onShutdown(String connector) { statusBackingStore.putSafe(new ConnectorStatus(connector, ConnectorStatus.State.UNASSIGNED, - workerId, generation())); + workerId, generation(), worker.connectorVersion(connector))); } @Override public void onFailure(String connector, Throwable cause) { statusBackingStore.putSafe(new ConnectorStatus(connector, ConnectorStatus.State.FAILED, - trace(cause), workerId, generation())); + trace(cause), workerId, generation(), worker.connectorVersion(connector))); } @Override public void onStartup(ConnectorTaskId id) { - statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RUNNING, workerId, generation())); + statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RUNNING, workerId, generation(), null, + worker.taskVersion(id))); } @Override public void onFailure(ConnectorTaskId id, Throwable cause) { - statusBackingStore.putSafe(new TaskStatus(id, TaskStatus.State.FAILED, workerId, generation(), trace(cause))); + statusBackingStore.putSafe(new TaskStatus(id, TaskStatus.State.FAILED, workerId, generation(), trace(cause), + worker.taskVersion(id))); } @Override public void onShutdown(ConnectorTaskId id) { - statusBackingStore.putSafe(new TaskStatus(id, TaskStatus.State.UNASSIGNED, workerId, generation())); + statusBackingStore.putSafe(new TaskStatus(id, TaskStatus.State.UNASSIGNED, workerId, generation(), null, + worker.taskVersion(id))); } @Override public void onResume(ConnectorTaskId id) { - statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RUNNING, workerId, generation())); + statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RUNNING, workerId, generation(), null, + worker.taskVersion(id))); } @Override public void onPause(ConnectorTaskId id) { - statusBackingStore.put(new TaskStatus(id, TaskStatus.State.PAUSED, workerId, generation())); + statusBackingStore.put(new TaskStatus(id, TaskStatus.State.PAUSED, workerId, generation(), null, + worker.taskVersion(id))); } @Override public void onDeletion(String connector) { for (TaskStatus status : statusBackingStore.getAll(connector)) onDeletion(status.id()); - statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.DESTROYED, workerId, generation())); + statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.DESTROYED, workerId, generation(), + worker.connectorVersion(connector))); } @Override public void onDeletion(ConnectorTaskId id) { - statusBackingStore.put(new TaskStatus(id, TaskStatus.State.DESTROYED, workerId, generation())); + statusBackingStore.put(new TaskStatus(id, TaskStatus.State.DESTROYED, workerId, generation(), null, + worker.taskVersion(id))); } public void onRestart(String connector) { statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.RESTARTING, - workerId, generation())); + workerId, generation(), worker.connectorVersion(connector))); } public void onRestart(ConnectorTaskId id) { - statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RESTARTING, workerId, generation())); + statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RESTARTING, workerId, generation(), null, + worker.taskVersion(id))); } @Override @@ -347,12 +355,12 @@ public ConnectorStateInfo connectorStatus(String connName) { Collection tasks = statusBackingStore.getAll(connName); ConnectorStateInfo.ConnectorState connectorState = new ConnectorStateInfo.ConnectorState( - connector.state().toString(), connector.workerId(), connector.trace()); + connector.state().toString(), connector.workerId(), connector.trace(), connector.version()); List taskStates = new ArrayList<>(); for (TaskStatus status : tasks) { taskStates.add(new ConnectorStateInfo.TaskState(status.id().task(), - status.state().toString(), status.workerId(), status.trace())); + status.state().toString(), status.workerId(), status.trace(), status.version())); } Collections.sort(taskStates); @@ -388,7 +396,7 @@ public ConnectorStateInfo.TaskState taskStatus(ConnectorTaskId id) { throw new NotFoundException("No status found for task " + id); return new ConnectorStateInfo.TaskState(id.task(), status.state().toString(), - status.workerId(), status.trace()); + status.workerId(), status.trace(), status.version()); } @Override @@ -626,7 +634,8 @@ public Optional buildRestartPlan(RestartRequest request) { ConnectorStateInfo.ConnectorState connectorInfoState = new ConnectorStateInfo.ConnectorState( connectorState.toString(), connectorStatus.workerId(), - connectorStatus.trace() + connectorStatus.trace(), + connectorStatus.version() ); // Collect the task states, If requested, mark the task as restarting @@ -638,7 +647,8 @@ public Optional buildRestartPlan(RestartRequest request) { taskStatus.id().task(), taskState.toString(), taskStatus.workerId(), - taskStatus.trace() + taskStatus.trace(), + taskStatus.version() ); }) .collect(Collectors.toList()); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java index 76036d610d738..fc8bc7ca05061 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java @@ -34,18 +34,29 @@ public enum State { private final State state; private final String trace; private final String workerId; + private final String version; private final int generation; public AbstractStatus(T id, State state, String workerId, int generation, - String trace) { + String trace, + String version) { this.id = id; this.state = state; this.workerId = workerId; this.generation = generation; this.trace = trace; + this.version = version; + } + + public AbstractStatus(T id, + State state, + String workerId, + int generation, + String trace) { + this(id, state, workerId, generation, trace, null); } public T id() { @@ -68,12 +79,17 @@ public int generation() { return generation; } + public String version() { + return version; + } + @Override public String toString() { return "Status{" + "id=" + id + ", state=" + state + ", workerId='" + workerId + '\'' + + ", version='" + version + '\'' + ", generation=" + generation + '}'; } @@ -89,7 +105,8 @@ public boolean equals(Object o) { && Objects.equals(id, that.id) && state == that.state && Objects.equals(trace, that.trace) - && Objects.equals(workerId, that.workerId); + && Objects.equals(workerId, that.workerId) + && Objects.equals(version, that.version); } @Override @@ -98,6 +115,7 @@ public int hashCode() { result = 31 * result + (state != null ? state.hashCode() : 0); result = 31 * result + (trace != null ? trace.hashCode() : 0); result = 31 * result + (workerId != null ? workerId.hashCode() : 0); + result = 31 * result + (version != null ? version.hashCode() : 0); result = 31 * result + generation; return result; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java index 683eb3abed0f2..9a74d81770fa7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTask.java @@ -203,6 +203,7 @@ protected abstract void producerSendFailed( private final boolean topicTrackingEnabled; private final TopicCreation topicCreation; private final Executor closeExecutor; + private final String version; // Visible for testing List toSend; @@ -236,11 +237,12 @@ protected AbstractWorkerSourceTask(ConnectorTaskId id, StatusBackingStore statusBackingStore, Executor closeExecutor, Supplier>> errorReportersSupplier, + TaskPluginsMetadata pluginsMetadata, Function pluginLoaderSwapper) { super(id, statusListener, initialState, loader, connectMetrics, errorMetrics, retryWithToleranceOperator, transformationChain, errorReportersSupplier, - time, statusBackingStore, pluginLoaderSwapper); + time, statusBackingStore, pluginsMetadata, pluginLoaderSwapper); this.workerConfig = workerConfig; this.task = task; @@ -258,6 +260,7 @@ protected AbstractWorkerSourceTask(ConnectorTaskId id, this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics); this.topicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG); this.topicCreation = TopicCreation.newTopicCreation(workerConfig, topicGroups); + this.version = task.version(); } @Override @@ -391,6 +394,11 @@ public void execute() { finalOffsetCommit(false); } + @Override + public String taskVersion() { + return version; + } + /** * Try to send a batch of records. If a send fails and is retriable, this saves the remainder of the batch so it can * be retried after backing off. If a send fails and is not retriable, this will throw a ConnectException. diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java index 1d144440f2c20..496c838cf45f9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectMetricsRegistry.java @@ -37,6 +37,10 @@ public class ConnectMetricsRegistry { public static final String WORKER_GROUP_NAME = "connect-worker-metrics"; public static final String WORKER_REBALANCE_GROUP_NAME = "connect-worker-rebalance-metrics"; public static final String TASK_ERROR_HANDLING_GROUP_NAME = "task-error-metrics"; + public static final String TRANSFORMS_GROUP = "connector-transform-metrics"; + public static final String PREDICATES_GROUP = "connector-predicate-metrics"; + public static final String TRANSFORM_TAG_NAME = "transform"; + public static final String PREDICATE_TAG_NAME = "predicate"; private final List allTemplates = new ArrayList<>(); public final MetricNameTemplate connectorStatus; @@ -59,6 +63,17 @@ public class ConnectMetricsRegistry { public final MetricNameTemplate taskBatchSizeAvg; public final MetricNameTemplate taskCommitFailurePercentage; public final MetricNameTemplate taskCommitSuccessPercentage; + public final MetricNameTemplate taskConnectorClass; + public final MetricNameTemplate taskConnectorClassVersion; + public final MetricNameTemplate taskConnectorType; + public final MetricNameTemplate taskClass; + public final MetricNameTemplate taskVersion; + public final MetricNameTemplate taskKeyConverterClass; + public final MetricNameTemplate taskValueConverterClass; + public final MetricNameTemplate taskKeyConverterVersion; + public final MetricNameTemplate taskValueConverterVersion; + public final MetricNameTemplate taskHeaderConverterClass; + public final MetricNameTemplate taskHeaderConverterVersion; public final MetricNameTemplate sourceRecordPollRate; public final MetricNameTemplate sourceRecordPollTotal; public final MetricNameTemplate sourceRecordWriteRate; @@ -115,6 +130,10 @@ public class ConnectMetricsRegistry { public final MetricNameTemplate transactionSizeMin; public final MetricNameTemplate transactionSizeMax; public final MetricNameTemplate transactionSizeAvg; + public final MetricNameTemplate transformClass; + public final MetricNameTemplate transformVersion; + public final MetricNameTemplate predicateClass; + public final MetricNameTemplate predicateVersion; public Map connectorStatusMetrics; @@ -164,6 +183,43 @@ public ConnectMetricsRegistry(Set tags) { taskCommitSuccessPercentage = createTemplate("offset-commit-success-percentage", TASK_GROUP_NAME, "The average percentage of this task's offset commit attempts that succeeded.", workerTaskTags); + taskConnectorClass = createTemplate("connector-class", TASK_GROUP_NAME, "The name of the connector class.", workerTaskTags); + taskConnectorClassVersion = createTemplate("connector-version", TASK_GROUP_NAME, + "The version of the connector class, as reported by the connector.", workerTaskTags); + taskConnectorType = createTemplate("connector-type", TASK_GROUP_NAME, "The type of the connector. One of 'source' or 'sink'.", + workerTaskTags); + taskClass = createTemplate("task-class", TASK_GROUP_NAME, "The class name of the task.", workerTaskTags); + taskVersion = createTemplate("task-version", TASK_GROUP_NAME, "The version of the task.", workerTaskTags); + taskKeyConverterClass = createTemplate("key-converter-class", TASK_GROUP_NAME, + "The fully qualified class name from key.converter", workerTaskTags); + taskValueConverterClass = createTemplate("value-converter-class", TASK_GROUP_NAME, + "The fully qualified class name from value.converter", workerTaskTags); + taskKeyConverterVersion = createTemplate("key-converter-version", TASK_GROUP_NAME, + "The version instantiated for key.converter. May be undefined", workerTaskTags); + taskValueConverterVersion = createTemplate("value-converter-version", TASK_GROUP_NAME, + "The version instantiated for value.converter. May be undefined", workerTaskTags); + taskHeaderConverterClass = createTemplate("header-converter-class", TASK_GROUP_NAME, + "The fully qualified class name from header.converter", workerTaskTags); + taskHeaderConverterVersion = createTemplate("header-converter-version", TASK_GROUP_NAME, + "The version instantiated for header.converter. May be undefined", workerTaskTags); + + /* Transformation Metrics */ + Set transformTags = new LinkedHashSet<>(tags); + transformTags.addAll(workerTaskTags); + transformTags.add(TRANSFORM_TAG_NAME); + transformClass = createTemplate("transform-class", TRANSFORMS_GROUP, + "The class name of the transformation class", transformTags); + transformVersion = createTemplate("transform-version", TRANSFORMS_GROUP, + "The version of the transformation class", transformTags); + + /* Predicate Metrics */ + Set predicateTags = new LinkedHashSet<>(tags); + predicateTags.addAll(workerTaskTags); + predicateTags.add(PREDICATE_TAG_NAME); + predicateClass = createTemplate("predicate-class", PREDICATES_GROUP, + "The class name of the predicate class", predicateTags); + predicateVersion = createTemplate("predicate-version", PREDICATES_GROUP, + "The version of the predicate class", predicateTags); /* Source worker task level */ Set sourceTaskTags = new LinkedHashSet<>(tags); @@ -426,4 +482,20 @@ public String workerRebalanceGroupName() { public String taskErrorHandlingGroupName() { return TASK_ERROR_HANDLING_GROUP_NAME; } + + public String transformsGroupName() { + return TRANSFORMS_GROUP; + } + + public String transformsTagName() { + return TRANSFORM_TAG_NAME; + } + + public String predicatesGroupName() { + return PREDICATES_GROUP; + } + + public String predicateTagName() { + return PREDICATE_TAG_NAME; + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index 3a301335502ff..efd421bd2e2e1 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -374,7 +374,7 @@ public > List> transformationS final String versionConfig = prefix + WorkerConfig.PLUGIN_VERSION_SUFFIX; final Transformation transformation = getTransformationOrPredicate(plugins, typeConfig, versionConfig); Map configs = originalsWithPrefix(prefix); - Object predicateAlias = configs.remove(TransformationStage.PREDICATE_CONFIG); + String predicateAlias = (String) configs.remove(TransformationStage.PREDICATE_CONFIG); Object negate = configs.remove(TransformationStage.NEGATE_CONFIG); transformation.configure(configs); Plugin> transformationPlugin = metrics.wrap(transformation, connectorTaskId, alias); @@ -384,10 +384,24 @@ public > List> transformationS final String predicateVersionConfig = predicatePrefix + WorkerConfig.PLUGIN_VERSION_SUFFIX; Predicate predicate = getTransformationOrPredicate(plugins, predicateTypeConfig, predicateVersionConfig); predicate.configure(originalsWithPrefix(predicatePrefix)); - Plugin> predicatePlugin = metrics.wrap(predicate, connectorTaskId, (String) predicateAlias); - transformations.add(new TransformationStage<>(predicatePlugin, negate != null && Boolean.parseBoolean(negate.toString()), transformationPlugin, plugins.safeLoaderSwapper())); + Plugin> predicatePlugin = metrics.wrap(predicate, connectorTaskId, predicateAlias); + transformations.add(new TransformationStage<>( + predicatePlugin, + predicateAlias, + plugins.pluginVersion(predicate.getClass().getName(), predicate.getClass().getClassLoader(), PluginType.PREDICATE), + negate != null && Boolean.parseBoolean(negate.toString()), + transformationPlugin, + alias, + plugins.pluginVersion(transformation.getClass().getName(), transformation.getClass().getClassLoader(), PluginType.TRANSFORMATION), + plugins.safeLoaderSwapper()) + ); } else { - transformations.add(new TransformationStage<>(transformationPlugin, plugins.safeLoaderSwapper())); + transformations.add(new TransformationStage<>( + transformationPlugin, + alias, + plugins.pluginVersion(transformation.getClass().getName(), transformation.getClass().getClassLoader(), PluginType.TRANSFORMATION), + plugins.safeLoaderSwapper()) + ); } } catch (Exception e) { throw new ConnectException(e); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java index 10ed188cdf883..d704a3374e296 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java @@ -19,8 +19,12 @@ public class ConnectorStatus extends AbstractStatus { - public ConnectorStatus(String connector, State state, String msg, String workerUrl, int generation) { - super(connector, state, workerUrl, generation, msg); + public ConnectorStatus(String connector, State state, String msg, String workerUrl, int generation, String version) { + super(connector, state, workerUrl, generation, msg, version); + } + + public ConnectorStatus(String connector, State state, String workerUrl, int generation, String version) { + super(connector, state, workerUrl, generation, null, version); } public ConnectorStatus(String connector, State state, String workerUrl, int generation) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java index 42e43babe557d..d6f4ffbd4b9be 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTask.java @@ -104,11 +104,12 @@ public ExactlyOnceWorkerSourceTask(ConnectorTaskId id, Runnable preProducerCheck, Runnable postProducerCheck, Supplier>> errorReportersSupplier, + TaskPluginsMetadata pluginsMetadata, Function pluginLoaderSwapper) { super(id, task, statusListener, initialState, configState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, transformationChain, buildTransactionContext(sourceConfig), producer, admin, topicGroups, offsetReader, offsetWriter, offsetStore, workerConfig, connectMetrics, errorMetrics, - loader, time, retryWithToleranceOperator, statusBackingStore, closeExecutor, errorReportersSupplier, pluginLoaderSwapper); + loader, time, retryWithToleranceOperator, statusBackingStore, closeExecutor, errorReportersSupplier, pluginsMetadata, pluginLoaderSwapper); this.transactionOpen = false; this.committableRecords = new LinkedHashMap<>(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskPluginsMetadata.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskPluginsMetadata.java new file mode 100644 index 0000000000000..14e6cb9b7a717 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskPluginsMetadata.java @@ -0,0 +1,131 @@ +/* + * 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.connect.runtime; + +import org.apache.kafka.connect.connector.Connector; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.runtime.isolation.PluginType; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorType; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; + +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +public class TaskPluginsMetadata { + + private final String connectorClass; + private final String connectorVersion; + private final ConnectorType connectorType; + private final String taskClass; + private final String taskVersion; + private final String keyConverterClass; + private final String keyConverterVersion; + private final String valueConverterClass; + private final String valueConverterVersion; + private final String headerConverterClass; + private final String headerConverterVersion; + private final Set transformations; + private final Set predicates; + + public TaskPluginsMetadata( + Class connectorClass, + Task task, + Converter keyConverter, + Converter valueConverter, + HeaderConverter headerConverter, + List transformationStageInfo, + Plugins plugins + ) { + + assert connectorClass != null; + assert task != null; + assert keyConverter != null; + assert valueConverter != null; + assert headerConverter != null; + assert transformationStageInfo != null; + + this.connectorClass = connectorClass.getName(); + this.connectorVersion = plugins.pluginVersion(connectorClass.getName(), connectorClass.getClassLoader(), PluginType.SINK, PluginType.SOURCE); + this.connectorType = ConnectorType.from(connectorClass); + this.taskClass = task.getClass().getName(); + this.taskVersion = task.version(); + this.keyConverterClass = keyConverter.getClass().getName(); + this.keyConverterVersion = plugins.pluginVersion(keyConverter.getClass().getName(), keyConverter.getClass().getClassLoader(), PluginType.CONVERTER); + this.valueConverterClass = valueConverter.getClass().getName(); + this.valueConverterVersion = plugins.pluginVersion(valueConverter.getClass().getName(), valueConverter.getClass().getClassLoader(), PluginType.CONVERTER); + this.headerConverterClass = headerConverter.getClass().getName(); + this.headerConverterVersion = plugins.pluginVersion(headerConverter.getClass().getName(), headerConverter.getClass().getClassLoader(), PluginType.HEADER_CONVERTER); + this.transformations = transformationStageInfo.stream().map(TransformationStage.StageInfo::transform).collect(Collectors.toSet()); + this.predicates = transformationStageInfo.stream().map(TransformationStage.StageInfo::predicate).filter(Objects::nonNull).collect(Collectors.toSet()); + } + + public String connectorClass() { + return connectorClass; + } + + public String connectorVersion() { + return connectorVersion; + } + + public ConnectorType connectorType() { + return connectorType; + } + + public String taskClass() { + return taskClass; + } + + public String taskVersion() { + return taskVersion; + } + + public String keyConverterClass() { + return keyConverterClass; + } + + public String keyConverterVersion() { + return keyConverterVersion; + } + + public String valueConverterClass() { + return valueConverterClass; + } + + public String valueConverterVersion() { + return valueConverterVersion; + } + + public String headerConverterClass() { + return headerConverterClass; + } + + public String headerConverterVersion() { + return headerConverterVersion; + } + + public Set transformations() { + return transformations; + } + + public Set predicates() { + return predicates; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java index e35efcafe2e91..45150ef7ef5a3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java @@ -20,8 +20,8 @@ public class TaskStatus extends AbstractStatus { - public TaskStatus(ConnectorTaskId id, State state, String workerUrl, int generation, String trace) { - super(id, state, workerUrl, generation, trace); + public TaskStatus(ConnectorTaskId id, State state, String workerUrl, int generation, String trace, String version) { + super(id, state, workerUrl, generation, trace, version); } public TaskStatus(ConnectorTaskId id, State state, String workerUrl, int generation) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java index f6b92697c443b..68d52f2c1ca3d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.Objects; import java.util.StringJoiner; +import java.util.stream.Collectors; /** * Represents a chain of {@link Transformation}s to be applied to a {@link ConnectRecord} serially. @@ -89,4 +90,8 @@ public String toString() { } return chain.toString(); } + + public List transformationChainInfo() { + return transformationStages.stream().map(TransformationStage::transformationStageInfo).collect(Collectors.toList()); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationStage.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationStage.java index a86c4878ab37e..56293e0363206 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationStage.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationStage.java @@ -24,6 +24,7 @@ import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.transforms.predicates.Predicate; +import java.util.Objects; import java.util.function.Function; /** @@ -39,18 +40,40 @@ public class TransformationStage> implements AutoClos private final Plugin> predicatePlugin; private final Plugin> transformationPlugin; private final boolean negate; + private final String transformAlias; + private final String predicateAlias; + private final String transformVersion; + private final String predicateVersion; private final Function pluginLoaderSwapper; - TransformationStage(Plugin> transformationPlugin, Function pluginLoaderSwapper) { - this(null, false, transformationPlugin, pluginLoaderSwapper); + TransformationStage( + Plugin> transformationPlugin, + String transformAlias, + String transformVersion, + Function pluginLoaderSwapper + ) { + this(null, null, null, false, transformationPlugin, transformAlias, transformVersion, pluginLoaderSwapper); } - TransformationStage(Plugin> predicatePlugin, boolean negate, Plugin> transformationPlugin, Function pluginLoaderSwapper) { + TransformationStage( + Plugin> predicatePlugin, + String predicateAlias, + String predicateVersion, + boolean negate, + Plugin> transformationPlugin, + String transformAlias, + String transformVersion, + Function pluginLoaderSwapper + ) { this.predicatePlugin = predicatePlugin; this.negate = negate; this.transformationPlugin = transformationPlugin; this.pluginLoaderSwapper = pluginLoaderSwapper; + this.transformAlias = transformAlias; + this.predicateAlias = predicateAlias; + this.transformVersion = transformVersion; + this.predicateVersion = predicateVersion; } public Class> transformClass() { @@ -89,4 +112,32 @@ public String toString() { ", negate=" + negate + '}'; } + + public record AliasedPluginInfo(String alias, String className, String version) { + public AliasedPluginInfo { + Objects.requireNonNull(alias, "alias cannot be null"); + Objects.requireNonNull(className, "className cannot be null"); + } + } + + + public record StageInfo(AliasedPluginInfo transform, AliasedPluginInfo predicate) { + public StageInfo { + Objects.requireNonNull(transform, "transform cannot be null"); + } + } + + + public StageInfo transformationStageInfo() { + AliasedPluginInfo transformInfo = new AliasedPluginInfo( + transformAlias, + transformationPlugin.get().getClass().getName(), + transformVersion + ); + AliasedPluginInfo predicateInfo = predicatePlugin != null ? new AliasedPluginInfo( + predicateAlias, + predicatePlugin.get().getClass().getName(), predicateVersion + ) : null; + return new StageInfo(transformInfo, predicateInfo); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 1c1acc5647e8e..a3e914d3f901e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -346,7 +346,7 @@ public void startConnector( } workerConnector = new WorkerConnector( connName, connector, connConfig, ctx, metrics, connectorStatusListener, offsetReader, offsetStore, connectorLoader); - log.info("Instantiated connector {} with version {} of type {}", connName, connector.version(), connector.getClass()); + log.info("Instantiated connector {} with version {} of type {}", connName, workerConnector.connectorVersion(), connector.getClass()); workerConnector.transitionTo(initialState, onConnectorStateChange); } } catch (Throwable t) { @@ -562,6 +562,22 @@ public boolean isRunning(String connName) { return workerConnector != null && workerConnector.isRunning(); } + public String connectorVersion(String connName) { + WorkerConnector conn = connectors.get(connName); + if (conn == null) { + return null; + } + return conn.connectorVersion(); + } + + public String taskVersion(ConnectorTaskId taskId) { + WorkerTask task = tasks.get(taskId); + if (task == null) { + return null; + } + return task.taskVersion(); + } + /** * Start a sink task managed by this worker. * @@ -714,7 +730,7 @@ private boolean startTask( .withKeyConverterPlugin(metrics.wrap(keyConverter, id, true)) .withValueConverterPlugin(metrics.wrap(valueConverter, id, false)) .withHeaderConverterPlugin(metrics.wrap(headerConverter, id)) - .withClassloader(connectorLoader) + .withClassLoader(connectorLoader) .build(); workerTask.initialize(taskConfig); @@ -1814,11 +1830,12 @@ public TaskBuilder withHeaderConverterPlugin(Plugin heade return this; } - public TaskBuilder withClassloader(ClassLoader classLoader) { + public TaskBuilder withClassLoader(ClassLoader classLoader) { this.classLoader = classLoader; return this; } + public WorkerTask build() { Objects.requireNonNull(task, "Task cannot be null"); Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null"); @@ -1836,10 +1853,13 @@ public WorkerTask build() { TransformationChain transformationChain = new TransformationChain<>(connectorConfig.transformationStages(plugins, id, metrics), retryWithToleranceOperator); log.info("Initializing: {}", transformationChain); + TaskPluginsMetadata taskPluginsMetadata = new TaskPluginsMetadata( + connectorClass, task, keyConverterPlugin.get(), valueConverterPlugin.get(), headerConverterPlugin.get(), transformationChain.transformationChainInfo(), plugins); + return doBuild(task, id, configState, statusListener, initialState, - connectorConfig, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, classLoader, - retryWithToleranceOperator, transformationChain, - errorHandlingMetrics, connectorClass); + connectorConfig, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, classLoader, + retryWithToleranceOperator, transformationChain, + errorHandlingMetrics, connectorClass, taskPluginsMetadata); } abstract WorkerTask doBuild( @@ -1856,7 +1876,8 @@ abstract WorkerTask doBuild( RetryWithToleranceOperator retryWithToleranceOperator, TransformationChain transformationChain, ErrorHandlingMetrics errorHandlingMetrics, - Class connectorClass + Class connectorClass, + TaskPluginsMetadata pluginsMetadata ); } @@ -1884,7 +1905,8 @@ public WorkerTask, SinkRecord> doBuild( RetryWithToleranceOperator> retryWithToleranceOperator, TransformationChain, SinkRecord> transformationChain, ErrorHandlingMetrics errorHandlingMetrics, - Class connectorClass + Class connectorClass, + TaskPluginsMetadata taskPluginsMetadata ) { SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings()); WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator, @@ -1898,7 +1920,7 @@ public WorkerTask, SinkRecord> doBuild( return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, transformationChain, consumer, classLoader, time, retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore(), - () -> sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass), plugins.safeLoaderSwapper()); + () -> sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass), taskPluginsMetadata, plugins.safeLoaderSwapper()); } } @@ -1925,7 +1947,8 @@ public WorkerTask doBuild( RetryWithToleranceOperator retryWithToleranceOperator, TransformationChain transformationChain, ErrorHandlingMetrics errorHandlingMetrics, - Class connectorClass + Class connectorClass, + TaskPluginsMetadata pluginsMetadata ) { SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig.originalsStrings(), config.topicCreationEnable()); @@ -1958,7 +1981,7 @@ public WorkerTask doBuild( return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, transformationChain, producer, topicAdmin, topicCreationGroups, offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time, - retryWithToleranceOperator, herder.statusBackingStore(), executor, () -> sourceTaskReporters(id, sourceConfig, errorHandlingMetrics), plugins.safeLoaderSwapper()); + retryWithToleranceOperator, herder.statusBackingStore(), executor, () -> sourceTaskReporters(id, sourceConfig, errorHandlingMetrics), pluginsMetadata, plugins.safeLoaderSwapper()); } } @@ -1992,7 +2015,8 @@ public WorkerTask doBuild( RetryWithToleranceOperator retryWithToleranceOperator, TransformationChain transformationChain, ErrorHandlingMetrics errorHandlingMetrics, - Class connectorClass + Class connectorClass, + TaskPluginsMetadata pluginsMetadata ) { SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, connectorConfig.originalsStrings(), config.topicCreationEnable()); @@ -2023,7 +2047,7 @@ public WorkerTask doBuild( headerConverterPlugin, transformationChain, producer, topicAdmin, topicCreationGroups, offsetReader, offsetWriter, offsetStore, config, configState, metrics, errorHandlingMetrics, classLoader, time, retryWithToleranceOperator, herder.statusBackingStore(), sourceConfig, executor, preProducerCheck, postProducerCheck, - () -> sourceTaskReporters(id, sourceConfig, errorHandlingMetrics), plugins.safeLoaderSwapper()); + () -> sourceTaskReporters(id, sourceConfig, errorHandlingMetrics), pluginsMetadata, plugins.safeLoaderSwapper()); } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java index e2473dbbf71e3..3faf70f898c7c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java @@ -78,6 +78,7 @@ private enum State { private volatile Throwable externalFailure; private volatile boolean stopping; // indicates whether the Worker has asked the connector to stop private volatile boolean cancelled; // indicates whether the Worker has cancelled the connector (e.g. because of slow shutdown) + private final String version; private State state; private final CloseableOffsetStorageReader offsetStorageReader; @@ -97,8 +98,9 @@ public WorkerConnector(String connName, this.loader = loader; this.ctx = ctx; this.connector = connector; + this.version = connector.version(); this.state = State.INIT; - this.metrics = new ConnectorMetricsGroup(connectMetrics, AbstractStatus.State.UNASSIGNED, statusListener); + this.metrics = new ConnectorMetricsGroup(connectMetrics, AbstractStatus.State.UNASSIGNED, this.version, statusListener); this.statusListener = this.metrics; this.offsetStorageReader = offsetStorageReader; this.offsetStore = offsetStore; @@ -418,6 +420,10 @@ public final boolean isSourceConnector() { return ConnectUtils.isSourceConnector(connector); } + public String connectorVersion() { + return version; + } + protected final String connectorType() { if (isSinkConnector()) return "sink"; @@ -450,7 +456,12 @@ class ConnectorMetricsGroup implements ConnectorStatus.Listener, AutoCloseable { private final MetricGroup metricGroup; private final ConnectorStatus.Listener delegate; - public ConnectorMetricsGroup(ConnectMetrics connectMetrics, AbstractStatus.State initialState, ConnectorStatus.Listener delegate) { + public ConnectorMetricsGroup( + ConnectMetrics connectMetrics, + AbstractStatus.State initialState, + String connectorVersion, + ConnectorStatus.Listener delegate + ) { Objects.requireNonNull(connectMetrics); Objects.requireNonNull(connector); Objects.requireNonNull(initialState); @@ -465,7 +476,7 @@ public ConnectorMetricsGroup(ConnectMetrics connectMetrics, AbstractStatus.State metricGroup.addImmutableValueMetric(registry.connectorType, connectorType()); metricGroup.addImmutableValueMetric(registry.connectorClass, connector.getClass().getName()); - metricGroup.addImmutableValueMetric(registry.connectorVersion, connector.version()); + metricGroup.addImmutableValueMetric(registry.connectorVersion, connectorVersion); metricGroup.addValueMetric(registry.connectorStatus, now -> state.toString().toLowerCase(Locale.getDefault())); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 4b8256115ed5d..14b093c91230b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -105,6 +105,7 @@ class WorkerSinkTask extends WorkerTask, SinkReco private boolean committing; private boolean taskStopped; private final WorkerErrantRecordReporter workerErrantRecordReporter; + private final String version; public WorkerSinkTask(ConnectorTaskId id, SinkTask task, @@ -125,9 +126,10 @@ public WorkerSinkTask(ConnectorTaskId id, WorkerErrantRecordReporter workerErrantRecordReporter, StatusBackingStore statusBackingStore, Supplier>>> errorReportersSupplier, + TaskPluginsMetadata pluginsMetadata, Function pluginLoaderSwapper) { super(id, statusListener, initialState, loader, connectMetrics, errorMetrics, - retryWithToleranceOperator, transformationChain, errorReportersSupplier, time, statusBackingStore, pluginLoaderSwapper); + retryWithToleranceOperator, transformationChain, errorReportersSupplier, time, statusBackingStore, pluginsMetadata, pluginLoaderSwapper); this.workerConfig = workerConfig; this.task = task; @@ -153,6 +155,7 @@ public WorkerSinkTask(ConnectorTaskId id, this.isTopicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG); this.taskStopped = false; this.workerErrantRecordReporter = workerErrantRecordReporter; + this.version = task.version(); } @Override @@ -227,6 +230,11 @@ public void execute() { } } + @Override + public String taskVersion() { + return version; + } + protected void iteration() { final long offsetCommitIntervalMs = workerConfig.getLong(WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 0806e8877355b..3ccd530be3900 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -94,12 +94,13 @@ public WorkerSourceTask(ConnectorTaskId id, StatusBackingStore statusBackingStore, Executor closeExecutor, Supplier>> errorReportersSupplier, + TaskPluginsMetadata pluginsMetadata, Function pluginLoaderSwapper) { super(id, task, statusListener, initialState, configState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, transformationChain, null, producer, admin, topicGroups, offsetReader, offsetWriter, offsetStore, workerConfig, connectMetrics, errorMetrics, loader, - time, retryWithToleranceOperator, statusBackingStore, closeExecutor, errorReportersSupplier, pluginLoaderSwapper); + time, retryWithToleranceOperator, statusBackingStore, closeExecutor, errorReportersSupplier, pluginsMetadata, pluginLoaderSwapper); this.committableOffsets = CommittableOffsets.EMPTY; this.submittedRecords = new SubmittedRecords(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java index fa28a4e7b0ea9..1661d710a8659 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java @@ -41,6 +41,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.List; import java.util.Locale; import java.util.concurrent.CountDownLatch; @@ -94,9 +95,10 @@ public WorkerTask(ConnectorTaskId id, Supplier>> errorReportersSupplier, Time time, StatusBackingStore statusBackingStore, + TaskPluginsMetadata pluginsMetadata, Function pluginLoaderSwapper) { this.id = id; - this.taskMetricsGroup = new TaskMetricsGroup(this.id, connectMetrics, statusListener); + this.taskMetricsGroup = new TaskMetricsGroup(this.id, connectMetrics, statusListener, pluginsMetadata); this.errorMetrics = errorMetrics; this.statusListener = taskMetricsGroup; this.loader = loader; @@ -196,6 +198,8 @@ void doStart() { protected abstract void close(); + protected abstract String taskVersion(); + protected boolean isFailed() { return failed; } @@ -397,14 +401,25 @@ TaskMetricsGroup taskMetricsGroup() { static class TaskMetricsGroup implements TaskStatus.Listener { private final TaskStatus.Listener delegateListener; private final MetricGroup metricGroup; + private final List transformationGroups = new ArrayList<>(); + private final List predicateGroups = new ArrayList<>(); private final Time time; private final StateTracker taskStateTimer; private final Sensor commitTime; private final Sensor batchSize; private final Sensor commitAttempts; + private final ConnectMetrics connectMetrics; + private final ConnectorTaskId id; public TaskMetricsGroup(ConnectorTaskId id, ConnectMetrics connectMetrics, TaskStatus.Listener statusListener) { + this(id, connectMetrics, statusListener, null); + } + + public TaskMetricsGroup(ConnectorTaskId id, ConnectMetrics connectMetrics, TaskStatus.Listener statusListener, TaskPluginsMetadata pluginsMetadata) { delegateListener = statusListener; + this.connectMetrics = connectMetrics; + this.id = id; + time = connectMetrics.time(); taskStateTimer = new StateTracker(); ConnectMetricsRegistry registry = connectMetrics.registry(); @@ -434,6 +449,7 @@ public TaskMetricsGroup(ConnectorTaskId id, ConnectMetrics connectMetrics, TaskS Frequencies commitFrequencies = Frequencies.forBooleanValues(offsetCommitFailures, offsetCommitSucceeds); commitAttempts = metricGroup.sensor("offset-commit-completion"); commitAttempts.add(commitFrequencies); + addPluginInfoMetric(pluginsMetadata); } private void addRatioMetric(final State matchingState, MetricNameTemplate template) { @@ -442,8 +458,52 @@ private void addRatioMetric(final State matchingState, MetricNameTemplate templa taskStateTimer.durationRatio(matchingState, now)); } + private void addPluginInfoMetric(TaskPluginsMetadata pluginsMetadata) { + if (pluginsMetadata == null) { + return; + } + ConnectMetricsRegistry registry = connectMetrics.registry(); + metricGroup.addValueMetric(registry.taskConnectorClass, now -> pluginsMetadata.connectorClass()); + metricGroup.addValueMetric(registry.taskConnectorClassVersion, now -> pluginsMetadata.connectorVersion()); + metricGroup.addValueMetric(registry.taskConnectorType, now -> pluginsMetadata.connectorType()); + metricGroup.addValueMetric(registry.taskClass, now -> pluginsMetadata.taskClass()); + metricGroup.addValueMetric(registry.taskVersion, now -> pluginsMetadata.taskVersion()); + metricGroup.addValueMetric(registry.taskKeyConverterClass, now -> pluginsMetadata.keyConverterClass()); + metricGroup.addValueMetric(registry.taskKeyConverterVersion, now -> pluginsMetadata.keyConverterVersion()); + metricGroup.addValueMetric(registry.taskValueConverterClass, now -> pluginsMetadata.valueConverterClass()); + metricGroup.addValueMetric(registry.taskValueConverterVersion, now -> pluginsMetadata.valueConverterVersion()); + metricGroup.addValueMetric(registry.taskHeaderConverterClass, now -> pluginsMetadata.headerConverterClass()); + metricGroup.addValueMetric(registry.taskHeaderConverterVersion, now -> pluginsMetadata.headerConverterVersion()); + + if (!pluginsMetadata.transformations().isEmpty()) { + for (TransformationStage.AliasedPluginInfo entry : pluginsMetadata.transformations()) { + MetricGroup transformationGroup = connectMetrics.group(registry.transformsGroupName(), + registry.connectorTagName(), id.connector(), + registry.taskTagName(), Integer.toString(id.task()), + registry.transformsTagName(), entry.alias()); + transformationGroup.addValueMetric(registry.transformClass, now -> entry.className()); + transformationGroup.addValueMetric(registry.transformVersion, now -> entry.version()); + this.transformationGroups.add(transformationGroup); + } + } + + if (!pluginsMetadata.predicates().isEmpty()) { + for (TransformationStage.AliasedPluginInfo entry : pluginsMetadata.predicates()) { + MetricGroup predicateGroup = connectMetrics.group(registry.predicatesGroupName(), + registry.connectorTagName(), id.connector(), + registry.taskTagName(), Integer.toString(id.task()), + registry.predicateTagName(), entry.alias()); + predicateGroup.addValueMetric(registry.predicateClass, now -> entry.className()); + predicateGroup.addValueMetric(registry.predicateVersion, now -> entry.version()); + this.predicateGroups.add(predicateGroup); + } + } + } + void close() { metricGroup.close(); + transformationGroups.forEach(MetricGroup::close); + predicateGroups.forEach(MetricGroup::close); } void recordCommit(long duration, boolean success) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java index 82d9957b40db1..8d94d56204534 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java @@ -65,11 +65,13 @@ public abstract static class AbstractState { private final String state; private final String trace; private final String workerId; + private final String version; - public AbstractState(String state, String workerId, String trace) { + public AbstractState(String state, String workerId, String trace, String version) { this.state = state; this.workerId = workerId; this.trace = trace; + this.version = version; } @JsonProperty @@ -87,14 +89,22 @@ public String workerId() { public String trace() { return trace; } + + @JsonProperty + @JsonInclude(value = JsonInclude.Include.CUSTOM, valueFilter = PluginInfo.NoVersionFilter.class) + public String version() { + return version; + } } public static class ConnectorState extends AbstractState { + @JsonCreator public ConnectorState(@JsonProperty("state") String state, @JsonProperty("worker_id") String worker, - @JsonProperty("msg") String msg) { - super(state, worker, msg); + @JsonProperty("msg") String msg, + @JsonProperty("version") String version) { + super(state, worker, msg, version); } } @@ -105,8 +115,9 @@ public static class TaskState extends AbstractState implements Comparable status) { struct.put(TRACE_KEY_NAME, status.trace()); struct.put(WORKER_ID_KEY_NAME, status.workerId()); struct.put(GENERATION_KEY_NAME, status.generation()); + struct.put(VERSION_KEY_NAME, status.version()); return converter.fromConnectData(statusTopic, STATUS_SCHEMA_V0, struct); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java index d9776e05dd3db..b5cdea37e7398 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java @@ -87,14 +87,18 @@ public Map, Map> offsets(Collection raw; try { Future> offsetReadFuture; + + // Note: this call can block for long time waiting for data flush to complete (`KafkaProducer.flush()`). + offsetReadFuture = backingStore.get(serializedToOriginal.keySet()); + synchronized (offsetReadFutures) { if (closed.get()) { + offsetReadFuture.cancel(true); throw new ConnectException( "Offset reader is closed. This is likely because the task has already been " + "scheduled to stop but has taken longer than the graceful shutdown " + "period to do so."); } - offsetReadFuture = backingStore.get(serializedToOriginal.keySet()); offsetReadFutures.add(offsetReadFuture); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java index 9ad8690ca1c69..97b8edf7605f7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java @@ -964,7 +964,7 @@ private void createWorkerTask(Converter keyConverter, Converter valueConverter, taskId, sourceTask, statusListener, TargetState.STARTED, configState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, transformationChain, workerTransactionContext, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore, config, metrics, errorHandlingMetrics, plugins.delegatingLoader(), Time.SYSTEM, retryWithToleranceOperator, - statusBackingStore, Runnable::run, errorReportersSupplier, TestPlugins.noOpLoaderSwap()) { + statusBackingStore, Runnable::run, errorReportersSupplier, null, TestPlugins.noOpLoaderSwap()) { @Override protected void prepareToInitializeTask() { } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index 70edfb0f59877..340585f0fd66e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -426,9 +426,9 @@ private void createSinkTask(TargetState initialState, RetryWithToleranceOperator oo.put("schemas.enable", "false"); converter.configure(oo); - Plugin> transformationPlugin = metrics.wrap(new FaultyPassthrough(), taskId, ""); + Plugin> transformationPlugin = metrics.wrap(new FaultyPassthrough(), taskId, "test"); TransformationChain, SinkRecord> sinkTransforms = - new TransformationChain<>(singletonList(new TransformationStage<>(transformationPlugin, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator); + new TransformationChain<>(singletonList(new TransformationStage<>(transformationPlugin, "test", null, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator); Plugin keyConverterPlugin = metrics.wrap(converter, taskId, true); Plugin valueConverterPlugin = metrics.wrap(converter, taskId, false); @@ -438,7 +438,7 @@ private void createSinkTask(TargetState initialState, RetryWithToleranceOperator ClusterConfigState.EMPTY, metrics, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, sinkTransforms, consumer, pluginLoader, time, retryWithToleranceOperator, workerErrantRecordReporter, - statusBackingStore, () -> errorReporters, TestPlugins.noOpLoaderSwap()); + statusBackingStore, () -> errorReporters, null, TestPlugins.noOpLoaderSwap()); } private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator, List> errorReporters) { @@ -462,9 +462,9 @@ private Converter badConverter() { private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator, List> errorReporters, Converter converter) { - Plugin> transformationPlugin = metrics.wrap(new FaultyPassthrough(), taskId, ""); + Plugin> transformationPlugin = metrics.wrap(new FaultyPassthrough(), taskId, "test"); TransformationChain sourceTransforms = new TransformationChain<>(singletonList( - new TransformationStage<>(transformationPlugin, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator); + new TransformationStage<>(transformationPlugin, "test", null, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator); Plugin keyConverterPlugin = metrics.wrap(converter, taskId, true); Plugin valueConverterPlugin = metrics.wrap(converter, taskId, false); @@ -477,7 +477,7 @@ private void createSourceTask(TargetState initialState, RetryWithToleranceOperat offsetReader, offsetWriter, offsetStore, workerConfig, ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, - statusBackingStore, Runnable::run, () -> errorReporters, TestPlugins.noOpLoaderSwap())); + statusBackingStore, Runnable::run, () -> errorReporters, null, TestPlugins.noOpLoaderSwap())); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java index a6375398d292b..f45ec27b46e2d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java @@ -198,6 +198,7 @@ public void setup(boolean enableTopicCreation) throws Exception { Thread.sleep(10); return result; }); + when(sourceTask.version()).thenReturn(null); } @AfterEach @@ -222,8 +223,8 @@ public void teardown() throws Exception { } verify(statusBackingStore, MockitoUtils.anyTimes()).getTopic(any(), any()); - verify(offsetStore, MockitoUtils.anyTimes()).primaryOffsetsTopic(); + verify(sourceTask).version(); verifyNoMoreInteractions(statusListener, producer, sourceTask, admin, offsetWriter, statusBackingStore, offsetStore, preProducerCheck, postProducerCheck); if (metrics != null) metrics.stop(); @@ -284,7 +285,7 @@ private void createWorkerTask(TargetState initialState, Converter keyConverter, workerTask = new ExactlyOnceWorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore, config, clusterConfigState, metrics, errorHandlingMetrics, plugins.delegatingLoader(), time, RetryWithToleranceOperatorTest.noneOperator(), statusBackingStore, - sourceConfig, Runnable::run, preProducerCheck, postProducerCheck, Collections::emptyList, TestPlugins.noOpLoaderSwap()); + sourceConfig, Runnable::run, preProducerCheck, postProducerCheck, Collections::emptyList, null, TestPlugins.noOpLoaderSwap()); } @ParameterizedTest diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/RestartPlanTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/RestartPlanTest.java index 8d6f54ce2581b..d0f3f974c635f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/RestartPlanTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/RestartPlanTest.java @@ -35,17 +35,15 @@ public class RestartPlanTest { @Test public void testRestartPlan() { ConnectorStateInfo.ConnectorState state = new ConnectorStateInfo.ConnectorState( - AbstractStatus.State.RESTARTING.name(), - "foo", - null + AbstractStatus.State.RESTARTING.name(), "foo", null, null ); List tasks = new ArrayList<>(); - tasks.add(new TaskState(1, AbstractStatus.State.RUNNING.name(), "worker1", null)); - tasks.add(new TaskState(2, AbstractStatus.State.PAUSED.name(), "worker1", null)); - tasks.add(new TaskState(3, AbstractStatus.State.RESTARTING.name(), "worker1", null)); - tasks.add(new TaskState(4, AbstractStatus.State.DESTROYED.name(), "worker1", null)); - tasks.add(new TaskState(5, AbstractStatus.State.RUNNING.name(), "worker1", null)); - tasks.add(new TaskState(6, AbstractStatus.State.RUNNING.name(), "worker1", null)); + tasks.add(new TaskState(1, AbstractStatus.State.RUNNING.name(), "worker1", null, null)); + tasks.add(new TaskState(2, AbstractStatus.State.PAUSED.name(), "worker1", null, null)); + tasks.add(new TaskState(3, AbstractStatus.State.RESTARTING.name(), "worker1", null, null)); + tasks.add(new TaskState(4, AbstractStatus.State.DESTROYED.name(), "worker1", null, null)); + tasks.add(new TaskState(5, AbstractStatus.State.RUNNING.name(), "worker1", null, null)); + tasks.add(new TaskState(6, AbstractStatus.State.RUNNING.name(), "worker1", null, null)); ConnectorStateInfo connectorStateInfo = new ConnectorStateInfo(CONNECTOR_NAME, state, tasks, ConnectorType.SOURCE); RestartRequest restartRequest = new RestartRequest(CONNECTOR_NAME, false, true); @@ -61,13 +59,11 @@ public void testRestartPlan() { @Test public void testNoRestartsPlan() { ConnectorStateInfo.ConnectorState state = new ConnectorStateInfo.ConnectorState( - AbstractStatus.State.RUNNING.name(), - "foo", - null + AbstractStatus.State.RUNNING.name(), "foo", null, null ); List tasks = new ArrayList<>(); - tasks.add(new TaskState(1, AbstractStatus.State.RUNNING.name(), "worker1", null)); - tasks.add(new TaskState(2, AbstractStatus.State.PAUSED.name(), "worker1", null)); + tasks.add(new TaskState(1, AbstractStatus.State.RUNNING.name(), "worker1", null, null)); + tasks.add(new TaskState(2, AbstractStatus.State.PAUSED.name(), "worker1", null, null)); ConnectorStateInfo connectorStateInfo = new ConnectorStateInfo(CONNECTOR_NAME, state, tasks, ConnectorType.SOURCE); RestartRequest restartRequest = new RestartRequest(CONNECTOR_NAME, false, true); RestartPlan restartPlan = new RestartPlan(restartRequest, connectorStateInfo); @@ -81,13 +77,11 @@ public void testNoRestartsPlan() { @Test public void testRestartsOnlyConnector() { ConnectorStateInfo.ConnectorState state = new ConnectorStateInfo.ConnectorState( - AbstractStatus.State.RESTARTING.name(), - "foo", - null + AbstractStatus.State.RESTARTING.name(), "foo", null, null ); List tasks = new ArrayList<>(); - tasks.add(new TaskState(1, AbstractStatus.State.RUNNING.name(), "worker1", null)); - tasks.add(new TaskState(2, AbstractStatus.State.PAUSED.name(), "worker1", null)); + tasks.add(new TaskState(1, AbstractStatus.State.RUNNING.name(), "worker1", null, null)); + tasks.add(new TaskState(2, AbstractStatus.State.PAUSED.name(), "worker1", null, null)); ConnectorStateInfo connectorStateInfo = new ConnectorStateInfo(CONNECTOR_NAME, state, tasks, ConnectorType.SOURCE); RestartRequest restartRequest = new RestartRequest(CONNECTOR_NAME, false, true); RestartPlan restartPlan = new RestartPlan(restartRequest, connectorStateInfo); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TransformationStageTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TransformationStageTest.java index e2791a63f7b26..959c5d2ac01f8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TransformationStageTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/TransformationStageTest.java @@ -61,8 +61,12 @@ private void applyAndAssert(boolean predicateResult, boolean negate, SourceRecor } TransformationStage stage = new TransformationStage<>( predicatePlugin, + "testPredicate", + null, negate, transformationPlugin, + "testTransformation", + null, TestPlugins.noOpLoaderSwap() ); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index 2607ee8b03b3c..539960badec03 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -229,7 +229,7 @@ private void createTask(ConnectorTaskId taskId, SinkTask task, TaskStatus.Listen taskId, task, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, connectMetrics, keyConverterPlugin, valueConverterPlugin, errorMetrics, headerConverterPlugin, transformationChain, consumer, loader, time, - retryWithToleranceOperator, null, statusBackingStore, errorReportersSupplier, TestPlugins.noOpLoaderSwap()); + retryWithToleranceOperator, null, statusBackingStore, errorReportersSupplier, null, TestPlugins.noOpLoaderSwap()); } @AfterEach diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 740211180984a..6c2c593c35b8a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -183,7 +183,7 @@ public void setup() { taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, transformationChain, consumer, pluginLoader, time, RetryWithToleranceOperatorTest.noneOperator(), null, statusBackingStore, - Collections::emptyList, TestPlugins.noOpLoaderSwap()); + Collections::emptyList, null, TestPlugins.noOpLoaderSwap()); recordsReturned = 0; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 23fb3618f8191..d2fd923fdb93b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -255,7 +255,7 @@ private void createWorkerTask(TargetState initialState, Converter keyConverter, workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, - retryWithToleranceOperator, statusBackingStore, Runnable::run, Collections::emptyList, TestPlugins.noOpLoaderSwap()); + retryWithToleranceOperator, statusBackingStore, Runnable::run, Collections::emptyList, null, TestPlugins.noOpLoaderSwap()); } @ParameterizedTest diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java index eae9c96998b2c..fa445454fd088 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java @@ -300,7 +300,7 @@ public TestWorkerTask(ConnectorTaskId id, Listener statusListener, TargetState i Supplier>> errorReporterSupplier, Time time, StatusBackingStore statusBackingStore) { super(id, statusListener, initialState, loader, connectMetrics, errorHandlingMetrics, - retryWithToleranceOperator, transformationChain, errorReporterSupplier, time, statusBackingStore, TestPlugins.noOpLoaderSwap()); + retryWithToleranceOperator, transformationChain, errorReporterSupplier, time, statusBackingStore, null, TestPlugins.noOpLoaderSwap()); } @Override @@ -318,6 +318,11 @@ protected void execute() { @Override protected void close() { } + + @Override + protected String taskVersion() { + return null; + } } protected void assertFailedMetric(TaskMetricsGroup metricsGroup) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 71dcabbedb669..38bcab1b59469 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -63,6 +63,7 @@ import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.isolation.Plugins.ClassLoaderUsage; +import org.apache.kafka.connect.runtime.isolation.TestPlugins; import org.apache.kafka.connect.runtime.rest.RestServer; import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; @@ -805,11 +806,11 @@ public void testTaskStatusMetricsStatuses(boolean enableTopicCreation) { // Each time we check the task metrics, the worker will call the herder when(herder.taskStatus(TASK_ID)).thenReturn( - new ConnectorStateInfo.TaskState(0, "RUNNING", "worker", "msg"), - new ConnectorStateInfo.TaskState(0, "PAUSED", "worker", "msg"), - new ConnectorStateInfo.TaskState(0, "FAILED", "worker", "msg"), - new ConnectorStateInfo.TaskState(0, "DESTROYED", "worker", "msg"), - new ConnectorStateInfo.TaskState(0, "UNASSIGNED", "worker", "msg") + new ConnectorStateInfo.TaskState(0, "RUNNING", "worker", "msg", null), + new ConnectorStateInfo.TaskState(0, "PAUSED", "worker", "msg", null), + new ConnectorStateInfo.TaskState(0, "FAILED", "worker", "msg", null), + new ConnectorStateInfo.TaskState(0, "DESTROYED", "worker", "msg", null), + new ConnectorStateInfo.TaskState(0, "UNASSIGNED", "worker", "msg", null) ); worker = new Worker(WORKER_ID, @@ -3072,6 +3073,7 @@ private void mockVersionedTaskIsolation(Class connectorClas when(plugins.pluginLoader(connectorClass.getName(), range)).thenReturn(pluginLoader); when(plugins.connectorClass(connectorClass.getName(), range)).thenReturn((Class) connectorClass); when(plugins.newTask(taskClass)).thenReturn(task); + when(plugins.safeLoaderSwapper()).thenReturn(TestPlugins.noOpLoaderSwap()); when(task.version()).thenReturn(range == null ? "unknown" : range.toString()); } @@ -3087,7 +3089,7 @@ private void verifyVersionedTaskIsolation(Class connectorCl verify(plugins).pluginLoader(connectorClass.getName(), range); verify(plugins).connectorClass(connectorClass.getName(), range); verify(plugins).newTask(taskClass); - verify(task).version(); + verify(task, times(2)).version(); } private void mockExecutorRealSubmit(Class runnableClass) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java index 3574137b6c301..3e1e75480b5ad 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTestUtils.java @@ -195,8 +195,12 @@ public static > TransformationChain buildTra when(transformationPlugin.get()).thenReturn(transformation); TransformationStage stage = new TransformationStage<>( predicatePlugin, + "testPredicate", + null, false, transformationPlugin, + "testTransformation", + null, TestPlugins.noOpLoaderSwap()); TransformationChain realTransformationChainRetriableException = new TransformationChain<>(List.of(stage), toleranceOperator); return Mockito.spy(realTransformationChainRetriableException); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index 73eaf93961e41..18589d66855a5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -1348,6 +1348,7 @@ public void testDoRestartConnectorAndTasksOnlyConnector() { return true; }).when(worker).startConnector(eq(CONN1), any(), any(), eq(herder), any(), stateCallback.capture()); doNothing().when(member).wakeup(); + when(worker.connectorVersion(any())).thenReturn(null); herder.doRestartConnectorAndTasks(restartRequest); @@ -1378,6 +1379,7 @@ public void testDoRestartConnectorAndTasksOnlyTasks() { doNothing().when(statusBackingStore).put(eq(status)); when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), any())).thenReturn(true); + when(worker.taskVersion(any())).thenReturn(null); herder.doRestartConnectorAndTasks(restartRequest); @@ -1419,6 +1421,8 @@ public void testDoRestartConnectorAndTasksBoth() { doNothing().when(statusBackingStore).put(eq(taskStatus)); when(worker.startSourceTask(eq(TASK0), any(), any(), any(), eq(herder), any())).thenReturn(true); + when(worker.taskVersion(any())).thenReturn(null); + when(worker.connectorVersion(any())).thenReturn(null); herder.doRestartConnectorAndTasks(restartRequest); @@ -1670,6 +1674,7 @@ public void testConnectorConfigUpdateFailedTransformation() { when(member.memberId()).thenReturn("member"); when(member.currentProtocolVersion()).thenReturn(CONNECT_PROTOCOL_V0); when(worker.isSinkConnector(CONN1)).thenReturn(Boolean.TRUE); + when(worker.connectorVersion(CONN1)).thenReturn(null); WorkerConfigTransformer configTransformer = mock(WorkerConfigTransformer.class); // join diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index 9dfead77220f6..e38cd2da60da6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -688,9 +688,7 @@ public void testRestartConnectorAndTasksRebalanceNeeded() { @Test public void testRestartConnectorAndTasksRequestAccepted() throws Throwable { ConnectorStateInfo.ConnectorState state = new ConnectorStateInfo.ConnectorState( - AbstractStatus.State.RESTARTING.name(), - "foo", - null + AbstractStatus.State.RESTARTING.name(), "foo", null, null ); ConnectorStateInfo connectorStateInfo = new ConnectorStateInfo(CONNECTOR_NAME, state, Collections.emptyList(), ConnectorType.SOURCE); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 9e893e79eba0e..76b25131cc590 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -280,6 +280,7 @@ public void testDestroyConnector() throws Exception { expectConfigValidation(SourceSink.SOURCE, config); when(statusBackingStore.getAll(CONNECTOR_NAME)).thenReturn(Collections.emptyList()); + when(worker.connectorVersion(CONNECTOR_NAME)).thenReturn(null); herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback); Herder.Created connectorInfo = createCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS); @@ -533,6 +534,7 @@ public void testRestartConnectorAndTasksOnlyConnector() throws Exception { expectConfigValidation(SourceSink.SINK, connectorConfig); doNothing().when(worker).stopAndAwaitConnector(CONNECTOR_NAME); + when(worker.connectorVersion(CONNECTOR_NAME)).thenReturn(null); mockStartConnector(connectorConfig, null, TargetState.STARTED, null); @@ -563,6 +565,7 @@ public void testRestartConnectorAndTasksOnlyTasks() throws Exception { doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); expectAdd(SourceSink.SINK); + when(worker.taskVersion(any())).thenReturn(null); Map connectorConfig = connectorConfig(SourceSink.SINK); expectConfigValidation(SourceSink.SINK, connectorConfig); @@ -616,6 +619,8 @@ public void testRestartConnectorAndTasksBoth() throws Exception { ArgumentCaptor taskStatus = ArgumentCaptor.forClass(TaskStatus.class); expectAdd(SourceSink.SINK, false); + when(worker.connectorVersion(any())).thenReturn(null); + when(worker.taskVersion(any())).thenReturn(null); Map connectorConfig = connectorConfig(SourceSink.SINK); expectConfigValidation(SourceSink.SINK, connectorConfig); @@ -1124,6 +1129,7 @@ private void expectAdd(SourceSink sourceSink, } when(worker.isRunning(CONNECTOR_NAME)).thenReturn(true); + if (sourceSink == SourceSink.SOURCE) { when(worker.isTopicCreationEnabled()).thenReturn(true); } @@ -1152,6 +1158,7 @@ private void expectAdd(SourceSink sourceSink, transformer); if (sourceSink.equals(SourceSink.SOURCE) && mockStartSourceTask) { + when(worker.taskVersion(any())).thenReturn(null); when(worker.startSourceTask(new ConnectorTaskId(CONNECTOR_NAME, 0), configState, connectorConfig(sourceSink), generatedTaskProps, herder, TargetState.STARTED)).thenReturn(true); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageReaderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageReaderTest.java new file mode 100644 index 0000000000000..01646c1323f93 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/OffsetStorageReaderTest.java @@ -0,0 +1,171 @@ +/* + * 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.connect.storage; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; +import org.mockito.stubbing.Answer; + +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.STRICT_STUBS) +public class OffsetStorageReaderTest { + + @Mock + private Converter taskKeyConverter; + + @Mock + private Converter taskValueConverter; + + @Mock + private OffsetBackingStore offsetBackingStore; + + @Test + @Timeout(60) + public void testClosingOffsetReaderWhenOffsetStoreHangs() throws Exception { + ExecutorService executor = Executors.newSingleThreadExecutor(); + OffsetStorageReaderImpl offsetStorageReaderImpl = new OffsetStorageReaderImpl( + offsetBackingStore, "namespace", taskKeyConverter, taskValueConverter); + + CountDownLatch latch = new CountDownLatch(1); + + // Hanging `offsetBackingStore.get()` + doAnswer(invocation -> { + latch.countDown(); + CompletableFuture future = new CompletableFuture<>(); + future.get(9999, TimeUnit.SECONDS); + throw new RuntimeException("Should never get here"); + }).when(offsetBackingStore).get(any()); + + // Connector task thread hanging + executor.submit(() -> { + // Does call offsetBackingStore.get() and hangs + offsetStorageReaderImpl.offsets(Collections.emptyList()); + }); + + // Ensure the task is hanging + latch.await(); + + verify(offsetBackingStore, times(1)).get(any()); + + // The herder thread should not block when trying to close `offsetStorageReaderImpl` + // and complete before test timeout + offsetStorageReaderImpl.close(); + + executor.shutdownNow(); + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + throw new RuntimeException("Failed to shutdown executor"); + } + } + + @Test + @Timeout(60) + public void testClosingOffsetReaderWhenOffsetStoreHangsAndHasIncompleteFutures() throws Exception { + // Test similar to `testClosingOffsetReaderWhenOffsetStoreHangs` above, but in this case + // `OffsetStorageReaderImpl.offsetReadFutures` contains a future when `offsetStorageReaderImpl.close()` is called. + + ExecutorService executor = Executors.newFixedThreadPool(2); + CompletableFuture hangingFuture = mock(CompletableFuture.class); + + OffsetStorageReaderImpl offsetStorageReaderImpl = new OffsetStorageReaderImpl( + offsetBackingStore, "namespace", taskKeyConverter, taskValueConverter); + + CountDownLatch latchTask1 = new CountDownLatch(1); + CountDownLatch latchTask2 = new CountDownLatch(1); + + // Mock hanging future + doAnswer(invocation -> { + CompletableFuture future = new CompletableFuture<>(); + future.get(9999, TimeUnit.SECONDS); + throw new RuntimeException("Should never get here"); + }).when(hangingFuture).get(); + + // Mock `offsetBackingStore.get()` + doAnswer(new Answer() { + int callCount = 0; + + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + if (callCount == 0) { + callCount += 1; + // First connector task + latchTask1.countDown(); + return hangingFuture; + } else { + // Second connector task + latchTask2.countDown(); + CompletableFuture future = new CompletableFuture<>(); + future.get(9999, TimeUnit.SECONDS); + throw new RuntimeException("Should never get here"); + } + } + } + ).when(offsetBackingStore).get(any()); + + + // Connector task thread calls `offsets()` --> hangs on `hangingFuture.get()` + // --> the future is added to `offsetStorageReaderImpl.offsetReadFutures` and never removed + executor.submit(() -> { + offsetStorageReaderImpl.offsets(Collections.emptyList()); + }); + // Ensure first task is hanging + latchTask1.await(); + + verify(offsetBackingStore, times(1)).get(any()); + verify(hangingFuture, times(1)).get(); + + // Another connector task thread calls `offsets()` --> hangs on offsetBackingStore.get() + // --> the future is never added to `offsetStorageReaderImpl.offsetReadFutures` + executor.submit(() -> { + offsetStorageReaderImpl.offsets(Collections.emptyList()); + }); + // Ensure second task is hanging + latchTask2.await(); + + verify(offsetBackingStore, times(2)).get(any()); + + // The herder thread should not block when trying to close `offsetStorageReaderImpl` and should complete + // before the test timeout + offsetStorageReaderImpl.close(); + + // The hanging future should be cancelled by `close()` + verify(hangingFuture, times(1)).cancel(true); + + executor.shutdownNow(); + if (!executor.awaitTermination(5, TimeUnit.SECONDS)) { + throw new RuntimeException("Failed to shutdown executor"); + } + } +} diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index d68ed06d3070d..5bd5f4ea6aa55 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -22,6 +22,10 @@ import kafka.server.ReplicaManager; import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.errors.KafkaStorageException; +import org.apache.kafka.common.errors.NotLeaderOrFollowerException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; @@ -34,8 +38,12 @@ import org.apache.kafka.server.share.fetch.ShareFetchPartitionData; import org.apache.kafka.server.share.metrics.ShareGroupMetrics; import org.apache.kafka.server.storage.log.FetchIsolation; +import org.apache.kafka.server.storage.log.FetchPartitionData; +import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.LogOffsetSnapshot; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; import com.yammer.metrics.core.Meter; @@ -44,10 +52,16 @@ import java.util.ArrayList; import java.util.LinkedHashMap; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; import java.util.function.BiConsumer; @@ -83,7 +97,9 @@ public class DelayedShareFetch extends DelayedOperation { // Tracks the start time to acquire any share partition for a fetch request. private long acquireStartTimeMs; private LinkedHashMap partitionsAcquired; - private LinkedHashMap partitionsAlreadyFetched; + private LinkedHashMap localPartitionsAlreadyFetched; + private Optional remoteFetchOpt; + private Optional remoteStorageFetchException; /** * This function constructs an instance of delayed share fetch operation for completing share fetch @@ -110,10 +126,24 @@ public DelayedShareFetch( sharePartitions, PartitionMaxBytesStrategy.type(PartitionMaxBytesStrategy.StrategyType.UNIFORM), shareGroupMetrics, - time + time, + Optional.empty() ); } + /** + * This function constructs an instance of delayed share fetch operation for completing share fetch + * requests instantaneously or with delay. The direct usage of this constructor is only from tests. + * + * @param shareFetch The share fetch parameters of the share fetch request. + * @param replicaManager The replica manager instance used to read from log/complete the request. + * @param exceptionHandler The handler to complete share fetch requests with exception. + * @param sharePartitions The share partitions referenced in the share fetch request. + * @param partitionMaxBytesStrategy The strategy to identify the max bytes for topic partitions in the share fetch request. + * @param shareGroupMetrics The share group metrics to record the metrics. + * @param time The system time. + * @param remoteFetchOpt Optional containing an in-flight remote fetch object or an empty optional. + */ DelayedShareFetch( ShareFetch shareFetch, ReplicaManager replicaManager, @@ -121,19 +151,22 @@ public DelayedShareFetch( LinkedHashMap sharePartitions, PartitionMaxBytesStrategy partitionMaxBytesStrategy, ShareGroupMetrics shareGroupMetrics, - Time time + Time time, + Optional remoteFetchOpt ) { super(shareFetch.fetchParams().maxWaitMs, Optional.empty()); this.shareFetch = shareFetch; this.replicaManager = replicaManager; this.partitionsAcquired = new LinkedHashMap<>(); - this.partitionsAlreadyFetched = new LinkedHashMap<>(); + this.localPartitionsAlreadyFetched = new LinkedHashMap<>(); this.exceptionHandler = exceptionHandler; this.sharePartitions = sharePartitions; this.partitionMaxBytesStrategy = partitionMaxBytesStrategy; this.shareGroupMetrics = shareGroupMetrics; this.time = time; this.acquireStartTimeMs = time.hiResClockMs(); + this.remoteFetchOpt = remoteFetchOpt; + this.remoteStorageFetchException = Optional.empty(); // Register metrics for DelayedShareFetch. KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "DelayedShareFetchMetrics"); this.expiredRequestMeter = metricsGroup.newMeter(EXPIRES_PER_SEC, "requests", TimeUnit.SECONDS); @@ -152,58 +185,68 @@ public void onExpiration() { @Override public void onComplete() { // We are utilizing lock so that onComplete doesn't do a dirty read for instance variables - - // partitionsAcquired and partitionsAlreadyFetched, since these variables can get updated in a different tryComplete thread. + // partitionsAcquired and localPartitionsAlreadyFetched, since these variables can get updated in a different tryComplete thread. lock.lock(); log.trace("Completing the delayed share fetch request for group {}, member {}, " + "topic partitions {}", shareFetch.groupId(), shareFetch.memberId(), partitionsAcquired.keySet()); try { - LinkedHashMap topicPartitionData; - // tryComplete did not invoke forceComplete, so we need to check if we have any partitions to fetch. - if (partitionsAcquired.isEmpty()) { - topicPartitionData = acquirablePartitions(); - // The TopicPartitionsAcquireTimeMs metric signifies the tension when acquiring the locks - // for the share partition, hence if no partitions are yet acquired by tryComplete, - // we record the metric here. Do not check if the request has successfully acquired any - // partitions now or not, as then the upper bound of request timeout shall be recorded - // for the metric. - updateAcquireElapsedTimeMetric(); - } else { - // tryComplete invoked forceComplete, so we can use the data from tryComplete. - topicPartitionData = partitionsAcquired; - } - - if (topicPartitionData.isEmpty()) { - // No locks for share partitions could be acquired, so we complete the request with an empty response. - shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), 0); - shareFetch.maybeComplete(Map.of()); - return; + if (remoteStorageFetchException.isPresent()) { + completeErroneousRemoteShareFetchRequest(); + } else if (remoteFetchOpt.isPresent()) { + completeRemoteStorageShareFetchRequest(); } else { - // Update metric to record acquired to requested partitions. - double requestTopicToAcquired = (double) topicPartitionData.size() / shareFetch.topicIdPartitions().size(); - shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), (int) (requestTopicToAcquired * 100)); + completeLocalLogShareFetchRequest(); } - log.trace("Fetchable share partitions data: {} with groupId: {} fetch params: {}", - topicPartitionData, shareFetch.groupId(), shareFetch.fetchParams()); - - completeShareFetchRequest(topicPartitionData); } finally { lock.unlock(); } } - private void completeShareFetchRequest(LinkedHashMap topicPartitionData) { + private void completeLocalLogShareFetchRequest() { + LinkedHashMap topicPartitionData; + // tryComplete did not invoke forceComplete, so we need to check if we have any partitions to fetch. + if (partitionsAcquired.isEmpty()) { + topicPartitionData = acquirablePartitions(sharePartitions); + // The TopicPartitionsAcquireTimeMs metric signifies the tension when acquiring the locks + // for the share partition, hence if no partitions are yet acquired by tryComplete, + // we record the metric here. Do not check if the request has successfully acquired any + // partitions now or not, as then the upper bound of request timeout shall be recorded + // for the metric. + updateAcquireElapsedTimeMetric(); + } else { + // tryComplete invoked forceComplete, so we can use the data from tryComplete. + topicPartitionData = partitionsAcquired; + } + + if (topicPartitionData.isEmpty()) { + // No locks for share partitions could be acquired, so we complete the request with an empty response. + shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), 0); + shareFetch.maybeComplete(Map.of()); + return; + } else { + // Update metric to record acquired to requested partitions. + double requestTopicToAcquired = (double) topicPartitionData.size() / shareFetch.topicIdPartitions().size(); + shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), (int) (requestTopicToAcquired * 100)); + } + log.trace("Fetchable share partitions data: {} with groupId: {} fetch params: {}", + topicPartitionData, shareFetch.groupId(), shareFetch.fetchParams()); + + processAcquiredTopicPartitionsForLocalLogFetch(topicPartitionData); + } + + private void processAcquiredTopicPartitionsForLocalLogFetch(LinkedHashMap topicPartitionData) { try { LinkedHashMap responseData; - if (partitionsAlreadyFetched.isEmpty()) + if (localPartitionsAlreadyFetched.isEmpty()) responseData = readFromLog( topicPartitionData, partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes, topicPartitionData.keySet(), topicPartitionData.size())); else // There shouldn't be a case when we have a partitionsAlreadyFetched value here and this variable is getting // updated in a different tryComplete thread. - responseData = combineLogReadResponse(topicPartitionData, partitionsAlreadyFetched); + responseData = combineLogReadResponse(topicPartitionData, localPartitionsAlreadyFetched); List shareFetchPartitionDataList = new ArrayList<>(); responseData.forEach((topicIdPartition, logReadResult) -> @@ -225,15 +268,7 @@ private void completeShareFetchRequest(LinkedHashMap top log.error("Error processing delayed share fetch request", e); handleFetchException(shareFetch, topicPartitionData.keySet(), e); } finally { - // Releasing the lock to move ahead with the next request in queue. - releasePartitionLocks(topicPartitionData.keySet()); - // If we have a fetch request completed for a topic-partition, we release the locks for that partition, - // then we should check if there is a pending share fetch request for the topic-partition and complete it. - // We add the action to delayed actions queue to avoid an infinite call stack, which could happen if - // we directly call delayedShareFetchPurgatory.checkAndComplete - replicaManager.addToActionQueue(() -> topicPartitionData.keySet().forEach(topicIdPartition -> - replicaManager.completeDelayedShareFetchRequest( - new DelayedShareFetchGroupKey(shareFetch.groupId(), topicIdPartition.topicId(), topicIdPartition.partition())))); + releasePartitionLocksAndAddToActionQueue(topicPartitionData.keySet()); } } @@ -242,8 +277,12 @@ private void completeShareFetchRequest(LinkedHashMap top */ @Override public boolean tryComplete() { - LinkedHashMap topicPartitionData = acquirablePartitions(); + // Check to see if the remote fetch is in flight. If there is an in flight remote fetch we want to resolve it first. + if (remoteFetchOpt.isPresent()) { + return maybeCompletePendingRemoteFetch(); + } + LinkedHashMap topicPartitionData = acquirablePartitions(sharePartitions); try { if (!topicPartitionData.isEmpty()) { // Update the metric to record the time taken to acquire the locks for the share partitions. @@ -252,17 +291,17 @@ public boolean tryComplete() { // replicaManager.readFromLog to populate the offset metadata and update the fetch offset metadata for // those topic partitions. LinkedHashMap replicaManagerReadResponse = maybeReadFromLog(topicPartitionData); + // Store the remote fetch info and the topic partition for which we need to perform remote fetch. + Optional topicPartitionRemoteFetchInfoOpt = maybePrepareRemoteStorageFetchInfo(topicPartitionData, replicaManagerReadResponse); + + if (topicPartitionRemoteFetchInfoOpt.isPresent()) { + return maybeProcessRemoteFetch(topicPartitionData, topicPartitionRemoteFetchInfoOpt.get()); + } maybeUpdateFetchOffsetMetadata(topicPartitionData, replicaManagerReadResponse); if (anyPartitionHasLogReadError(replicaManagerReadResponse) || isMinBytesSatisfied(topicPartitionData, partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes, topicPartitionData.keySet(), topicPartitionData.size()))) { partitionsAcquired = topicPartitionData; - partitionsAlreadyFetched = replicaManagerReadResponse; - boolean completedByMe = forceComplete(); - // If invocation of forceComplete is not successful, then that means the request is already completed - // hence release the acquired locks. - if (!completedByMe) { - releasePartitionLocks(partitionsAcquired.keySet()); - } - return completedByMe; + localPartitionsAlreadyFetched = replicaManagerReadResponse; + return forceCompleteRequest(); } else { log.debug("minBytes is not satisfied for the share fetch request for group {}, member {}, " + "topic partitions {}", shareFetch.groupId(), shareFetch.memberId(), @@ -277,10 +316,18 @@ public boolean tryComplete() { return false; } catch (Exception e) { log.error("Error processing delayed share fetch request", e); - releasePartitionLocks(topicPartitionData.keySet()); - partitionsAcquired.clear(); - partitionsAlreadyFetched.clear(); - return forceComplete(); + // In case we have a remote fetch exception, we have already released locks for partitions which have potential + // local log read. We do not release locks for partitions which have a remote storage read because we need to + // complete the share fetch request in onComplete and if we release the locks early here, some other DelayedShareFetch + // request might get the locks for those partitions without this one getting complete. + if (remoteStorageFetchException.isEmpty()) { + releasePartitionLocks(topicPartitionData.keySet()); + partitionsAcquired.clear(); + localPartitionsAlreadyFetched.clear(); + return forceCompleteRequest(); + } else { + return forceCompleteRequest(); + } } } @@ -288,11 +335,13 @@ public boolean tryComplete() { * Prepare fetch request structure for partitions in the share fetch request for which we can acquire records. */ // Visible for testing - LinkedHashMap acquirablePartitions() { + LinkedHashMap acquirablePartitions( + LinkedHashMap sharePartitionsForAcquire + ) { // Initialize the topic partitions for which the fetch should be attempted. LinkedHashMap topicPartitionData = new LinkedHashMap<>(); - sharePartitions.forEach((topicIdPartition, sharePartition) -> { + sharePartitionsForAcquire.forEach((topicIdPartition, sharePartition) -> { // Add the share partition to the list of partitions to be fetched only if we can // acquire the fetch lock on it. if (sharePartition.maybeAcquireFetchLock()) { @@ -529,8 +578,307 @@ Lock lock() { return lock; } + // Visible for testing. + RemoteFetch remoteFetch() { + return remoteFetchOpt.orElse(null); + } + // Visible for testing. Meter expiredRequestMeter() { return expiredRequestMeter; } + + private Optional maybePrepareRemoteStorageFetchInfo( + LinkedHashMap topicPartitionData, + LinkedHashMap replicaManagerReadResponse + ) { + Optional topicPartitionRemoteFetchInfoOpt = Optional.empty(); + for (Map.Entry entry : replicaManagerReadResponse.entrySet()) { + TopicIdPartition topicIdPartition = entry.getKey(); + LogReadResult logReadResult = entry.getValue(); + if (logReadResult.info().delayedRemoteStorageFetch.isPresent()) { + // TODO: There is a limitation in remote storage fetch for consumer groups that we can only perform remote fetch for + // a single topic partition in a fetch request. Since, the logic of fetch is largely based on how consumer groups work, + // we are following the same logic. However, this problem should be addressed as part of KAFKA-19133 which should help us perform + // fetch for multiple remote fetch topic partition in a single share fetch request + topicPartitionRemoteFetchInfoOpt = Optional.of(new TopicPartitionRemoteFetchInfo(topicIdPartition, logReadResult)); + partitionsAcquired.put(topicIdPartition, topicPartitionData.get(topicIdPartition)); + break; + } + } + return topicPartitionRemoteFetchInfoOpt; + } + + private boolean maybeProcessRemoteFetch( + LinkedHashMap topicPartitionData, + TopicPartitionRemoteFetchInfo topicPartitionRemoteFetchInfo + ) { + Set nonRemoteFetchTopicPartitions = new LinkedHashSet<>(); + topicPartitionData.keySet().forEach(topicIdPartition -> { + // topic partitions for which fetch would not be happening in this share fetch request. + if (!topicPartitionRemoteFetchInfo.topicIdPartition().equals(topicIdPartition)) { + nonRemoteFetchTopicPartitions.add(topicIdPartition); + } + }); + // Release fetch lock for the topic partitions that were acquired but were not a part of remote fetch and add + // them to the delayed actions queue. + releasePartitionLocksAndAddToActionQueue(nonRemoteFetchTopicPartitions); + processRemoteFetchOrException(topicPartitionRemoteFetchInfo); + // Check if remote fetch can be completed. + return maybeCompletePendingRemoteFetch(); + } + + /** + * Throws an exception if a task for remote storage fetch could not be scheduled successfully else updates remoteFetchOpt. + * @param topicPartitionRemoteFetchInfo - The remote storage fetch information. + */ + private void processRemoteFetchOrException( + TopicPartitionRemoteFetchInfo topicPartitionRemoteFetchInfo + ) { + TopicIdPartition remoteFetchTopicIdPartition = topicPartitionRemoteFetchInfo.topicIdPartition(); + RemoteStorageFetchInfo remoteStorageFetchInfo = topicPartitionRemoteFetchInfo.logReadResult().info().delayedRemoteStorageFetch.get(); + + Future remoteFetchTask; + CompletableFuture remoteFetchResult = new CompletableFuture<>(); + try { + remoteFetchTask = replicaManager.remoteLogManager().get().asyncRead( + remoteStorageFetchInfo, + result -> { + remoteFetchResult.complete(result); + replicaManager.completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(shareFetch.groupId(), remoteFetchTopicIdPartition.topicId(), remoteFetchTopicIdPartition.partition())); + } + ); + } catch (Exception e) { + // Throw the error if any in scheduling the remote fetch task. + remoteStorageFetchException = Optional.of(e); + throw e; + } + remoteFetchOpt = Optional.of(new RemoteFetch(remoteFetchTopicIdPartition, topicPartitionRemoteFetchInfo.logReadResult(), remoteFetchTask, remoteFetchResult, remoteStorageFetchInfo)); + } + + /** + * This function checks if the remote fetch can be completed or not. It should always be called once you confirm remoteFetchOpt.isPresent(). + * The operation can be completed if: + * Case a: The partition is in an offline log directory on this broker + * Case b: This broker does not know the partition it tries to fetch + * Case c: This broker is no longer the leader of the partition it tries to fetch + * Case d: The remote storage read request completed (succeeded or failed) + * @return boolean representing whether the remote fetch is completed or not. + */ + private boolean maybeCompletePendingRemoteFetch() { + boolean canComplete = false; + + TopicIdPartition topicIdPartition = remoteFetchOpt.get().topicIdPartition(); + try { + replicaManager.getPartitionOrException(topicIdPartition.topicPartition()); + } catch (KafkaStorageException e) { // Case a + log.debug("TopicPartition {} is in an offline log directory, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } catch (UnknownTopicOrPartitionException e) { // Case b + log.debug("Broker no longer knows of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } catch (NotLeaderOrFollowerException e) { // Case c + log.debug("Broker is no longer the leader or follower of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } + + if (canComplete || remoteFetchOpt.get().remoteFetchResult().isDone()) { // Case d + return forceCompleteRequest(); + } else + return false; + } + + /** + * This function completes a share fetch request for which we have identified erroneous remote storage fetch in tryComplete() + * It should only be called when we know that there is remote fetch in-flight/completed. + */ + private void completeErroneousRemoteShareFetchRequest() { + try { + handleFetchException(shareFetch, partitionsAcquired.keySet(), remoteStorageFetchException.get()); + } finally { + releasePartitionLocksAndAddToActionQueue(partitionsAcquired.keySet()); + } + + } + + private void releasePartitionLocksAndAddToActionQueue(Set topicIdPartitions) { + if (topicIdPartitions.isEmpty()) { + return; + } + // Releasing the lock to move ahead with the next request in queue. + releasePartitionLocks(topicIdPartitions); + // If we have a fetch request completed for a topic-partition, we release the locks for that partition, + // then we should check if there is a pending share fetch request for the topic-partition and complete it. + // We add the action to delayed actions queue to avoid an infinite call stack, which could happen if + // we directly call delayedShareFetchPurgatory.checkAndComplete + replicaManager.addToActionQueue(() -> topicIdPartitions.forEach(topicIdPartition -> + replicaManager.completeDelayedShareFetchRequest( + new DelayedShareFetchGroupKey(shareFetch.groupId(), topicIdPartition.topicId(), topicIdPartition.partition())))); + } + + /** + * This function completes a share fetch request for which we have identified remoteFetch during tryComplete() + * Note - This function should only be called when we know that there is remote fetch. + */ + private void completeRemoteStorageShareFetchRequest() { + LinkedHashMap acquiredNonRemoteFetchTopicPartitionData = new LinkedHashMap<>(); + try { + List shareFetchPartitionData = new ArrayList<>(); + int readableBytes = 0; + if (remoteFetchOpt.get().remoteFetchResult().isDone()) { + RemoteFetch remoteFetch = remoteFetchOpt.get(); + RemoteLogReadResult remoteLogReadResult = remoteFetch.remoteFetchResult().get(); + if (remoteLogReadResult.error.isPresent()) { + Throwable error = remoteLogReadResult.error.get(); + // If there is any error for the remote fetch topic partition, we populate the error accordingly. + shareFetchPartitionData.add( + new ShareFetchPartitionData( + remoteFetch.topicIdPartition(), + partitionsAcquired.get(remoteFetch.topicIdPartition()), + ReplicaManager.createLogReadResult(error).toFetchPartitionData(false) + ) + ); + } else { + FetchDataInfo info = remoteLogReadResult.fetchDataInfo.get(); + TopicIdPartition topicIdPartition = remoteFetch.topicIdPartition(); + LogReadResult logReadResult = remoteFetch.logReadResult(); + shareFetchPartitionData.add( + new ShareFetchPartitionData( + topicIdPartition, + partitionsAcquired.get(remoteFetch.topicIdPartition()), + new FetchPartitionData( + logReadResult.error(), + logReadResult.highWatermark(), + logReadResult.leaderLogStartOffset(), + info.records, + Optional.empty(), + logReadResult.lastStableOffset().isDefined() ? OptionalLong.of((Long) logReadResult.lastStableOffset().get()) : OptionalLong.empty(), + info.abortedTransactions, + logReadResult.preferredReadReplica().isDefined() ? OptionalInt.of((Integer) logReadResult.preferredReadReplica().get()) : OptionalInt.empty(), + false + ) + ) + ); + readableBytes += info.records.sizeInBytes(); + } + } else { + cancelRemoteFetchTask(); + } + + // If remote fetch bytes < shareFetch.fetchParams().maxBytes, then we will try for a local read. + if (readableBytes < shareFetch.fetchParams().maxBytes) { + // Get the local log read based topic partitions. + LinkedHashMap nonRemoteFetchSharePartitions = new LinkedHashMap<>(); + sharePartitions.forEach((topicIdPartition, sharePartition) -> { + if (!partitionsAcquired.containsKey(topicIdPartition)) { + nonRemoteFetchSharePartitions.put(topicIdPartition, sharePartition); + } + }); + acquiredNonRemoteFetchTopicPartitionData = acquirablePartitions(nonRemoteFetchSharePartitions); + if (!acquiredNonRemoteFetchTopicPartitionData.isEmpty()) { + log.trace("Fetchable local share partitions for a remote share fetch request data: {} with groupId: {} fetch params: {}", + acquiredNonRemoteFetchTopicPartitionData, shareFetch.groupId(), shareFetch.fetchParams()); + + LinkedHashMap responseData = readFromLog( + acquiredNonRemoteFetchTopicPartitionData, + partitionMaxBytesStrategy.maxBytes(shareFetch.fetchParams().maxBytes - readableBytes, acquiredNonRemoteFetchTopicPartitionData.keySet(), acquiredNonRemoteFetchTopicPartitionData.size())); + for (Map.Entry entry : responseData.entrySet()) { + if (entry.getValue().info().delayedRemoteStorageFetch.isEmpty()) { + shareFetchPartitionData.add( + new ShareFetchPartitionData( + entry.getKey(), + acquiredNonRemoteFetchTopicPartitionData.get(entry.getKey()), + entry.getValue().toFetchPartitionData(false) + ) + ); + } + } + } + } + + // Update metric to record acquired to requested partitions. + double acquiredRatio = (double) (partitionsAcquired.size() + acquiredNonRemoteFetchTopicPartitionData.size()) / shareFetch.topicIdPartitions().size(); + if (acquiredRatio > 0) + shareGroupMetrics.recordTopicPartitionsFetchRatio(shareFetch.groupId(), (int) (acquiredRatio * 100)); + + Map remoteFetchResponse = ShareFetchUtils.processFetchResponse( + shareFetch, shareFetchPartitionData, sharePartitions, replicaManager, exceptionHandler); + shareFetch.maybeComplete(remoteFetchResponse); + log.trace("Remote share fetch request completed successfully, response: {}", remoteFetchResponse); + } catch (InterruptedException | ExecutionException e) { + log.error("Exception occurred in completing remote fetch {} for delayed share fetch request {}", remoteFetchOpt.get(), e); + handleExceptionInCompletingRemoteStorageShareFetchRequest(acquiredNonRemoteFetchTopicPartitionData.keySet(), e); + } catch (Exception e) { + log.error("Unexpected error in processing delayed share fetch request", e); + handleExceptionInCompletingRemoteStorageShareFetchRequest(acquiredNonRemoteFetchTopicPartitionData.keySet(), e); + } finally { + Set topicIdPartitions = new LinkedHashSet<>(partitionsAcquired.keySet()); + topicIdPartitions.addAll(acquiredNonRemoteFetchTopicPartitionData.keySet()); + releasePartitionLocksAndAddToActionQueue(topicIdPartitions); + } + } + + private void handleExceptionInCompletingRemoteStorageShareFetchRequest( + Set acquiredNonRemoteFetchTopicPartitions, + Exception e + ) { + Set topicIdPartitions = new LinkedHashSet<>(partitionsAcquired.keySet()); + topicIdPartitions.addAll(acquiredNonRemoteFetchTopicPartitions); + handleFetchException(shareFetch, topicIdPartitions, e); + } + + /** + * Cancel the remote storage read task, if it has not been executed yet and avoid interrupting the task if it is + * already running as it may force closing opened/cached resources as transaction index. + * Note - This function should only be called when we know that there is remote fetch. + */ + private void cancelRemoteFetchTask() { + boolean cancelled = remoteFetchOpt.get().remoteFetchTask().cancel(false); + if (!cancelled) { + log.debug("Remote fetch task for RemoteStorageFetchInfo: {} could not be cancelled and its isDone value is {}", + remoteFetchOpt.get().remoteFetchInfo(), remoteFetchOpt.get().remoteFetchTask().isDone()); + } + } + + private boolean forceCompleteRequest() { + boolean completedByMe = forceComplete(); + // If the delayed operation is completed by me, the acquired locks are already released in onComplete(). + // Otherwise, we need to release the acquired locks. + if (!completedByMe) { + releasePartitionLocksAndAddToActionQueue(partitionsAcquired.keySet()); + } + return completedByMe; + } + + public record RemoteFetch( + TopicIdPartition topicIdPartition, + LogReadResult logReadResult, + Future remoteFetchTask, + CompletableFuture remoteFetchResult, + RemoteStorageFetchInfo remoteFetchInfo + ) { + @Override + public String toString() { + return "RemoteFetch(" + + "topicIdPartition=" + topicIdPartition + + ", logReadResult=" + logReadResult + + ", remoteFetchTask=" + remoteFetchTask + + ", remoteFetchResult=" + remoteFetchResult + + ", remoteFetchInfo=" + remoteFetchInfo + + ")"; + } + } + + public record TopicPartitionRemoteFetchInfo( + TopicIdPartition topicIdPartition, + LogReadResult logReadResult + ) { + @Override + public String toString() { + return "TopicPartitionRemoteFetchInfo(" + + "topicIdPartition=" + topicIdPartition + + ", logReadResult=" + logReadResult + + ")"; + } + } } diff --git a/core/src/main/java/kafka/server/share/ShareFetchUtils.java b/core/src/main/java/kafka/server/share/ShareFetchUtils.java index 3cfab25e6845a..603ae8e048b23 100644 --- a/core/src/main/java/kafka/server/share/ShareFetchUtils.java +++ b/core/src/main/java/kafka/server/share/ShareFetchUtils.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.FileLogInputStream.FileChannelRecordBatch; import org.apache.kafka.common.record.FileRecords; +import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.requests.ListOffsetsRequest; import org.apache.kafka.coordinator.group.GroupConfigManager; @@ -83,7 +84,7 @@ static Map processFetchR if (fetchPartitionData.error.code() != Errors.NONE.code()) { partitionData - .setRecords(null) + .setRecords(MemoryRecords.EMPTY) .setErrorCode(fetchPartitionData.error.code()) .setErrorMessage(fetchPartitionData.error.message()) .setAcquiredRecords(List.of()); @@ -123,7 +124,7 @@ static Map processFetchR // if we want parallel requests for the same share partition or not. if (shareAcquiredRecords.acquiredRecords().isEmpty()) { partitionData - .setRecords(null) + .setRecords(MemoryRecords.EMPTY) .setAcquiredRecords(List.of()); } else { partitionData diff --git a/core/src/main/java/kafka/server/share/SharePartitionManager.java b/core/src/main/java/kafka/server/share/SharePartitionManager.java index 1b50a70c18ca6..a53f846a01c24 100644 --- a/core/src/main/java/kafka/server/share/SharePartitionManager.java +++ b/core/src/main/java/kafka/server/share/SharePartitionManager.java @@ -448,8 +448,7 @@ public ShareFetchContext newContext(String groupId, List share ImplicitLinkedHashCollection<>(shareFetchData.size()); shareFetchData.forEach(topicIdPartition -> cachedSharePartitions.mustAdd(new CachedSharePartition(topicIdPartition, false))); - ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), - time.milliseconds(), cachedSharePartitions); + ShareSessionKey responseShareSessionKey = cache.maybeCreateSession(groupId, reqMetadata.memberId(), cachedSharePartitions); if (responseShareSessionKey == null) { log.error("Could not create a share session for group {} member {}", groupId, reqMetadata.memberId()); throw Errors.SHARE_SESSION_NOT_FOUND.exception(); @@ -476,7 +475,7 @@ public ShareFetchContext newContext(String groupId, List share } Map> modifiedTopicIdPartitions = shareSession.update( shareFetchData, toForget); - cache.touch(shareSession, time.milliseconds()); + cache.updateNumPartitions(shareSession); shareSession.epoch = ShareRequestMetadata.nextEpoch(shareSession.epoch); log.debug("Created a new ShareSessionContext for session key {}, epoch {}: " + "added {}, updated {}, removed {}", shareSession.key(), shareSession.epoch, @@ -517,7 +516,7 @@ public void acknowledgeSessionUpdate(String groupId, ShareRequestMetadata reqMet shareSession.epoch, reqMetadata.epoch()); throw Errors.INVALID_SHARE_SESSION_EPOCH.exception(); } - cache.touch(shareSession, time.milliseconds()); + cache.updateNumPartitions(shareSession); shareSession.epoch = ShareRequestMetadata.nextEpoch(shareSession.epoch); } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 506e46e6ce5d7..e555f1b0b467d 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -23,7 +23,6 @@ import java.nio.file.{Files, NoSuchFileException} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import kafka.server.{KafkaConfig, KafkaRaftServer} -import kafka.server.metadata.BrokerMetadataPublisher.info import kafka.utils.threadsafe import kafka.utils.{CoreUtils, Logging, Pool} import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} @@ -42,7 +41,7 @@ import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsem import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.{FileLock, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, LogOffsetsListener, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} +import org.apache.kafka.storage.internals.log.{CleanerConfig, LogCleaner, LogConfig, LogDirFailureChannel, LogManager => JLogManager, LogOffsetsListener, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats @@ -80,8 +79,6 @@ class LogManager(logDirs: Seq[File], remoteStorageSystemEnable: Boolean, val initialTaskDelayMs: Long) extends Logging { - import LogManager._ - private val metricsGroup = new KafkaMetricsGroup(this.getClass) private val logCreationOrDeletionLock = new Object @@ -127,9 +124,9 @@ class LogManager(logDirs: Seq[File], def directoryIdsSet: Predef.Set[Uuid] = directoryIds.values.toSet @volatile private var recoveryPointCheckpoints = liveLogDirs.map(dir => - (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile), logDirFailureChannel))).toMap + (dir, new OffsetCheckpointFile(new File(dir, JLogManager.RECOVERY_POINT_CHECKPOINT_FILE), logDirFailureChannel))).toMap @volatile private var logStartOffsetCheckpoints = liveLogDirs.map(dir => - (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile), logDirFailureChannel))).toMap + (dir, new OffsetCheckpointFile(new File(dir, JLogManager.LOG_START_OFFSET_CHECKPOINT_FILE), logDirFailureChannel))).toMap private val preferredLogDirs = new ConcurrentHashMap[TopicPartition, String]() @@ -261,7 +258,7 @@ class LogManager(logDirs: Seq[File], private def lockLogDirs(dirs: Seq[File]): Seq[FileLock] = { dirs.flatMap { dir => try { - val lock = new FileLock(new File(dir, LockFileName)) + val lock = new FileLock(new File(dir, JLogManager.LOCK_FILE_NAME)) if (!lock.tryLock()) throw new KafkaException("Failed to acquire lock on file .lock in " + lock.file.getParent + ". A Kafka instance in another process or thread is using this directory.") @@ -680,7 +677,7 @@ class LogManager(logDirs: Seq[File], try { jobs.foreachEntry { (dir, dirJobs) => - if (waitForAllToComplete(dirJobs, + if (JLogManager.waitForAllToComplete(dirJobs.toList.asJava, e => warn(s"There was an error in one of the threads during LogManager shutdown: ${e.getCause}"))) { val logs = logsInDir(localLogsByDir, dir) @@ -1520,25 +1517,6 @@ class LogManager(logDirs: Seq[File], } object LogManager { - val LockFileName = ".lock" - - /** - * Wait all jobs to complete - * @param jobs jobs - * @param callback this will be called to handle the exception caused by each Future#get - * @return true if all pass. Otherwise, false - */ - private[log] def waitForAllToComplete(jobs: Seq[Future[_]], callback: Throwable => Unit): Boolean = { - jobs.count(future => Try(future.get) match { - case Success(_) => false - case Failure(e) => - callback(e) - true - }) == 0 - } - - val RecoveryPointCheckpointFile = "recovery-point-offset-checkpoint" - val LogStartOffsetCheckpointFile = "log-start-offset-checkpoint" def apply(config: KafkaConfig, initialOfflineDirs: Seq[String], @@ -1575,45 +1553,4 @@ object LogManager { remoteStorageSystemEnable = config.remoteLogManagerConfig.isRemoteStorageSystemEnabled, initialTaskDelayMs = config.logInitialTaskDelayMs) } - - /** - * Returns true if the given log should not be on the current broker - * according to the metadata image. - * - * @param brokerId The ID of the current broker. - * @param newTopicsImage The new topics image after broker has been reloaded - * @param log The log object to check - * @return true if the log should not exist on the broker, false otherwise. - */ - def isStrayKraftReplica( - brokerId: Int, - newTopicsImage: TopicsImage, - log: UnifiedLog - ): Boolean = { - if (log.topicId.isEmpty) { - // Missing topic ID could result from storage failure or unclean shutdown after topic creation but before flushing - // data to the `partition.metadata` file. And before appending data to the log, the `partition.metadata` is always - // flushed to disk. So if the topic ID is missing, it mostly means no data was appended, and we can treat this as - // a stray log. - info(s"The topicId does not exist in $log, treat it as a stray log") - return true - } - - val topicId = log.topicId.get - val partitionId = log.topicPartition.partition() - Option(newTopicsImage.getPartition(topicId, partitionId)) match { - case Some(partition) => - if (!partition.replicas.contains(brokerId)) { - info(s"Found stray log dir $log: the current replica assignment ${partition.replicas.mkString("[", ", ", "]")} " + - s"does not contain the local brokerId $brokerId.") - true - } else { - false - } - - case None => - info(s"Found stray log dir $log: the topicId $topicId does not exist in the metadata image") - true - } - } } diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala index 84dfa5ebee001..0727c660fe406 100644 --- a/core/src/main/scala/kafka/raft/RaftManager.scala +++ b/core/src/main/scala/kafka/raft/RaftManager.scala @@ -24,7 +24,6 @@ import java.util.OptionalInt import java.util.concurrent.CompletableFuture import java.util.{Map => JMap} import java.util.{Collection => JCollection} -import kafka.log.LogManager import kafka.server.KafkaConfig import kafka.utils.CoreUtils import kafka.utils.Logging @@ -48,7 +47,7 @@ import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.util.{FileLock, KafkaScheduler} import org.apache.kafka.server.fault.FaultHandler import org.apache.kafka.server.util.timer.SystemTimer -import org.apache.kafka.storage.internals.log.UnifiedLog +import org.apache.kafka.storage.internals.log.{LogManager, UnifiedLog} import scala.jdk.CollectionConverters._ import scala.jdk.OptionConverters._ @@ -62,7 +61,7 @@ object KafkaRaftManager { } private def lockDataDir(dataDir: File): FileLock = { - val lock = new FileLock(new File(dataDir, LogManager.LockFileName)) + val lock = new FileLock(new File(dataDir, LogManager.LOCK_FILE_NAME)) if (!lock.tryLock()) { throw new KafkaException( diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala index 8f9b983cb782f..a2c6d4b98f58e 100644 --- a/core/src/main/scala/kafka/server/BrokerServer.scala +++ b/core/src/main/scala/kafka/server/BrokerServer.scala @@ -426,9 +426,7 @@ class BrokerServer( )) val fetchManager = new FetchManager(Time.SYSTEM, new FetchSessionCache(fetchSessionCacheShards)) - val shareFetchSessionCache : ShareSessionCache = new ShareSessionCache( - config.shareGroupConfig.shareGroupMaxGroups * config.groupCoordinatorConfig.shareGroupMaxSize, - KafkaBroker.MIN_INCREMENTAL_FETCH_SESSION_EVICTION_MS) + val shareFetchSessionCache : ShareSessionCache = new ShareSessionCache(config.shareGroupConfig.shareGroupMaxShareSessions()) sharePartitionManager = new SharePartitionManager( replicaManager, diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 6a22963ac7d6a..70af5fd645677 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -275,11 +275,21 @@ class KafkaApis(val requestChannel: RequestChannel, ): CompletableFuture[Unit] = { val offsetCommitRequest = request.body[OffsetCommitRequest] - // Reject the request if not authorized to the group + // Reject the request if not authorized to the group. if (!authHelper.authorize(request.context, READ, GROUP, offsetCommitRequest.data.groupId)) { requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED.exception)) CompletableFuture.completedFuture[Unit](()) } else { + val useTopicIds = OffsetCommitResponse.useTopicIds(request.header.apiVersion) + + if (useTopicIds) { + offsetCommitRequest.data.topics.forEach { topic => + if (topic.topicId != Uuid.ZERO_UUID) { + metadataCache.getTopicName(topic.topicId).ifPresent(name => topic.setName(name)) + } + } + } + val authorizedTopics = authHelper.filterByAuthorized( request.context, READ, @@ -287,28 +297,40 @@ class KafkaApis(val requestChannel: RequestChannel, offsetCommitRequest.data.topics.asScala )(_.name) - val responseBuilder = new OffsetCommitResponse.Builder() + val responseBuilder = OffsetCommitResponse.newBuilder(useTopicIds) val authorizedTopicsRequest = new mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic]() offsetCommitRequest.data.topics.forEach { topic => - if (!authorizedTopics.contains(topic.name)) { + if (useTopicIds && topic.name.isEmpty) { + // If the topic name is undefined, it means that the topic id is unknown so we add + // the topic and all its partitions to the response with UNKNOWN_TOPIC_ID. + responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition]( + topic.topicId, topic.name, topic.partitions, _.partitionIndex, Errors.UNKNOWN_TOPIC_ID) + } else if (!authorizedTopics.contains(topic.name)) { // If the topic is not authorized, we add the topic and all its partitions // to the response with TOPIC_AUTHORIZATION_FAILED. responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition]( - topic.name, topic.partitions, _.partitionIndex, Errors.TOPIC_AUTHORIZATION_FAILED) + topic.topicId, topic.name, topic.partitions, _.partitionIndex, Errors.TOPIC_AUTHORIZATION_FAILED) } else if (!metadataCache.contains(topic.name)) { // If the topic is unknown, we add the topic and all its partitions // to the response with UNKNOWN_TOPIC_OR_PARTITION. responseBuilder.addPartitions[OffsetCommitRequestData.OffsetCommitRequestPartition]( - topic.name, topic.partitions, _.partitionIndex, Errors.UNKNOWN_TOPIC_OR_PARTITION) + topic.topicId, topic.name, topic.partitions, _.partitionIndex, Errors.UNKNOWN_TOPIC_OR_PARTITION) } else { // Otherwise, we check all partitions to ensure that they all exist. - val topicWithValidPartitions = new OffsetCommitRequestData.OffsetCommitRequestTopic().setName(topic.name) + val topicWithValidPartitions = new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(topic.topicId) + .setName(topic.name) topic.partitions.forEach { partition => - if (metadataCache.getLeaderAndIsr(topic.name, partition.partitionIndex).isPresent()) { + if (metadataCache.getLeaderAndIsr(topic.name, partition.partitionIndex).isPresent) { topicWithValidPartitions.partitions.add(partition) } else { - responseBuilder.addPartition(topic.name, partition.partitionIndex, Errors.UNKNOWN_TOPIC_OR_PARTITION) + responseBuilder.addPartition( + topic.topicId, + topic.name, + partition.partitionIndex, + Errors.UNKNOWN_TOPIC_OR_PARTITION + ) } } @@ -322,42 +344,23 @@ class KafkaApis(val requestChannel: RequestChannel, requestHelper.sendMaybeThrottle(request, responseBuilder.build()) CompletableFuture.completedFuture(()) } else { - // For version > 0, store offsets in Coordinator. - commitOffsetsToCoordinator( - request, - offsetCommitRequest, - authorizedTopicsRequest, - responseBuilder, - requestLocal - ) - } - } - } - - private def commitOffsetsToCoordinator( - request: RequestChannel.Request, - offsetCommitRequest: OffsetCommitRequest, - authorizedTopicsRequest: mutable.ArrayBuffer[OffsetCommitRequestData.OffsetCommitRequestTopic], - responseBuilder: OffsetCommitResponse.Builder, - requestLocal: RequestLocal - ): CompletableFuture[Unit] = { - val offsetCommitRequestData = new OffsetCommitRequestData() - .setGroupId(offsetCommitRequest.data.groupId) - .setMemberId(offsetCommitRequest.data.memberId) - .setGenerationIdOrMemberEpoch(offsetCommitRequest.data.generationIdOrMemberEpoch) - .setRetentionTimeMs(offsetCommitRequest.data.retentionTimeMs) - .setGroupInstanceId(offsetCommitRequest.data.groupInstanceId) - .setTopics(authorizedTopicsRequest.asJava) - - groupCoordinator.commitOffsets( - request.context, - offsetCommitRequestData, - requestLocal.bufferSupplier - ).handle[Unit] { (results, exception) => - if (exception != null) { - requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(exception)) - } else { - requestHelper.sendMaybeThrottle(request, responseBuilder.merge(results).build()) + groupCoordinator.commitOffsets( + request.context, + new OffsetCommitRequestData() + .setGroupId(offsetCommitRequest.data.groupId) + .setMemberId(offsetCommitRequest.data.memberId) + .setGenerationIdOrMemberEpoch(offsetCommitRequest.data.generationIdOrMemberEpoch) + .setRetentionTimeMs(offsetCommitRequest.data.retentionTimeMs) + .setGroupInstanceId(offsetCommitRequest.data.groupInstanceId) + .setTopics(authorizedTopicsRequest.asJava), + requestLocal.bufferSupplier + ).handle[Unit] { (results, exception) => + if (exception != null) { + requestHelper.sendMaybeThrottle(request, offsetCommitRequest.getErrorResponse(exception)) + } else { + requestHelper.sendMaybeThrottle(request, responseBuilder.merge(results).build()) + } + } } } } @@ -3116,6 +3119,7 @@ class KafkaApis(val requestChannel: RequestChannel, .setPartitionIndex(partitionIndex) .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(value) + .setRecords(MemoryRecords.EMPTY) topic.partitions.add(fetchPartitionData) } topicPartitionAcknowledgements.remove(topicId) @@ -3131,6 +3135,7 @@ class KafkaApis(val requestChannel: RequestChannel, .setPartitionIndex(partitionIndex) .setErrorCode(Errors.NONE.code) .setAcknowledgeErrorCode(value) + .setRecords(MemoryRecords.EMPTY) topicData.partitions.add(fetchPartitionData) } shareFetchResponse.data.responses.add(topicData) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 7049d5f24744b..b982e72e141fa 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -379,10 +379,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) if (!protocols.contains(GroupType.CLASSIC)) { throw new ConfigException(s"Disabling the '${GroupType.CLASSIC}' protocol is not supported.") } - if (protocols.contains(GroupType.SHARE)) { - warn(s"Share groups and the new '${GroupType.SHARE}' rebalance protocol are enabled. " + - "This is part of the early access of KIP-932 and MUST NOT be used in production.") - } if (protocols.contains(GroupType.STREAMS)) { warn(s"Streams groups and the new '${GroupType.STREAMS}' rebalance protocol are enabled. " + "This is part of the early access of KIP-1071 and MUST NOT be used in production.") diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala index de8f16e1e5808..d95be34ff51de 100644 --- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala @@ -34,6 +34,7 @@ import org.apache.kafka.image.{MetadataDelta, MetadataImage, TopicDelta} import org.apache.kafka.metadata.publisher.AclPublisher import org.apache.kafka.server.common.RequestLocal import org.apache.kafka.server.fault.FaultHandler +import org.apache.kafka.storage.internals.log.{LogManager => JLogManager} import java.util.concurrent.CompletableFuture import scala.collection.mutable @@ -300,7 +301,7 @@ class BrokerMetadataPublisher( // recovery-from-unclean-shutdown if required. logManager.startup( metadataCache.getAllTopics().asScala, - isStray = log => LogManager.isStrayKraftReplica(brokerId, newImage.topics(), log) + isStray = log => JLogManager.isStrayKraftReplica(brokerId, newImage.topics(), log) ) // Rename all future replicas which are in the same directory as the diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index bb8b51b40e297..43ece70ca0ee6 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -25,10 +25,15 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.KafkaStorageException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.message.ShareFetchResponseData; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.server.log.remote.storage.RemoteLogManager; import org.apache.kafka.server.purgatory.DelayedOperationKey; import org.apache.kafka.server.purgatory.DelayedOperationPurgatory; import org.apache.kafka.server.share.SharePartitionKey; @@ -46,6 +51,8 @@ import org.apache.kafka.storage.internals.log.FetchDataInfo; import org.apache.kafka.storage.internals.log.LogOffsetMetadata; import org.apache.kafka.storage.internals.log.LogOffsetSnapshot; +import org.apache.kafka.storage.internals.log.RemoteLogReadResult; +import org.apache.kafka.storage.internals.log.RemoteStorageFetchInfo; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; import org.junit.jupiter.api.AfterEach; @@ -61,10 +68,15 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.stream.Collectors; +import scala.Option; import scala.Tuple2; +import scala.collection.Seq; import scala.jdk.javaapi.CollectionConverters; import static kafka.server.share.SharePartitionManagerTest.DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL; @@ -73,6 +85,7 @@ import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createShareAcquiredRecords; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -95,6 +108,8 @@ public class DelayedShareFetchTest { private static final FetchParams FETCH_PARAMS = new FetchParams( FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty(), true); + private static final FetchDataInfo REMOTE_FETCH_INFO = new FetchDataInfo(new LogOffsetMetadata(0, 0, 0), + MemoryRecords.EMPTY, false, Optional.empty(), Optional.of(mock(RemoteStorageFetchInfo.class))); private static final BrokerTopicStats BROKER_TOPIC_STATS = new BrokerTopicStats(); private Timer mockTimer; @@ -487,7 +502,7 @@ public void testToCompleteAnAlreadyCompletedFuture() { delayedShareFetch.forceComplete(); assertTrue(delayedShareFetch.isCompleted()); // Verifying that the first forceComplete calls acquirablePartitions method in DelayedShareFetch. - Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(); + Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(sharePartitions); assertEquals(0, future.join().size()); assertTrue(delayedShareFetch.lock().tryLock()); delayedShareFetch.lock().unlock(); @@ -497,7 +512,7 @@ public void testToCompleteAnAlreadyCompletedFuture() { delayedShareFetch.forceComplete(); assertTrue(delayedShareFetch.isCompleted()); // Verifying that the second forceComplete does not call acquirablePartitions method in DelayedShareFetch. - Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(); + Mockito.verify(delayedShareFetch, times(1)).acquirablePartitions(sharePartitions); Mockito.verify(delayedShareFetch, times(0)).releasePartitionLocks(any()); assertTrue(delayedShareFetch.lock().tryLock()); // Assert both metrics shall be recorded only once. @@ -1155,6 +1170,525 @@ public void testOnCompleteExecutionOnTimeout() { assertEquals(1, delayedShareFetch.expiredRequestMeter().count()); } + @Test + public void testRemoteStorageFetchTryCompleteReturnsFalse() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + SharePartition sp2 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + when(sp2.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + sharePartitions.put(tp2, sp2); + + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + new CompletableFuture<>(), List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + when(sp2.nextFetchOffset()).thenReturn(30L); + + // Fetch offset matches with the cached entry for sp0 but not for sp1 and sp2. Hence, a replica manager fetch will happen for sp1 and sp2. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(10, 1, 0))); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp2.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking local log read result for tp1 and remote storage read result for tp2. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp1), Set.of(tp2))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object does not complete within tryComplete in this mock. + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + when(remoteLogManager.asyncRead(any(), any())).thenReturn(mock(Future.class)); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + assertFalse(delayedShareFetch.tryComplete()); + assertFalse(delayedShareFetch.isCompleted()); + // Remote fetch object gets created for delayed share fetch object. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released for local log read topic partitions tp0 and tp1. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1)); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchTryCompleteThrowsException() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + + // Fetch offset does not match with the cached entry for sp0 and sp1. Hence, a replica manager fetch will happen for sp0 and sp1. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking local log read result for tp0 and remote storage read result for tp1. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0), Set.of(tp1))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Exception will be thrown during the creation of remoteFetch object. + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + when(remoteLogManager.asyncRead(any(), any())).thenThrow(new RejectedExecutionException("Exception thrown")); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + BiConsumer exceptionHandler = mockExceptionHandler(); + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withExceptionHandler(exceptionHandler) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + // tryComplete returns true and goes to forceComplete once the exception occurs. + assertTrue(delayedShareFetch.tryComplete()); + assertTrue(delayedShareFetch.isCompleted()); + // The future of shareFetch completes. + assertTrue(shareFetch.isCompleted()); + assertFalse(future.isCompletedExceptionally()); + assertEquals(Set.of(tp1), future.join().keySet()); + // Exception occurred and was handled. + Mockito.verify(exceptionHandler, times(1)).accept(any(), any()); + // Verify the locks are released for both local and remote read topic partitions tp0 and tp1 because of exception occurrence. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp1)); + Mockito.verify(delayedShareFetch, times(1)).onComplete(); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchTryCompletionDueToBrokerBecomingOffline() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + SharePartition sp2 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + when(sp2.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + sharePartitions.put(tp2, sp2); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + when(sp2.nextFetchOffset()).thenReturn(30L); + + // Fetch offset matches with the cached entry for sp0 but not for sp1 and sp2. Hence, a replica manager fetch will happen for sp1 and sp2 during tryComplete. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.of(new LogOffsetMetadata(10, 1, 0))); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp2.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + when(sp1.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + // Mocking local log read result for tp1 and remote storage read result for tp2 on first replicaManager readFromLog call(from tryComplete). + // Mocking local log read result for tp0 and tp1 on second replicaManager readFromLog call(from onComplete). + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp1), Set.of(tp2)) + ).doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0, tp1), Set.of()) + ).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object does not complete within tryComplete in this mock but the broker becomes unavailable. + Future remoteFetchTask = mock(Future.class); + doAnswer(invocation -> { + when(remoteFetchTask.isCancelled()).thenReturn(true); + return false; + }).when(remoteFetchTask).cancel(false); + + when(remoteFetchTask.cancel(false)).thenReturn(true); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + when(remoteLogManager.asyncRead(any(), any())).thenReturn(remoteFetchTask); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenThrow(mock(KafkaStorageException.class)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + assertTrue(delayedShareFetch.remoteFetch().remoteFetchTask().isCancelled()); + // Partition locks should be released for all 3 topic partitions + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1, tp2)); + assertTrue(shareFetch.isCompleted()); + // Share fetch response contained tp0 and tp1 (local fetch) but not tp2, since it errored out. + assertEquals(Set.of(tp0, tp1), future.join().keySet()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchRequestCompletionOnFutureCompletionFailure() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + + // sp0 is acquirable, sp1 is not acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(false); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(false); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + // Fetch offset does not match with the cached entry for sp0. Hence, a replica manager fetch will happen for sp0. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking remote storage read result for tp0. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), Set.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.empty(), + Optional.of(new TimeoutException("Error occurred while creating remote fetch result")) // Remote fetch result is returned with an error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .build()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released for tp0. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); + assertTrue(shareFetch.isCompleted()); + assertEquals(Set.of(tp0), future.join().keySet()); + assertEquals(Errors.REQUEST_TIMED_OUT.code(), future.join().get(tp0).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchRequestCompletionOnFutureCompletionSuccessfully() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + + SharePartition sp0 = mock(SharePartition.class); + + // sp0 is acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + // Fetch offset does not match with the cached entry for sp0. Hence, a replica manager fetch will happen for sp0. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking remote storage read result for tp0. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), Set.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.of(REMOTE_FETCH_INFO), + Optional.empty() // Remote fetch result is returned successfully without error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0))) + .build()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released for tp0. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); + assertTrue(shareFetch.isCompleted()); + assertEquals(Set.of(tp0), future.join().keySet()); + assertEquals(Errors.NONE.code(), future.join().get(tp0).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchRequestCompletionAlongWithLocalLogRead() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 2)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + SharePartition sp2 = mock(SharePartition.class); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp2.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + when(sp2.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + sharePartitions.put(tp2, sp2); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1, tp2), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(20L); + when(sp2.nextFetchOffset()).thenReturn(30L); + + // Fetch offset does not match with the cached entry for sp0, sp1 and sp2. Hence, a replica manager fetch will happen for all of them in tryComplete. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp2.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + when(sp1.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + when(sp2.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + // Mocking local log read result for tp0, tp1 and remote storage read result for tp2 on first replicaManager readFromLog call(from tryComplete). + // Mocking local log read result for tp0 and tp1 on second replicaManager readFromLog call(from onComplete). + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0, tp1), Set.of(tp2)) + ).doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(tp0, tp1), Set.of()) + ).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.of(REMOTE_FETCH_INFO), + Optional.empty() // Remote fetch result is returned successfully without error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withReplicaManager(replicaManager) + .withSharePartitions(sharePartitions) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1, tp2))) + .build()); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // the future of shareFetch completes. + assertTrue(shareFetch.isCompleted()); + assertEquals(Set.of(tp0, tp1, tp2), future.join().keySet()); + // Verify the locks are released for both local log and remote storage read topic partitions tp0, tp1 and tp2. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1, tp2)); + assertEquals(Errors.NONE.code(), future.join().get(tp0).errorCode()); + assertEquals(Errors.NONE.code(), future.join().get(tp1).errorCode()); + assertEquals(Errors.NONE.code(), future.join().get(tp2).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + + @Test + public void testRemoteStorageFetchOnlyHappensForFirstTopicPartition() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 1)); + + SharePartition sp0 = mock(SharePartition.class); + SharePartition sp1 = mock(SharePartition.class); + + // sp0 and sp1 are acquirable. + when(sp0.maybeAcquireFetchLock()).thenReturn(true); + when(sp1.maybeAcquireFetchLock()).thenReturn(true); + when(sp0.canAcquireRecords()).thenReturn(true); + when(sp1.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + sharePartitions.put(tp1, sp1); + + CompletableFuture> future = new CompletableFuture<>(); + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + future, List.of(tp0, tp1), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + when(sp1.nextFetchOffset()).thenReturn(10L); + // Fetch offset does not match with the cached entry for sp0 and sp1. Hence, a replica manager fetch will happen for both. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + when(sp1.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + LinkedHashSet remoteStorageFetchPartitions = new LinkedHashSet<>(); + remoteStorageFetchPartitions.add(tp0); + remoteStorageFetchPartitions.add(tp1); + + // Mocking remote storage read result for tp0 and tp1. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), remoteStorageFetchPartitions)).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object completes within tryComplete in this mock, hence request will move on to forceComplete. + RemoteLogReadResult remoteFetchResult = new RemoteLogReadResult( + Optional.of(REMOTE_FETCH_INFO), + Optional.empty() // Remote fetch result is returned successfully without error. + ); + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + doAnswer(invocationOnMock -> { + // Make sure that the callback is called to populate remoteFetchResult for the mock behaviour. + Consumer callback = invocationOnMock.getArgument(1); + callback.accept(remoteFetchResult); + return CompletableFuture.completedFuture(remoteFetchResult); + }).when(remoteLogManager).asyncRead(any(), any()); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0, tp1))) + .build()); + + when(sp0.acquire(any(), anyInt(), anyInt(), anyLong(), any(), any())).thenReturn( + createShareAcquiredRecords(new ShareFetchResponseData.AcquiredRecords().setFirstOffset(0).setLastOffset(3).setDeliveryCount((short) 1))); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + + assertTrue(delayedShareFetch.isCompleted()); + // Pending remote fetch object gets created for delayed share fetch. + assertNotNull(delayedShareFetch.remoteFetch()); + // Verify the locks are released separately for tp1 (from tryComplete). + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp1)); + // From onComplete, the locks will be released for both tp0 and tp1. tp0 because it was acquired from + // tryComplete and has remote fetch processed. tp1 will be reacquired in onComplete when we check for local log read. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0, tp1)); + assertTrue(shareFetch.isCompleted()); + // Share fetch response only contains the first remote storage fetch topic partition - tp0. + assertEquals(Set.of(tp0), future.join().keySet()); + assertEquals(Errors.NONE.code(), future.join().get(tp0).errorCode()); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + static void mockTopicIdPartitionToReturnDataEqualToMinBytes(ReplicaManager replicaManager, TopicIdPartition topicIdPartition, int minBytes) { LogOffsetMetadata hwmOffsetMetadata = new LogOffsetMetadata(1, 1, minBytes); LogOffsetSnapshot endOffsetSnapshot = new LogOffsetSnapshot(1, mock(LogOffsetMetadata.class), @@ -1182,6 +1716,37 @@ private PartitionMaxBytesStrategy mockPartitionMaxBytes(Set pa return partitionMaxBytesStrategy; } + private Seq> buildLocalAndRemoteFetchResult( + Set localLogReadTopicIdPartitions, + Set remoteReadTopicIdPartitions) { + List> logReadResults = new ArrayList<>(); + localLogReadTopicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult( + new FetchDataInfo(new LogOffsetMetadata(0, 0, 0), MemoryRecords.EMPTY), + Option.empty(), + -1L, + -1L, + -1L, + -1L, + -1L, + Option.empty(), + Option.empty(), + Option.empty() + )))); + remoteReadTopicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult( + REMOTE_FETCH_INFO, + Option.empty(), + -1L, + -1L, + -1L, + -1L, + -1L, + Option.empty(), + Option.empty(), + Option.empty() + )))); + return CollectionConverters.asScala(logReadResults).toSeq(); + } + @SuppressWarnings("unchecked") private static BiConsumer mockExceptionHandler() { return mock(BiConsumer.class); @@ -1194,6 +1759,7 @@ static class DelayedShareFetchBuilder { private LinkedHashMap sharePartitions = mock(LinkedHashMap.class); private PartitionMaxBytesStrategy partitionMaxBytesStrategy = mock(PartitionMaxBytesStrategy.class); private Time time = new MockTime(); + private final Optional remoteFetch = Optional.empty(); private ShareGroupMetrics shareGroupMetrics = mock(ShareGroupMetrics.class); DelayedShareFetchBuilder withShareFetchData(ShareFetch shareFetch) { @@ -1243,7 +1809,8 @@ public DelayedShareFetch build() { sharePartitions, partitionMaxBytesStrategy, shareGroupMetrics, - time); + time, + remoteFetch); } } } diff --git a/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java b/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java index d5acaef2060b8..3bec497d7a1f2 100644 --- a/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java +++ b/core/src/test/java/kafka/server/share/ShareFetchUtilsTest.java @@ -61,7 +61,6 @@ import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.createShareAcquiredRecords; import static org.apache.kafka.server.share.fetch.ShareFetchTestUtils.memoryRecordsBuilder; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; @@ -329,7 +328,7 @@ public void testProcessFetchResponseWhenNoRecordsAreAcquired() { assertEquals(1, resultData.size()); assertTrue(resultData.containsKey(tp0)); assertEquals(0, resultData.get(tp0).partitionIndex()); - assertNull(resultData.get(tp0).records()); + assertEquals(MemoryRecords.EMPTY, resultData.get(tp0).records()); assertTrue(resultData.get(tp0).acquiredRecords().isEmpty()); assertEquals(Errors.NONE.code(), resultData.get(tp0).errorCode()); @@ -344,7 +343,7 @@ public void testProcessFetchResponseWhenNoRecordsAreAcquired() { assertEquals(1, resultData.size()); assertTrue(resultData.containsKey(tp0)); assertEquals(0, resultData.get(tp0).partitionIndex()); - assertNull(resultData.get(tp0).records()); + assertEquals(MemoryRecords.EMPTY, resultData.get(tp0).records()); assertTrue(resultData.get(tp0).acquiredRecords().isEmpty()); assertEquals(Errors.NONE.code(), resultData.get(tp0).errorCode()); diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index a69c6c83071b2..83601a0220161 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -121,7 +121,6 @@ import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; @@ -185,7 +184,7 @@ public void tearDown() throws Exception { @Test public void testNewContextReturnsFinalContextWithoutRequestData() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -212,7 +211,7 @@ public void testNewContextReturnsFinalContextWithoutRequestData() { @Test public void testNewContextReturnsFinalContextWithRequestData() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -244,7 +243,7 @@ public void testNewContextReturnsFinalContextWithRequestData() { @Test public void testNewContextReturnsFinalContextWhenTopicPartitionsArePresentInRequestData() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -275,7 +274,7 @@ public void testNewContextReturnsFinalContextWhenTopicPartitionsArePresentInRequ @Test public void testNewContext() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -372,100 +371,6 @@ public void testNewContext() { assertEquals(0, cache.size()); } - @Test - public void testShareSessionExpiration() { - ShareSessionCache cache = new ShareSessionCache(2, 1000); - sharePartitionManager = SharePartitionManagerBuilder.builder() - .withCache(cache) - .withTime(time) - .build(); - - Map topicNames = new HashMap<>(); - Uuid fooId = Uuid.randomUuid(); - topicNames.put(fooId, "foo"); - TopicIdPartition foo0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0)); - TopicIdPartition foo1 = new TopicIdPartition(fooId, new TopicPartition("foo", 1)); - - // Create a new share session, session 1 - List session1req = List.of(foo0, foo1); - - String groupId = "grp"; - ShareRequestMetadata reqMetadata1 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - - ShareFetchContext session1context = sharePartitionManager.newContext(groupId, session1req, EMPTY_PART_LIST, reqMetadata1, false); - assertInstanceOf(ShareSessionContext.class, session1context); - - LinkedHashMap respData1 = new LinkedHashMap<>(); - respData1.put(foo0, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo0.partition())); - respData1.put(foo1, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo1.partition())); - - ShareFetchResponse session1resp = session1context.updateAndGenerateResponseData(groupId, reqMetadata1.memberId(), respData1); - assertEquals(Errors.NONE, session1resp.error()); - assertEquals(2, session1resp.responseData(topicNames).size()); - - ShareSessionKey session1Key = new ShareSessionKey(groupId, reqMetadata1.memberId()); - // check share session entered into cache - assertNotNull(cache.get(session1Key)); - - time.sleep(500); - - // Create a second new share session - List session2req = List.of(foo0, foo1); - - ShareRequestMetadata reqMetadata2 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - - ShareFetchContext session2context = sharePartitionManager.newContext(groupId, session2req, EMPTY_PART_LIST, reqMetadata2, false); - assertInstanceOf(ShareSessionContext.class, session2context); - - LinkedHashMap respData2 = new LinkedHashMap<>(); - respData2.put(foo0, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo0.partition())); - respData2.put(foo1, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo1.partition())); - - ShareFetchResponse session2resp = session2context.updateAndGenerateResponseData(groupId, reqMetadata2.memberId(), respData2); - assertEquals(Errors.NONE, session2resp.error()); - assertEquals(2, session2resp.responseData(topicNames).size()); - - ShareSessionKey session2Key = new ShareSessionKey(groupId, reqMetadata2.memberId()); - - // both newly created entries are present in cache - assertNotNull(cache.get(session1Key)); - assertNotNull(cache.get(session2Key)); - - time.sleep(500); - - // Create a subsequent share fetch context for session 1 - ShareFetchContext session1context2 = sharePartitionManager.newContext(groupId, List.of(), EMPTY_PART_LIST, - new ShareRequestMetadata(reqMetadata1.memberId(), 1), true); - assertInstanceOf(ShareSessionContext.class, session1context2); - - // total sleep time will now be large enough that share session 1 will be evicted if not correctly touched - time.sleep(501); - - // create one final share session to test that the least recently used entry is evicted - // the second share session should be evicted because the first share session was incrementally fetched - // more recently than the second session was created - List session3req = List.of(foo0, foo1); - - ShareRequestMetadata reqMetadata3 = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH); - - ShareFetchContext session3context = sharePartitionManager.newContext(groupId, session3req, EMPTY_PART_LIST, reqMetadata3, false); - - LinkedHashMap respData3 = new LinkedHashMap<>(); - respData3.put(foo0, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo0.partition())); - respData3.put(foo1, new ShareFetchResponseData.PartitionData().setPartitionIndex(foo1.partition())); - - ShareFetchResponse session3resp = session3context.updateAndGenerateResponseData(groupId, reqMetadata3.memberId(), respData3); - assertEquals(Errors.NONE, session3resp.error()); - assertEquals(2, session3resp.responseData(topicNames).size()); - - ShareSessionKey session3Key = new ShareSessionKey(groupId, reqMetadata3.memberId()); - - assertNotNull(cache.get(session1Key)); - assertNull(cache.get(session2Key), "share session 2 should have been evicted by latest share session, " + - "as share session 1 was used more recently"); - assertNotNull(cache.get(session3Key)); - } - @Test public void testSubsequentShareSession() { sharePartitionManager = SharePartitionManagerBuilder.builder().build(); @@ -530,7 +435,7 @@ public void testSubsequentShareSession() { @Test public void testZeroSizeShareSession() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -576,7 +481,7 @@ public void testZeroSizeShareSession() { @Test public void testToForgetPartitions() { String groupId = "grp"; - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -614,7 +519,7 @@ public void testToForgetPartitions() { @Test public void testShareSessionUpdateTopicIdsBrokerSide() { String groupId = "grp"; - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -665,7 +570,7 @@ public void testShareSessionUpdateTopicIdsBrokerSide() { @Test public void testGetErroneousAndValidTopicIdPartitions() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -758,7 +663,7 @@ public void testGetErroneousAndValidTopicIdPartitions() { @Test public void testShareFetchContextResponseSize() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -859,7 +764,7 @@ public void testShareFetchContextResponseSize() { @Test public void testCachedTopicPartitionsWithNoTopicPartitions() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -870,7 +775,7 @@ public void testCachedTopicPartitionsWithNoTopicPartitions() { @Test public void testCachedTopicPartitionsForValidShareSessions() { - ShareSessionCache cache = new ShareSessionCache(10, 1000); + ShareSessionCache cache = new ShareSessionCache(10); sharePartitionManager = SharePartitionManagerBuilder.builder() .withCache(cache) .build(); @@ -3108,7 +3013,7 @@ static class SharePartitionManagerBuilder { private final Persister persister = new NoOpStatePersister(); private ReplicaManager replicaManager = mock(ReplicaManager.class); private Time time = new MockTime(); - private ShareSessionCache cache = new ShareSessionCache(10, 1000); + private ShareSessionCache cache = new ShareSessionCache(10); private Map partitionCacheMap = new HashMap<>(); private Timer timer = new MockTimer(); private ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 7fccbdc9e28c9..e9e476fcd1293 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -372,7 +372,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } private def createOffsetCommitRequest = { - new requests.OffsetCommitRequest.Builder( + requests.OffsetCommitRequest.Builder.forTopicNames( new OffsetCommitRequestData() .setGroupId(group) .setMemberId(JoinGroupRequest.UNKNOWN_MEMBER_ID) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 6b23b2b9d3de1..e35db63e81270 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySe import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.MetadataLogConfig -import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import scala.collection.mutable @@ -73,7 +73,6 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness { modifyConfigs(cfgs) if (isShareGroupTest()) { cfgs.foreach(_.setProperty(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, "classic,consumer,share")) - cfgs.foreach(_.setProperty(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true")) } cfgs.foreach(_.setProperty(MetadataLogConfig.METADATA_LOG_DIR_CONFIG, TestUtils.tempDir().getAbsolutePath)) insertControllerListenersIfNeeded(cfgs) diff --git a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala index 165f95e3a62a9..a6ac36ab2ae5c 100644 --- a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala +++ b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.network.SocketServer import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.utils.Utils @@ -72,7 +72,7 @@ object IntegrationTestUtils { val responseBuffer = ByteBuffer.wrap(responseBytes) ResponseHeader.parse(responseBuffer, apiKey.responseHeaderVersion(version)) - AbstractResponse.parseResponse(apiKey, responseBuffer, version) match { + AbstractResponse.parseResponse(apiKey, new ByteBufferAccessor(responseBuffer), version) match { case response: T => response case response => throw new ClassCastException(s"Expected response with type ${classTag.runtimeClass}, but found ${response.getClass}") diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 770c519dfae13..ac227893d9fd8 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -64,6 +64,7 @@ import java.util.{Collections, Optional, OptionalLong, Properties} import scala.collection.{Seq, mutable} import scala.concurrent.duration.{FiniteDuration, MILLISECONDS, SECONDS} import scala.jdk.CollectionConverters._ +import scala.util.Using @Timeout(120) @Tag("integration") @@ -1619,6 +1620,51 @@ class KRaftClusterTest { } } + /** + * Test that once a cluster is formatted, a bootstrap.metadata file that contains an unsupported + * MetadataVersion is not a problem. This is a regression test for KAFKA-19192. + */ + @Test + def testOldBootstrapMetadataFile(): Unit = { + val baseDirectory = TestUtils.tempDir().toPath() + Using.resource(new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1). + setBaseDirectory(baseDirectory). + build()). + setDeleteOnClose(false). + build() + ) { cluster => + cluster.format() + cluster.startup() + cluster.waitForReadyBrokers() + } + val oldBootstrapMetadata = BootstrapMetadata.fromRecords( + util.Arrays.asList( + new ApiMessageAndVersion( + new FeatureLevelRecord(). + setName(MetadataVersion.FEATURE_NAME). + setFeatureLevel(1), + 0.toShort) + ), + "oldBootstrapMetadata") + // Re-create the cluster using the same directory structure as above. + // Since we do not need to use the bootstrap metadata, the fact that + // it specifies an obsolete metadata.version should not be a problem. + Using.resource(new KafkaClusterTestKit.Builder( + new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(1). + setBaseDirectory(baseDirectory). + setBootstrapMetadata(oldBootstrapMetadata). + build()).build() + ) { cluster => + cluster.startup() + cluster.waitForReadyBrokers() + } + } + @Test def testIncreaseNumIoThreads(): Unit = { val cluster = new KafkaClusterTestKit.Builder( diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 5e721596ce057..67880e0ced54c 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -23,15 +23,14 @@ import org.apache.directory.api.util.FileUtils import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.OffsetOutOfRangeException import org.apache.kafka.common.utils.{Time, Utils} -import org.apache.kafka.common.{DirectoryId, KafkaException, TopicIdPartition, TopicPartition, Uuid} +import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} import org.apache.kafka.coordinator.transaction.TransactionLogConfig -import org.apache.kafka.image.{TopicImage, TopicsImage} -import org.apache.kafka.metadata.{ConfigRepository, LeaderRecoveryState, MockConfigRepository, PartitionRegistration} +import org.apache.kafka.metadata.{ConfigRepository, MockConfigRepository} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any -import org.mockito.{ArgumentCaptor, ArgumentMatchers, Mockito} +import org.mockito.{ArgumentCaptor, ArgumentMatchers} import org.mockito.Mockito.{doAnswer, doNothing, mock, never, spy, times, verify} import java.io._ @@ -39,12 +38,12 @@ import java.lang.{Long => JLong} import java.nio.file.Files import java.nio.file.attribute.PosixFilePermission import java.util -import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap, Future} +import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import java.util.{Collections, Optional, OptionalLong, Properties} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.storage.log.FetchIsolation import org.apache.kafka.server.util.{FileLock, KafkaScheduler, MockTime, Scheduler} -import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} +import org.apache.kafka.storage.internals.log.{CleanerConfig, FetchDataInfo, LogConfig, LogDirFailureChannel, LogMetricNames, LogManager => JLogManager, LogOffsetsListener, LogStartOffsetIncrementReason, ProducerStateManagerConfig, RemoteIndexCache, UnifiedLog} import org.apache.kafka.storage.internals.checkpoint.{CleanShutdownFileHandler, OffsetCheckpointFile} import org.apache.kafka.storage.log.metrics.BrokerTopicStats import org.junit.jupiter.api.function.Executable @@ -56,7 +55,6 @@ import scala.jdk.CollectionConverters._ import scala.util.{Failure, Try} class LogManagerTest { - import LogManagerTest._ val time = new MockTime() val maxRollInterval = 100 @@ -592,7 +590,7 @@ class LogManagerTest { } logManager.checkpointLogRecoveryOffsets() - val checkpoints = new OffsetCheckpointFile(new File(logDir, LogManager.RecoveryPointCheckpointFile), null).read() + val checkpoints = new OffsetCheckpointFile(new File(logDir, JLogManager.RECOVERY_POINT_CHECKPOINT_FILE), null).read() topicPartitions.zip(logs).foreach { case (tp, log) => assertEquals(checkpoints.get(tp), log.recoveryPoint, "Recovery point should equal checkpoint") @@ -672,7 +670,7 @@ class LogManagerTest { logManager.checkpointRecoveryOffsetsInDir(logDir) - val checkpoints = new OffsetCheckpointFile(new File(logDir, LogManager.RecoveryPointCheckpointFile), null).read() + val checkpoints = new OffsetCheckpointFile(new File(logDir, JLogManager.RECOVERY_POINT_CHECKPOINT_FILE), null).read() tps.zip(allLogs).foreach { case (tp, log) => assertEquals(checkpoints.get(tp), log.recoveryPoint, @@ -1094,36 +1092,6 @@ class LogManagerTest { verifyMetrics(1) } - @Test - def testWaitForAllToComplete(): Unit = { - var invokedCount = 0 - val success: Future[Boolean] = Mockito.mock(classOf[Future[Boolean]]) - Mockito.when(success.get()).thenAnswer { _ => - invokedCount += 1 - true - } - val failure: Future[Boolean] = Mockito.mock(classOf[Future[Boolean]]) - Mockito.when(failure.get()).thenAnswer{ _ => - invokedCount += 1 - throw new RuntimeException - } - - var failureCount = 0 - // all futures should be evaluated - assertFalse(LogManager.waitForAllToComplete(Seq(success, failure), _ => failureCount += 1)) - assertEquals(2, invokedCount) - assertEquals(1, failureCount) - assertFalse(LogManager.waitForAllToComplete(Seq(failure, success), _ => failureCount += 1)) - assertEquals(4, invokedCount) - assertEquals(2, failureCount) - assertTrue(LogManager.waitForAllToComplete(Seq(success, success), _ => failureCount += 1)) - assertEquals(6, invokedCount) - assertEquals(2, failureCount) - assertFalse(LogManager.waitForAllToComplete(Seq(failure, failure), _ => failureCount += 1)) - assertEquals(8, invokedCount) - assertEquals(4, failureCount) - } - @Test def testLoadDirectoryIds(): Unit = { val dirs: Seq[File] = Seq.fill(5)(TestUtils.tempDir()) @@ -1161,7 +1129,7 @@ class LogManagerTest { remoteStorageSystemEnable = true ) - val checkpointFile = new File(logDir, LogManager.LogStartOffsetCheckpointFile) + val checkpointFile = new File(logDir, JLogManager.LOG_START_OFFSET_CHECKPOINT_FILE) val checkpoint = new OffsetCheckpointFile(checkpointFile, null) val topicPartition = new TopicPartition("test", 0) val log = logManager.getOrCreateLog(topicPartition, topicId = Optional.empty) @@ -1192,7 +1160,7 @@ class LogManagerTest { @Test def testCheckpointLogStartOffsetForNormalTopic(): Unit = { - val checkpointFile = new File(logDir, LogManager.LogStartOffsetCheckpointFile) + val checkpointFile = new File(logDir, JLogManager.LOG_START_OFFSET_CHECKPOINT_FILE) val checkpoint = new OffsetCheckpointFile(checkpointFile, null) val topicPartition = new TopicPartition("test", 0) val log = logManager.getOrCreateLog(topicPartition, topicId = Optional.empty) @@ -1233,65 +1201,6 @@ class LogManagerTest { new File(dir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath, false) } - val foo0 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 0)) - val foo1 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 1)) - val bar0 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 0)) - val bar1 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 1)) - val baz0 = new TopicIdPartition(Uuid.fromString("2Ik9_5-oRDOKpSXd2SuG5w"), new TopicPartition("baz", 0)) - val baz1 = new TopicIdPartition(Uuid.fromString("2Ik9_5-oRDOKpSXd2SuG5w"), new TopicPartition("baz", 1)) - val baz2 = new TopicIdPartition(Uuid.fromString("2Ik9_5-oRDOKpSXd2SuG5w"), new TopicPartition("baz", 2)) - val quux0 = new TopicIdPartition(Uuid.fromString("YS9owjv5TG2OlsvBM0Qw6g"), new TopicPartition("quux", 0)) - val recreatedFoo0 = new TopicIdPartition(Uuid.fromString("_dOOzPe3TfiWV21Lh7Vmqg"), new TopicPartition("foo", 0)) - val recreatedFoo1 = new TopicIdPartition(Uuid.fromString("_dOOzPe3TfiWV21Lh7Vmqg"), new TopicPartition("foo", 1)) - - @Test - def testIsStrayKraftReplicaWithEmptyImage(): Unit = { - val image: TopicsImage = topicsImage(Seq()) - val onDisk = Seq(foo0, foo1, bar0, bar1, quux0).map(mockLog) - assertTrue(onDisk.forall(log => LogManager.isStrayKraftReplica(0, image, log))) - } - - @Test - def testIsStrayKraftReplicaInImage(): Unit = { - val image: TopicsImage = topicsImage(Seq( - topicImage(Map( - foo0 -> Seq(0, 1, 2), - )), - topicImage(Map( - bar0 -> Seq(0, 1, 2), - bar1 -> Seq(0, 1, 2), - )) - )) - val onDisk = Seq(foo0, foo1, bar0, bar1, quux0).map(mockLog) - val expectedStrays = Set(foo1, quux0).map(_.topicPartition()) - - onDisk.foreach(log => assertEquals(expectedStrays.contains(log.topicPartition), LogManager.isStrayKraftReplica(0, image, log))) - } - - @Test - def testIsStrayKraftReplicaInImageWithRemoteReplicas(): Unit = { - val image: TopicsImage = topicsImage(Seq( - topicImage(Map( - foo0 -> Seq(0, 1, 2), - )), - topicImage(Map( - bar0 -> Seq(1, 2, 3), - bar1 -> Seq(2, 3, 0), - )) - )) - val onDisk = Seq(foo0, bar0, bar1).map(mockLog) - val expectedStrays = Set(bar0).map(_.topicPartition) - - onDisk.foreach(log => assertEquals(expectedStrays.contains(log.topicPartition), LogManager.isStrayKraftReplica(0, image, log))) - } - - @Test - def testIsStrayKraftMissingTopicId(): Unit = { - val log = Mockito.mock(classOf[UnifiedLog]) - Mockito.when(log.topicId).thenReturn(Optional.empty) - assertTrue(LogManager.isStrayKraftReplica(0, topicsImage(Seq()), log)) - } - /** * Test LogManager takes file lock by default and the lock is released after shutdown. */ @@ -1302,12 +1211,12 @@ class LogManagerTest { try { // ${tmpLogDir}.lock is acquired by tmpLogManager - val fileLock = new FileLock(new File(tmpLogDir, LogManager.LockFileName)) + val fileLock = new FileLock(new File(tmpLogDir, JLogManager.LOCK_FILE_NAME)) assertFalse(fileLock.tryLock()) } finally { // ${tmpLogDir}.lock is removed after shutdown tmpLogManager.shutdown() - val f = new File(tmpLogDir, LogManager.LockFileName) + val f = new File(tmpLogDir, JLogManager.LOCK_FILE_NAME) assertFalse(f.exists()) } } @@ -1376,56 +1285,3 @@ class LogManagerTest { } } } - -object LogManagerTest { - def mockLog( - topicIdPartition: TopicIdPartition - ): UnifiedLog = { - val log = Mockito.mock(classOf[UnifiedLog]) - Mockito.when(log.topicId).thenReturn(Optional.of(topicIdPartition.topicId())) - Mockito.when(log.topicPartition).thenReturn(topicIdPartition.topicPartition()) - log - } - - def topicImage( - partitions: Map[TopicIdPartition, Seq[Int]] - ): TopicImage = { - var topicName: String = null - var topicId: Uuid = null - partitions.keySet.foreach { - partition => if (topicId == null) { - topicId = partition.topicId() - } else if (!topicId.equals(partition.topicId())) { - throw new IllegalArgumentException("partition topic IDs did not match") - } - if (topicName == null) { - topicName = partition.topic() - } else if (!topicName.equals(partition.topic())) { - throw new IllegalArgumentException("partition topic names did not match") - } - } - if (topicId == null) { - throw new IllegalArgumentException("Invalid empty partitions map.") - } - val partitionRegistrations = partitions.map { case (partition, replicas) => - Int.box(partition.partition()) -> new PartitionRegistration.Builder(). - setReplicas(replicas.toArray). - setDirectories(DirectoryId.unassignedArray(replicas.size)). - setIsr(replicas.toArray). - setLeader(replicas.head). - setLeaderRecoveryState(LeaderRecoveryState.RECOVERED). - setLeaderEpoch(0). - setPartitionEpoch(0). - build() - } - new TopicImage(topicName, topicId, partitionRegistrations.asJava) - } - - def topicsImage( - topics: Seq[TopicImage] - ): TopicsImage = { - var retval = TopicsImage.EMPTY - topics.foreach { t => retval = retval.including(t) } - retval - } -} diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index 4255648347cfc..3c816f635db78 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -22,7 +22,6 @@ import java.nio.channels.OverlappingFileLockException import java.nio.file.{Files, Path, StandardOpenOption} import java.util.Properties import java.util.concurrent.CompletableFuture -import kafka.log.LogManager import kafka.server.KafkaConfig import kafka.tools.TestRaftServer.ByteArraySerde import kafka.utils.TestUtils @@ -35,6 +34,7 @@ import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig} import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.fault.FaultHandler +import org.apache.kafka.storage.internals.log.LogManager import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest @@ -164,7 +164,7 @@ class RaftManagerTest { ) ) - val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LockFileName) + val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) assertTrue(fileLocked(lockPath)) raftManager.shutdown() @@ -188,7 +188,7 @@ class RaftManagerTest { ) ) - val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LockFileName) + val lockPath = metadataDir.getOrElse(logDir.head).resolve(LogManager.LOCK_FILE_NAME) assertTrue(fileLocked(lockPath)) raftManager.shutdown() diff --git a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala index b27d03919689b..4793723bc6aea 100644 --- a/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterPartitionManagerTest.scala @@ -505,7 +505,7 @@ class AlterPartitionManagerTest { null, // Response is serialized and deserialized to ensure that its does // not contain ignorable fields used by other versions. - AlterPartitionResponse.parse(MessageUtil.toByteBufferAccessor(response.data, version).buffer(), version) + AlterPartitionResponse.parse(MessageUtil.toByteBufferAccessor(response.data, version), version) ) } diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index 3a0ffe1b4779f..3ace4590aacd5 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -20,7 +20,7 @@ package kafka.server import kafka.api.IntegrationTestHarness import kafka.network.SocketServer import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.BrokerState @@ -96,7 +96,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness { val responseBuffer = ByteBuffer.wrap(responseBytes) ResponseHeader.parse(responseBuffer, apiKey.responseHeaderVersion(version)) - AbstractResponse.parseResponse(apiKey, responseBuffer, version) match { + AbstractResponse.parseResponse(apiKey, new ByteBufferAccessor(responseBuffer), version) match { case response: T => response case response => throw new ClassCastException(s"Expected response with type ${classTag.runtimeClass}, but found ${response.getClass}") diff --git a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala index 37c81ce20e508..8f5f759250b61 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala @@ -690,7 +690,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord val topicName = "foo" // Create the topic. - createTopic( + val topicId = createTopic( topic = topicName, numPartitions = 3 ) @@ -702,6 +702,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord memberId = "member-id", memberEpoch = -1, topic = topicName, + topicId = topicId, partition = 0, offset = 1000L, expectedError = Errors.NONE, @@ -765,7 +766,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -865,6 +866,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord memberId = memberId1, memberEpoch = 1, topic = "foo", + topicId = topicId, partition = partitionId, offset = 100L + 10 * version + partitionId, expectedError = Errors.NONE, @@ -1096,7 +1098,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -1164,6 +1166,7 @@ class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoord memberId = memberId1, memberEpoch = 1, topic = "foo", + topicId = topicId, partition = partitionId, offset = 100L + 10 * version + partitionId, expectedError = Errors.NONE, diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala index 88733a8657614..fe4501e640a31 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala @@ -48,7 +48,7 @@ class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinator createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -89,6 +89,7 @@ class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = 0, offset = 100L, expectedError = Errors.GROUP_ID_NOT_FOUND, diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index cf7c6bb4d3c97..4865d08dd647d 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.types.Type -import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, Errors} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord} import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader} import org.apache.kafka.common.security.auth.SecurityProtocol @@ -152,7 +152,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val responseBuffer = ByteBuffer.wrap(response) val responseHeader = ResponseHeader.parse(responseBuffer, responseHeaderVersion) - val produceResponse = ProduceResponse.parse(responseBuffer, version) + val produceResponse = ProduceResponse.parse(new ByteBufferAccessor(responseBuffer), version) assertEquals(0, responseBuffer.remaining, "The response should parse completely") assertEquals(correlationId, responseHeader.correlationId, "The correlationId should match request") diff --git a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala index dce9261f519b5..be96826858a95 100644 --- a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala @@ -19,7 +19,7 @@ package kafka.server import kafka.network.SocketServer import kafka.utils.TestUtils import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} -import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} +import org.apache.kafka.common.{TopicCollection, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection} import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse @@ -75,7 +75,7 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { protected def createTopic( topic: String, numPartitions: Int - ): Unit = { + ): Uuid = { val admin = cluster.admin() try { TestUtils.createTopicWithAdmin( @@ -85,6 +85,12 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { topic = topic, numPartitions = numPartitions ) + admin + .describeTopics(TopicCollection.ofTopicNames(List(topic).asJava)) + .allTopicNames() + .get() + .get(topic) + .topicId() } finally { admin.close() } @@ -166,18 +172,24 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { memberId: String, memberEpoch: Int, topic: String, + topicId: Uuid, partition: Int, offset: Long, expectedError: Errors, version: Short = ApiKeys.OFFSET_COMMIT.latestVersion(isUnstableApiEnabled) ): Unit = { - val request = new OffsetCommitRequest.Builder( + if (version >= 10 && topicId == Uuid.ZERO_UUID) { + throw new IllegalArgumentException(s"Cannot call OffsetCommit API version $version without a topic id") + } + + val request = OffsetCommitRequest.Builder.forTopicIdsOrNames( new OffsetCommitRequestData() .setGroupId(groupId) .setMemberId(memberId) .setGenerationIdOrMemberEpoch(memberEpoch) .setTopics(List( new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(topicId) .setName(topic) .setPartitions(List( new OffsetCommitRequestData.OffsetCommitRequestPartition() @@ -191,7 +203,8 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { val expectedResponse = new OffsetCommitResponseData() .setTopics(List( new OffsetCommitResponseData.OffsetCommitResponseTopic() - .setName(topic) + .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID) + .setName(if (version < 10) topic else "") .setPartitions(List( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(partition) @@ -670,8 +683,7 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { val shareGroupDescribeRequest = new ShareGroupDescribeRequest.Builder( new ShareGroupDescribeRequestData() .setGroupIds(groupIds.asJava) - .setIncludeAuthorizedOperations(includeAuthorizedOperations), - true + .setIncludeAuthorizedOperations(includeAuthorizedOperations) ).build(version) val shareGroupDescribeResponse = connectAndReceive[ShareGroupDescribeResponse](shareGroupDescribeRequest) @@ -751,8 +763,7 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) { .setMemberId(memberId) .setMemberEpoch(memberEpoch) .setRackId(rackId) - .setSubscribedTopicNames(subscribedTopicNames.asJava), - true + .setSubscribedTopicNames(subscribedTopicNames.asJava) ).build() // Send the request until receiving a successful response. There is a delay diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index c644043168438..fd31cbf181699 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -992,27 +992,43 @@ class KafkaApisTest extends Logging { ) } - @Test - def testHandleOffsetCommitRequest(): Unit = { - addTopicToMetadataCache("foo", numPartitions = 1) + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) + def testHandleOffsetCommitRequest(version: Short): Unit = { + val topicName = "foo" + val topicId = Uuid.randomUuid() + addTopicToMetadataCache(topicName, topicId = topicId, numPartitions = 1) val offsetCommitRequest = new OffsetCommitRequestData() .setGroupId("group") .setMemberId("member") .setTopics(List( new OffsetCommitRequestData.OffsetCommitRequestTopic() - .setName("foo") + .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID) + .setName(if (version < 10) topicName else "") + .setPartitions(List( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(10)).asJava)).asJava) + + val expectedOffsetCommitRequest = new OffsetCommitRequestData() + .setGroupId("group") + .setMemberId("member") + .setTopics(List( + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID) + .setName(topicName) .setPartitions(List( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(10)).asJava)).asJava) - val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) + val requestChannelRequest = buildRequest(OffsetCommitRequest.Builder.forTopicIdsOrNames(offsetCommitRequest, true).build(version)) val future = new CompletableFuture[OffsetCommitResponseData]() when(groupCoordinator.commitOffsets( requestChannelRequest.context, - offsetCommitRequest, + expectedOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier )).thenReturn(future) kafkaApis = createKafkaApis() @@ -1025,7 +1041,8 @@ class KafkaApisTest extends Logging { val offsetCommitResponse = new OffsetCommitResponseData() .setTopics(List( new OffsetCommitResponseData.OffsetCommitResponseTopic() - .setName("foo") + .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID) + .setName(if (version < 10) topicName else "") .setPartitions(List( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) @@ -1036,27 +1053,43 @@ class KafkaApisTest extends Logging { assertEquals(offsetCommitResponse, response.data) } - @Test - def testHandleOffsetCommitRequestFutureFailed(): Unit = { - addTopicToMetadataCache("foo", numPartitions = 1) + @ParameterizedTest + @ApiKeyVersionsSource(apiKey = ApiKeys.OFFSET_COMMIT) + def testHandleOffsetCommitRequestFutureFailed(version: Short): Unit = { + val topicName = "foo" + val topicId = Uuid.randomUuid() + addTopicToMetadataCache(topicName, topicId = topicId, numPartitions = 1) val offsetCommitRequest = new OffsetCommitRequestData() .setGroupId("group") .setMemberId("member") .setTopics(List( new OffsetCommitRequestData.OffsetCommitRequestTopic() - .setName("foo") + .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID) + .setName(if (version < 10) topicName else "") .setPartitions(List( new OffsetCommitRequestData.OffsetCommitRequestPartition() .setPartitionIndex(0) .setCommittedOffset(10)).asJava)).asJava) - val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) + val expectedOffsetCommitRequest = new OffsetCommitRequestData() + .setGroupId("group") + .setMemberId("member") + .setTopics(List( + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID) + .setName(topicName) + .setPartitions(List( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(10)).asJava)).asJava) + + val requestChannelRequest = buildRequest(OffsetCommitRequest.Builder.forTopicIdsOrNames(offsetCommitRequest, true).build(version)) val future = new CompletableFuture[OffsetCommitResponseData]() when(groupCoordinator.commitOffsets( requestChannelRequest.context, - offsetCommitRequest, + expectedOffsetCommitRequest, RequestLocal.noCaching.bufferSupplier )).thenReturn(future) @@ -1069,7 +1102,8 @@ class KafkaApisTest extends Logging { val expectedOffsetCommitResponse = new OffsetCommitResponseData() .setTopics(List( new OffsetCommitResponseData.OffsetCommitResponseTopic() - .setName("foo") + .setTopicId(if (version >= 10) topicId else Uuid.ZERO_UUID) + .setName(if (version < 10) topicName else "") .setPartitions(List( new OffsetCommitResponseData.OffsetCommitResponsePartition() .setPartitionIndex(0) @@ -1080,6 +1114,161 @@ class KafkaApisTest extends Logging { assertEquals(expectedOffsetCommitResponse, response.data) } + @Test + def testHandleOffsetCommitRequestTopicsAndPartitionsValidationWithTopicIds(): Unit = { + val fooId = Uuid.randomUuid() + val barId = Uuid.randomUuid() + val zarId = Uuid.randomUuid() + val fooName = "foo" + val barName = "bar" + addTopicToMetadataCache(fooName, topicId = fooId, numPartitions = 2) + addTopicToMetadataCache(barName, topicId = barId, numPartitions = 2) + + val offsetCommitRequest = new OffsetCommitRequestData() + .setGroupId("group") + .setMemberId("member") + .setTopics(List( + // foo exists but only has 2 partitions. + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(fooId) + .setPartitions(List( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(10), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedOffset(20), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(2) + .setCommittedOffset(30)).asJava), + // bar exists. + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(barId) + .setPartitions(List( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(40), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedOffset(50)).asJava), + // zar does not exist. + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(zarId) + .setPartitions(List( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(60), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedOffset(70)).asJava)).asJava) + + val requestChannelRequest = buildRequest(OffsetCommitRequest.Builder.forTopicIdsOrNames(offsetCommitRequest, true).build()) + + // This is the request expected by the group coordinator. + val expectedOffsetCommitRequest = new OffsetCommitRequestData() + .setGroupId("group") + .setMemberId("member") + .setTopics(List( + // foo exists but only has 2 partitions. + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(fooId) + .setName(fooName) + .setPartitions(List( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(10), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedOffset(20)).asJava), + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(barId) + .setName(barName) + .setPartitions(List( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(40), + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(1) + .setCommittedOffset(50)).asJava)).asJava) + + val future = new CompletableFuture[OffsetCommitResponseData]() + when(groupCoordinator.commitOffsets( + requestChannelRequest.context, + expectedOffsetCommitRequest, + RequestLocal.noCaching.bufferSupplier + )).thenReturn(future) + kafkaApis = createKafkaApis() + kafkaApis.handle( + requestChannelRequest, + RequestLocal.noCaching + ) + + // This is the response returned by the group coordinator. + val offsetCommitResponse = new OffsetCommitResponseData() + .setTopics(List( + new OffsetCommitResponseData.OffsetCommitResponseTopic() + .setTopicId(fooId) + .setName(fooName) + .setPartitions(List( + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code), + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(1) + .setErrorCode(Errors.NONE.code)).asJava), + new OffsetCommitResponseData.OffsetCommitResponseTopic() + .setTopicId(barId) + .setName(barName) + .setPartitions(List( + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code), + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(1) + .setErrorCode(Errors.NONE.code)).asJava)).asJava) + + val expectedOffsetCommitResponse = new OffsetCommitResponseData() + .setTopics(List( + new OffsetCommitResponseData.OffsetCommitResponseTopic() + .setTopicId(fooId) + .setPartitions(List( + // foo-2 is first because partitions failing the validation + // are put in the response first. + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(2) + .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code), + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code), + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(1) + .setErrorCode(Errors.NONE.code)).asJava), + // zar is before bar because topics failing the validation are + // put in the response first. + new OffsetCommitResponseData.OffsetCommitResponseTopic() + .setTopicId(zarId) + .setPartitions(List( + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code), + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(1) + .setErrorCode(Errors.UNKNOWN_TOPIC_ID.code)).asJava), + new OffsetCommitResponseData.OffsetCommitResponseTopic() + .setTopicId(barId) + .setPartitions(List( + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code), + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(1) + .setErrorCode(Errors.NONE.code)).asJava)).asJava) + + future.complete(offsetCommitResponse) + val response = verifyNoThrottling[OffsetCommitResponse](requestChannelRequest) + assertEquals(expectedOffsetCommitResponse, response.data) + } + @Test def testHandleOffsetCommitRequestTopicsAndPartitionsValidation(): Unit = { addTopicToMetadataCache("foo", numPartitions = 2) @@ -1123,7 +1312,7 @@ class KafkaApisTest extends Logging { .setPartitionIndex(1) .setCommittedOffset(70)).asJava)).asJava) - val requestChannelRequest = buildRequest(new OffsetCommitRequest.Builder(offsetCommitRequest).build()) + val requestChannelRequest = buildRequest(OffsetCommitRequest.Builder.forTopicNames(offsetCommitRequest).build()) // This is the request expected by the group coordinator. val expectedOffsetCommitRequest = new OffsetCommitRequestData() @@ -1226,48 +1415,6 @@ class KafkaApisTest extends Logging { assertEquals(expectedOffsetCommitResponse, response.data) } - @Test - def testOffsetCommitWithInvalidPartition(): Unit = { - val topic = "topic" - addTopicToMetadataCache(topic, numPartitions = 1) - - def checkInvalidPartition(invalidPartitionId: Int): Unit = { - reset(replicaManager, clientRequestQuotaManager, requestChannel) - - val offsetCommitRequest = new OffsetCommitRequest.Builder( - new OffsetCommitRequestData() - .setGroupId("groupId") - .setTopics(Collections.singletonList( - new OffsetCommitRequestData.OffsetCommitRequestTopic() - .setName(topic) - .setPartitions(Collections.singletonList( - new OffsetCommitRequestData.OffsetCommitRequestPartition() - .setPartitionIndex(invalidPartitionId) - .setCommittedOffset(15) - .setCommittedLeaderEpoch(RecordBatch.NO_PARTITION_LEADER_EPOCH) - .setCommittedMetadata("")) - ) - ))).build() - - val request = buildRequest(offsetCommitRequest) - when(clientRequestQuotaManager.maybeRecordAndGetThrottleTimeMs(any[RequestChannel.Request](), - any[Long])).thenReturn(0) - val kafkaApis = createKafkaApis() - try { - kafkaApis.handleOffsetCommitRequest(request, RequestLocal.withThreadConfinedCaching) - - val response = verifyNoThrottling[OffsetCommitResponse](request) - assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, - Errors.forCode(response.data.topics().get(0).partitions().get(0).errorCode)) - } finally { - kafkaApis.close() - } - } - - checkInvalidPartition(-1) - checkInvalidPartition(1) // topic has only one partition - } - @Test def testTxnOffsetCommitWithInvalidPartition(): Unit = { val topic = "topic" @@ -3994,7 +4141,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4047,9 +4193,8 @@ class KafkaApisTest extends Logging { when(sharePartitionManager.newContext(any(), any(), any(), any(), any())).thenThrow( Errors.INVALID_REQUEST.exception() - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2 - ))) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4076,7 +4221,6 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4176,7 +4320,6 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4256,7 +4399,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4303,9 +4445,8 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, partitionIndex, topicName), false)) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4332,7 +4473,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4366,9 +4506,8 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId, partitionIndex, topicName), false)) when(sharePartitionManager.newContext(any(), any(), any(), any(), any())) - .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ) + .thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -4395,7 +4534,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4449,7 +4587,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4518,7 +4655,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4608,7 +4744,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4727,10 +4862,10 @@ class KafkaApisTest extends Logging { new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 10L, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 3)) ) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( @@ -4752,7 +4887,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -4996,10 +5130,10 @@ class KafkaApisTest extends Logging { new TopicIdPartition(topicId2, new TopicPartition(topicName2, 0)), new TopicIdPartition(topicId2, new TopicPartition(topicName2, 1)) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 0L, 0L, 2)) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 0L, 0L, 3)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions1, 2)) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 2), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions2, 3)) ).thenReturn(new FinalContext()) when(sharePartitionManager.releaseSession(any(), any())).thenReturn( @@ -5075,7 +5209,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -5417,7 +5550,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -5557,7 +5689,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -5686,7 +5817,6 @@ class KafkaApisTest extends Logging { // First share fetch request is to establish the share session with the broker. kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -5836,7 +5966,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val fetchResult: Map[TopicIdPartition, ShareFetchResponseData.PartitionData] = @@ -5978,9 +6107,8 @@ class KafkaApisTest extends Logging { new ShareSessionContext(new ShareRequestMetadata(memberId, 0), util.List.of( new TopicIdPartition(topicId, partitionIndex, topicName) )) - ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), new ShareSession( - new ShareSessionKey(groupId, memberId), cachedSharePartitions, 0L, 0L, 2)) - ) + ).thenReturn(new ShareSessionContext(new ShareRequestMetadata(memberId, 1), + new ShareSession(new ShareSessionKey(groupId, memberId), cachedSharePartitions, 2))) when(clientQuotaManager.maybeRecordAndGetThrottleTimeMs( any[RequestChannel.Request](), anyDouble, anyLong)).thenReturn(0) @@ -6008,7 +6136,6 @@ class KafkaApisTest extends Logging { var request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -6092,7 +6219,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), ) kafkaApis.handleShareFetchRequest(request) @@ -6143,7 +6269,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) @@ -6207,7 +6332,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareFetchRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareFetchRequest(request) @@ -6222,7 +6346,7 @@ class KafkaApisTest extends Logging { assertEquals(partitionIndex, topicResponses.get(0).partitions.get(0).partitionIndex) assertEquals(Errors.NONE.code, topicResponses.get(0).partitions.get(0).errorCode) assertEquals(Errors.NONE.code, topicResponses.get(0).partitions.get(0).acknowledgeErrorCode) - assertNull(topicResponses.get(0).partitions.get(0).records) + assertEquals(MemoryRecords.EMPTY, topicResponses.get(0).partitions.get(0).records) assertEquals(0, topicResponses.get(0).partitions.get(0).acquiredRecords.toArray().length) } @@ -6274,7 +6398,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6321,7 +6444,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "false"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6372,7 +6494,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), authorizer = Option(authorizer), ) @@ -6424,7 +6545,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6475,7 +6595,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6524,7 +6643,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6599,7 +6717,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6662,7 +6779,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6729,7 +6845,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6797,7 +6912,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) kafkaApis.handleShareAcknowledgeRequest(request) @@ -6879,7 +6993,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicNames, erroneous) @@ -6945,7 +7058,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareFetchRequest(shareFetchRequest, topicIdNames, erroneous) @@ -7018,7 +7130,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicNames, erroneous) @@ -7085,7 +7196,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val acknowledgeBatches = kafkaApis.getAcknowledgeBatchesFromShareAcknowledgeRequest(shareAcknowledgeRequest, topicIdNames, erroneous) @@ -7157,7 +7267,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7236,7 +7345,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7316,7 +7424,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7390,7 +7497,6 @@ class KafkaApisTest extends Logging { kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val ackResult = kafkaApis.handleAcknowledgements( @@ -7487,7 +7593,6 @@ class KafkaApisTest extends Logging { val request = buildRequest(shareAcknowledgeRequest) kafkaApis = createKafkaApis( overrideProperties = Map( - ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG -> "true", ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true"), ) val response = kafkaApis.processShareAcknowledgeResponse(responseAcknowledgeData, request) @@ -9104,13 +9209,13 @@ class KafkaApisTest extends Logging { any() ) val response = capturedResponse.getValue - val buffer = MessageUtil.toByteBufferAccessor( + val readable = MessageUtil.toByteBufferAccessor( response.data, request.context.header.apiVersion - ).buffer() + ) AbstractResponse.parseResponse( request.context.header.apiKey, - buffer, + readable, request.context.header.apiVersion, ).asInstanceOf[T] } @@ -9125,10 +9230,10 @@ class KafkaApisTest extends Logging { any() ) val response = capturedResponse.getValue - val buffer = MessageUtil.toByteBufferAccessor( + val readable = MessageUtil.toByteBufferAccessor( response.data, request.context.header.apiVersion - ).buffer() + ) // Create the RequestChannel.Response that is created when sendResponse is called in order to update the metrics. val sendResponse = new RequestChannel.SendResponse( @@ -9141,7 +9246,7 @@ class KafkaApisTest extends Logging { AbstractResponse.parseResponse( request.context.header.apiKey, - buffer, + readable, request.context.header.apiVersion, ).asInstanceOf[T] } @@ -10772,7 +10877,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatReturnsUnsupportedVersion(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -10787,7 +10892,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatRequest(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() when(groupCoordinator.shareGroupHeartbeat( @@ -10812,7 +10917,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatRequestGroupAuthorizationFailed(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -10872,7 +10977,7 @@ class KafkaApisTest extends Logging { def testShareGroupHeartbeatRequestFutureFailed(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequestData().setGroupId("group") - val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupHeartbeatRequest.Builder(shareGroupHeartbeatRequest).build()) val future = new CompletableFuture[ShareGroupHeartbeatResponseData]() when(groupCoordinator.shareGroupHeartbeat( @@ -11078,7 +11183,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupResponse( + val response = getReadShareGroupStateResponse( readRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -11133,7 +11238,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupResponse( + val response = getReadShareGroupStateResponse( readRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -11181,7 +11286,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupSummaryResponse( + val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -11229,7 +11334,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getReadShareGroupSummaryResponse( + val response = getReadShareGroupStateSummaryResponse( readSummaryRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -11253,7 +11358,7 @@ class KafkaApisTest extends Logging { )) ) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build()) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build()) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11270,7 +11375,7 @@ class KafkaApisTest extends Logging { )) ) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -11298,7 +11403,7 @@ class KafkaApisTest extends Logging { util.List.of(new DescribeShareGroupOffsetsRequestGroup().setGroupId("group").setTopics(null)) ) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -11349,7 +11454,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] when(groupCoordinator.describeShareGroupOffsets( @@ -11465,7 +11570,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) // The group coordinator will only be asked for information about topics which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] @@ -11634,7 +11739,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) // The group coordinator is being asked for information about all topics, not just those which are authorized val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] @@ -11796,7 +11901,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData() .setGroups(util.List.of(describeShareGroupOffsetsRequestGroup1, describeShareGroupOffsetsRequestGroup2)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val futureGroup1 = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] when(groupCoordinator.describeShareGroupAllOffsets( @@ -11889,7 +11994,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] kafkaApis = createKafkaApis( @@ -11914,7 +12019,7 @@ class KafkaApisTest extends Logging { val describeShareGroupOffsetsRequest = new DescribeShareGroupOffsetsRequestData().setGroups(util.List.of(describeShareGroupOffsetsRequestGroup)) - val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DescribeShareGroupOffsetsRequest.Builder(describeShareGroupOffsetsRequest).build) val future = new CompletableFuture[DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseGroup] when(groupCoordinator.describeShareGroupOffsets( @@ -11943,7 +12048,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(new DeleteShareGroupOffsetsRequestTopic().setTopicName("topic-1").setPartitions(util.List.of(1)))) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest, true).build()) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build()) metadataCache = new KRaftMetadataCache(brokerId, () => KRaftVersion.KRAFT_VERSION_0) kafkaApis = createKafkaApis() kafkaApis.handle(requestChannelRequest, RequestLocal.noCaching) @@ -11958,7 +12063,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(new DeleteShareGroupOffsetsRequestTopic().setTopicName("topic-1").setPartitions(util.List.of(1)))) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build) val authorizer: Authorizer = mock(classOf[Authorizer]) when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) @@ -12015,7 +12120,7 @@ class KafkaApisTest extends Logging { .setGroupId(groupId) .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic2)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] when(groupCoordinator.deleteShareGroupOffsets( @@ -12116,7 +12221,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic1, deleteShareGroupOffsetsRequestTopic2, deleteShareGroupOffsetsRequestTopic3)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] when(groupCoordinator.deleteShareGroupOffsets( @@ -12201,7 +12306,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic1, deleteShareGroupOffsetsRequestTopic2)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) when(groupCoordinator.deleteShareGroupOffsets( requestChannelRequest.context, @@ -12243,7 +12348,7 @@ class KafkaApisTest extends Logging { .setGroupId("group") .setTopics(util.List.of(deleteShareGroupOffsetsRequestTopic1, deleteShareGroupOffsetsRequestTopic2)) - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequestData).build) val groupCoordinatorResponse: DeleteShareGroupOffsetsResponseData = new DeleteShareGroupOffsetsResponseData() .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()) @@ -12274,7 +12379,7 @@ class KafkaApisTest extends Logging { val deleteShareGroupOffsetsRequest = new DeleteShareGroupOffsetsRequestData() .setGroupId("group") - val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest, true).build) + val requestChannelRequest = buildRequest(new DeleteShareGroupOffsetsRequest.Builder(deleteShareGroupOffsetsRequest).build) val resultFuture = new CompletableFuture[DeleteShareGroupOffsetsResponseData] kafkaApis = createKafkaApis( @@ -12328,7 +12433,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getWriteShareGroupResponse( + val response = getWriteShareGroupStateResponse( writeRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -12383,7 +12488,7 @@ class KafkaApisTest extends Logging { when(authorizer.authorize(any[RequestContext], any[util.List[Action]])) .thenReturn(Seq(AuthorizationResult.DENIED).asJava, Seq(AuthorizationResult.ALLOWED).asJava) - val response = getWriteShareGroupResponse( + val response = getWriteShareGroupStateResponse( writeRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -12428,7 +12533,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getDeleteShareGroupResponse( + val response = getDeleteShareGroupStateResponse( deleteRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -12473,7 +12578,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getDeleteShareGroupResponse( + val response = getDeleteShareGroupStateResponse( deleteRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -12519,7 +12624,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getInitializeShareGroupResponse( + val response = getInitializeShareGroupStateResponse( initRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = true, @@ -12565,7 +12670,7 @@ class KafkaApisTest extends Logging { ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG -> "true", ) - val response = getInitializeShareGroupResponse( + val response = getInitializeShareGroupStateResponse( initRequestData, config ++ ShareCoordinatorTestConfig.testConfigMap().asScala, verifyNoErr = false, @@ -12586,7 +12691,7 @@ class KafkaApisTest extends Logging { describedGroups: util.List[ShareGroupDescribeResponseData.DescribedGroup]): ShareGroupDescribeResponse = { val shareGroupDescribeRequestData = new ShareGroupDescribeRequestData() shareGroupDescribeRequestData.groupIds.addAll(groupIds) - val requestChannelRequest = buildRequest(new ShareGroupDescribeRequest.Builder(shareGroupDescribeRequestData, true).build()) + val requestChannelRequest = buildRequest(new ShareGroupDescribeRequest.Builder(shareGroupDescribeRequestData).build()) val future = new CompletableFuture[util.List[ShareGroupDescribeResponseData.DescribedGroup]]() when(groupCoordinator.shareGroupDescribe( @@ -12611,10 +12716,10 @@ class KafkaApisTest extends Logging { response } - def getReadShareGroupResponse(requestData: ReadShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - readStateResult: util.List[ReadShareGroupStateResponseData.ReadStateResult]): ReadShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new ReadShareGroupStateRequest.Builder(requestData, true).build()) + def getReadShareGroupStateResponse(requestData: ReadShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + readStateResult: util.List[ReadShareGroupStateResponseData.ReadStateResult]): ReadShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new ReadShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[ReadShareGroupStateResponseData]() when(shareCoordinator.readState( @@ -12640,10 +12745,10 @@ class KafkaApisTest extends Logging { response } - def getReadShareGroupSummaryResponse(requestData: ReadShareGroupStateSummaryRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - readStateSummaryResult: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult]): ReadShareGroupStateSummaryResponse = { - val requestChannelRequest = buildRequest(new ReadShareGroupStateSummaryRequest.Builder(requestData, true).build()) + def getReadShareGroupStateSummaryResponse(requestData: ReadShareGroupStateSummaryRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + readStateSummaryResult: util.List[ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult]): ReadShareGroupStateSummaryResponse = { + val requestChannelRequest = buildRequest(new ReadShareGroupStateSummaryRequest.Builder(requestData).build()) val future = new CompletableFuture[ReadShareGroupStateSummaryResponseData]() when(shareCoordinator.readStateSummary( @@ -12669,10 +12774,10 @@ class KafkaApisTest extends Logging { response } - def getWriteShareGroupResponse(requestData: WriteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - writeStateResult: util.List[WriteShareGroupStateResponseData.WriteStateResult]): WriteShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData, true).build()) + def getWriteShareGroupStateResponse(requestData: WriteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + writeStateResult: util.List[WriteShareGroupStateResponseData.WriteStateResult]): WriteShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new WriteShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[WriteShareGroupStateResponseData]() when(shareCoordinator.writeState( @@ -12698,10 +12803,10 @@ class KafkaApisTest extends Logging { response } - def getDeleteShareGroupResponse(requestData: DeleteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - deleteStateResult: util.List[DeleteShareGroupStateResponseData.DeleteStateResult]): DeleteShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new DeleteShareGroupStateRequest.Builder(requestData, true).build()) + def getDeleteShareGroupStateResponse(requestData: DeleteShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + deleteStateResult: util.List[DeleteShareGroupStateResponseData.DeleteStateResult]): DeleteShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new DeleteShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[DeleteShareGroupStateResponseData]() when(shareCoordinator.deleteState( @@ -12727,10 +12832,10 @@ class KafkaApisTest extends Logging { response } - def getInitializeShareGroupResponse(requestData: InitializeShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, - verifyNoErr: Boolean = true, authorizer: Authorizer = null, - initStateResult: util.List[InitializeShareGroupStateResponseData.InitializeStateResult]): InitializeShareGroupStateResponse = { - val requestChannelRequest = buildRequest(new InitializeShareGroupStateRequest.Builder(requestData, true).build()) + def getInitializeShareGroupStateResponse(requestData: InitializeShareGroupStateRequestData, configOverrides: Map[String, String] = Map.empty, + verifyNoErr: Boolean = true, authorizer: Authorizer = null, + initStateResult: util.List[InitializeShareGroupStateResponseData.InitializeStateResult]): InitializeShareGroupStateResponse = { + val requestChannelRequest = buildRequest(new InitializeShareGroupStateRequest.Builder(requestData).build()) val future = new CompletableFuture[InitializeShareGroupStateResponseData]() when(shareCoordinator.initializeState( diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 74e34b06dacd1..517741cf2d869 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1026,7 +1026,6 @@ class KafkaConfigTest { case ShareGroupConfig.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case ShareGroupConfig.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) case ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) - 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_MAX_SHARE_SESSIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", 0, -1) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala index f289c241d1b4b..eceb21a407787 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala @@ -16,6 +16,7 @@ */ package kafka.server +import org.apache.kafka.common.Uuid import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.test.ClusterInstance @@ -46,7 +47,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -55,7 +56,6 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator // a session long enough for the duration of the test. val (memberId, memberEpoch) = joinConsumerGroup("grp", useNewProtocol) - // Start from version 1 because version 0 goes to ZK. for (version <- ApiKeys.OFFSET_COMMIT.oldestVersion to ApiKeys.OFFSET_COMMIT.latestVersion(isUnstableApiEnabled)) { // Commit offset. commitOffset( @@ -63,6 +63,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = 0, offset = 100L, expectedError = if (useNewProtocol && version < 9) Errors.UNSUPPORTED_VERSION else Errors.NONE, @@ -75,6 +76,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = 0, offset = 100L, expectedError = @@ -89,6 +91,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = 0, offset = 100L, expectedError = @@ -103,6 +106,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = "", memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = 0, offset = 100L, expectedError = Errors.UNKNOWN_MEMBER_ID, @@ -115,6 +119,7 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = memberId, memberEpoch = memberEpoch + 1, topic = "foo", + topicId = topicId, partition = 0, offset = 100L, expectedError = @@ -131,11 +136,27 @@ class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = "", memberEpoch = -1, topic = "foo", + topicId = topicId, partition = 0, offset = 100L, expectedError = Errors.NONE, version = version.toShort ) + + // Commit offset to a group with an unknown topic id. + if (version >= 10) { + commitOffset( + groupId = "grp", + memberId = memberId, + memberEpoch = memberEpoch, + topic = "bar", + topicId = Uuid.randomUuid(), + partition = 0, + offset = 100L, + expectedError = Errors.UNKNOWN_TOPIC_ID, + version = version.toShort + ) + } } } } diff --git a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala index c9201b24e9870..0fc414e24c99e 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala @@ -45,7 +45,7 @@ class OffsetDeleteRequestTest(cluster: ClusterInstance) extends GroupCoordinator createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -65,6 +65,7 @@ class OffsetDeleteRequestTest(cluster: ClusterInstance) extends GroupCoordinator memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = partitionId, offset = 100L + partitionId, expectedError = Errors.NONE, diff --git a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index b49de57793172..be95cef7844f9 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -71,7 +71,7 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -87,6 +87,7 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = partitionId, offset = 100L + partitionId, expectedError = Errors.NONE, @@ -239,7 +240,7 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -255,6 +256,7 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = partitionId, offset = 100L + partitionId, expectedError = Errors.NONE, @@ -348,7 +350,7 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB createOffsetsTopic() // Create the topic. - createTopic( + val topicId = createTopic( topic = "foo", numPartitions = 3 ) @@ -365,6 +367,7 @@ class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorB memberId = memberId, memberEpoch = memberEpoch, topic = "foo", + topicId = topicId, partition = partitionId, offset = 100L + partitionId, expectedError = Errors.NONE, diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 6f16f1b7a73ff..7610e46620732 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -270,7 +270,7 @@ class RequestQuotaTest extends BaseRequestTest { .setTargetTimes(List(topic).asJava) case ApiKeys.OFFSET_COMMIT => - new OffsetCommitRequest.Builder( + OffsetCommitRequest.Builder.forTopicNames( new OffsetCommitRequestData() .setGroupId("test-group") .setGenerationIdOrMemberEpoch(1) @@ -644,16 +644,16 @@ class RequestQuotaTest extends BaseRequestTest { new DescribeTopicPartitionsRequest.Builder(new DescribeTopicPartitionsRequestData()) case ApiKeys.SHARE_GROUP_HEARTBEAT => - new ShareGroupHeartbeatRequest.Builder(new ShareGroupHeartbeatRequestData(), true) + new ShareGroupHeartbeatRequest.Builder(new ShareGroupHeartbeatRequestData()) case ApiKeys.SHARE_GROUP_DESCRIBE => - new ShareGroupDescribeRequest.Builder(new ShareGroupDescribeRequestData(), true) + new ShareGroupDescribeRequest.Builder(new ShareGroupDescribeRequestData()) case ApiKeys.SHARE_FETCH => - new ShareFetchRequest.Builder(new ShareFetchRequestData(), true) + new ShareFetchRequest.Builder(new ShareFetchRequestData()) case ApiKeys.SHARE_ACKNOWLEDGE => - new ShareAcknowledgeRequest.Builder(new ShareAcknowledgeRequestData(), true) + new ShareAcknowledgeRequest.Builder(new ShareAcknowledgeRequestData()) case ApiKeys.ADD_RAFT_VOTER => new AddRaftVoterRequest.Builder(new AddRaftVoterRequestData()) @@ -665,19 +665,19 @@ class RequestQuotaTest extends BaseRequestTest { new UpdateRaftVoterRequest.Builder(new UpdateRaftVoterRequestData()) case ApiKeys.INITIALIZE_SHARE_GROUP_STATE => - new InitializeShareGroupStateRequest.Builder(new InitializeShareGroupStateRequestData(), true) + new InitializeShareGroupStateRequest.Builder(new InitializeShareGroupStateRequestData()) case ApiKeys.READ_SHARE_GROUP_STATE => - new ReadShareGroupStateRequest.Builder(new ReadShareGroupStateRequestData(), true) + new ReadShareGroupStateRequest.Builder(new ReadShareGroupStateRequestData()) case ApiKeys.WRITE_SHARE_GROUP_STATE => - new WriteShareGroupStateRequest.Builder(new WriteShareGroupStateRequestData(), true) + new WriteShareGroupStateRequest.Builder(new WriteShareGroupStateRequestData()) case ApiKeys.DELETE_SHARE_GROUP_STATE => - new DeleteShareGroupStateRequest.Builder(new DeleteShareGroupStateRequestData(), true) + new DeleteShareGroupStateRequest.Builder(new DeleteShareGroupStateRequestData()) case ApiKeys.READ_SHARE_GROUP_STATE_SUMMARY => - new ReadShareGroupStateSummaryRequest.Builder(new ReadShareGroupStateSummaryRequestData(), true) + new ReadShareGroupStateSummaryRequest.Builder(new ReadShareGroupStateSummaryRequestData()) case ApiKeys.STREAMS_GROUP_HEARTBEAT => new StreamsGroupHeartbeatRequest.Builder(new StreamsGroupHeartbeatRequestData(), true) @@ -686,13 +686,13 @@ class RequestQuotaTest extends BaseRequestTest { new StreamsGroupDescribeRequest.Builder(new StreamsGroupDescribeRequestData(), true) case ApiKeys.DESCRIBE_SHARE_GROUP_OFFSETS => - new DescribeShareGroupOffsetsRequest.Builder(new DescribeShareGroupOffsetsRequestData(), true) + new DescribeShareGroupOffsetsRequest.Builder(new DescribeShareGroupOffsetsRequestData()) case ApiKeys.ALTER_SHARE_GROUP_OFFSETS => - new AlterShareGroupOffsetsRequest.Builder(new AlterShareGroupOffsetsRequestData(), true) + new AlterShareGroupOffsetsRequest.Builder(new AlterShareGroupOffsetsRequestData()) case ApiKeys.DELETE_SHARE_GROUP_OFFSETS => - new DeleteShareGroupOffsetsRequest.Builder(new DeleteShareGroupOffsetsRequestData(), true) + new DeleteShareGroupOffsetsRequest.Builder(new DeleteShareGroupOffsetsRequestData()) case _ => throw new IllegalArgumentException("Unsupported API key " + apiKey) diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index a6b71d912b387..39327bbeaf858 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -21,7 +21,6 @@ import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils} import java.io.File import java.util.concurrent.CancellationException import kafka.integration.KafkaServerTestHarness -import kafka.log.LogManager import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.security.auth.SecurityProtocol @@ -29,6 +28,7 @@ import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerial import org.apache.kafka.common.utils.Exit import org.apache.kafka.metadata.BrokerState import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.storage.internals.log.LogManager import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.function.Executable @@ -105,7 +105,7 @@ class ServerShutdownTest extends KafkaServerTestHarness { // do a clean shutdown and check that offset checkpoint file exists shutdownBroker() for (logDir <- config.logDirs) { - val OffsetCheckpointFile = new File(logDir, LogManager.RecoveryPointCheckpointFile) + val OffsetCheckpointFile = new File(logDir, LogManager.RECOVERY_POINT_CHECKPOINT_FILE) assertTrue(OffsetCheckpointFile.exists) assertTrue(OffsetCheckpointFile.length() > 0) } diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index 9fc3165bd7b5c..45afae3a0dc13 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -44,11 +44,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo closeProducer } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") - ) - ) + @ClusterTest def testShareFetchRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) @@ -64,11 +60,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo assertEquals(0, shareFetchResponse.data.acquisitionLockTimeoutMs) } - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = "unstable.api.versions.enable", value = "true") - ) - ) + @ClusterTest def testShareAcknowledgeRequestIsInAccessibleWhenConfigsDisabled(): Unit = { val groupId: String = "group" val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) @@ -98,8 +90,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ), brokers = 2 ), @@ -153,8 +144,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) @@ -220,8 +210,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) @@ -322,8 +311,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ), brokers = 3 ), @@ -443,8 +431,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -563,7 +550,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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.record.lock.duration.ms", value = "15000") ) ), @@ -686,8 +672,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -801,8 +786,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -920,8 +904,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1038,8 +1021,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1163,7 +1145,6 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 ) ), @@ -1326,8 +1307,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1423,8 +1403,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1524,8 +1503,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1637,8 +1615,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1759,8 +1736,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1816,8 +1792,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1867,8 +1842,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -1944,8 +1918,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2026,8 +1999,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2104,8 +2076,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2187,8 +2158,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ), ) @@ -2290,8 +2260,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) @@ -2357,8 +2326,7 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo 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 = "share.coordinator.state.topic.num.partitions", value = "1") ) ) ) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala index dfd478616da30..6c707c7bae13f 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala @@ -29,7 +29,6 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Timeout @@ -42,15 +41,10 @@ import scala.jdk.CollectionConverters._ )) class ShareGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { - @ClusterTest( - serverProperties = Array( - new ClusterConfigProperty(key = ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, value = "true") - ) - ) + @ClusterTest def testShareGroupDescribeIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupDescribeRequest = new ShareGroupDescribeRequest.Builder( - new ShareGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava), - true + new ShareGroupDescribeRequestData().setGroupIds(List("grp-1", "grp-2").asJava) ).build(ApiKeys.SHARE_GROUP_DESCRIBE.latestVersion(isUnstableApiEnabled)) val shareGroupDescribeResponse = connectAndReceive[ShareGroupDescribeResponse](shareGroupDescribeRequest) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index ea9308730c3c5..75b428ee0b41c 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -42,7 +42,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { ) def testShareGroupHeartbeatIsInAccessibleWhenConfigsDisabled(): Unit = { val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder( - new ShareGroupHeartbeatRequestData(), true + new ShareGroupHeartbeatRequestData() ).build() val shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) @@ -76,8 +76,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -105,8 +104,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(shareGroupHeartbeatResponse.data.memberId) - .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch), - true + .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch) ).build() // This is the expected assignment. here @@ -132,8 +130,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(shareGroupHeartbeatResponse.data.memberId) - .setMemberEpoch(-1), - true + .setMemberEpoch(-1) ).build() shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) @@ -171,8 +168,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -195,8 +191,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the second member request until receiving a successful response. @@ -225,8 +220,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId1) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() // Heartbeats until the partitions are assigned for member 1. @@ -241,8 +235,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId1) - .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch()), - true + .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch()) ).build() false } @@ -257,8 +250,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId2) - .setMemberEpoch(2), - true + .setMemberEpoch(2) ).build() // Heartbeats until the partitions are assigned for member 2. @@ -288,8 +280,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId1) - .setMemberEpoch(3), - true + .setMemberEpoch(3) ).build() // Heartbeats until the response for no change of assignment occurs for member 1 with same epoch. @@ -333,8 +324,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -369,8 +359,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() TestUtils.waitUntilTrue(() => { @@ -387,8 +376,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberEpoch(-1) - .setMemberId(memberId), - true + .setMemberId(memberId) ).build() // Send the member request until receiving a successful response. @@ -406,8 +394,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberEpoch(0) .setMemberId(memberId) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() shareGroupHeartbeatResponse = connectAndReceive(shareGroupHeartbeatRequest) @@ -446,8 +433,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo", "bar", "baz").asJava), - true + .setSubscribedTopicNames(List("foo", "bar", "baz").asJava) ).build() // Send the request until receiving a successful response. There is a delay // here because the group coordinator is loaded in the background. @@ -487,8 +473,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() cluster.waitForTopic("foo", 2) @@ -526,8 +511,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(3), - true + .setMemberEpoch(3) ).build() TestUtils.waitUntilTrue(() => { @@ -558,8 +542,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(5), - true + .setMemberEpoch(5) ).build() TestUtils.waitUntilTrue(() => { @@ -592,8 +575,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(7), - true + .setMemberEpoch(7) ).build() TestUtils.waitUntilTrue(() => { @@ -640,8 +622,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay @@ -677,8 +658,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(memberEpoch), - true + .setMemberEpoch(memberEpoch) ).build() TestUtils.waitUntilTrue(() => { @@ -697,8 +677,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(memberId) .setMemberEpoch(memberEpoch) - .setSubscribedTopicNames(List("foo", "bar").asJava), - true + .setSubscribedTopicNames(List("foo", "bar").asJava) ).build() val barId = TestUtils.createTopicWithAdminRaw( @@ -729,7 +708,6 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(memberId) .setMemberEpoch(shareGroupHeartbeatResponse.data.memberEpoch), - true ).build() false } @@ -744,8 +722,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(memberEpoch), - true + .setMemberEpoch(memberEpoch) ).build() TestUtils.waitUntilTrue(() => { @@ -764,8 +741,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(memberEpoch), - true + .setMemberEpoch(memberEpoch) ).build() TestUtils.waitUntilTrue(() => { @@ -779,8 +755,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(memberId) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo", "bar").asJava), - true + .setSubscribedTopicNames(List("foo", "bar").asJava) ).build() TestUtils.waitUntilTrue(() => { @@ -822,8 +797,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { .setGroupId("grp") .setMemberId(Uuid.randomUuid.toString) .setMemberEpoch(0) - .setSubscribedTopicNames(List("foo").asJava), - true + .setSubscribedTopicNames(List("foo").asJava) ).build() // Send the request until receiving a successful response. There is a delay // here because the group coordinator is loaded in the background. @@ -853,8 +827,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(1), - true + .setMemberEpoch(1) ).build() TestUtils.waitUntilTrue(() => { @@ -875,8 +848,7 @@ class ShareGroupHeartbeatRequestTest(cluster: ClusterInstance) { new ShareGroupHeartbeatRequestData() .setGroupId("grp") .setMemberId(memberId) - .setMemberEpoch(2), - true + .setMemberEpoch(2) ).build() // Should receive no error and no assignment changes. diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index 24be1e921bc24..8518a469c7d78 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -220,10 +220,14 @@ class SchedulerTest { scheduler.scheduleOnce("task1", task1, 0) scheduler.scheduleOnce("task2", () => latch2.countDown(), 5) scheduler.scheduleOnce("task3", () => latch2.countDown(), 5) - assertEquals(2, scheduler.pendingTaskSize()) + retry(30000) { + assertEquals(2, scheduler.pendingTaskSize()) + } latch1.countDown() latch2.await() - assertEquals(0, scheduler.pendingTaskSize()) + retry(30000) { + assertEquals(0, scheduler.pendingTaskSize()) + } scheduler.shutdown() assertEquals(0, scheduler.pendingTaskSize()) } diff --git a/docs/ops.html b/docs/ops.html index 4d6d60c6a4dbd..f5b2a76bbea2c 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -3011,6 +3011,11 @@
GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT = List.of( Group.GroupType.CLASSIC.toString(), Group.GroupType.CONSUMER.toString()); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java index 42790515a2d20..457d090fc535e 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpers.java @@ -176,18 +176,21 @@ public static CoordinatorRecord newConsumerGroupSubscriptionMetadataTombstoneRec * * @param groupId The consumer group id. * @param newGroupEpoch The consumer group epoch. + * @param metadataHash The consumer group metadata hash. * @return The record. */ public static CoordinatorRecord newConsumerGroupEpochRecord( String groupId, - int newGroupEpoch + int newGroupEpoch, + long metadataHash ) { return CoordinatorRecord.record( new ConsumerGroupMetadataKey() .setGroupId(groupId), new ApiMessageAndVersion( new ConsumerGroupMetadataValue() - .setEpoch(newGroupEpoch), + .setEpoch(newGroupEpoch) + .setMetadataHash(metadataHash), (short) 0 ) ); @@ -647,18 +650,21 @@ public static CoordinatorRecord newShareGroupSubscriptionMetadataTombstoneRecord * * @param groupId The group id. * @param newGroupEpoch The group epoch. + * @param metadataHash The group metadata hash. * @return The record. */ public static CoordinatorRecord newShareGroupEpochRecord( String groupId, - int newGroupEpoch + int newGroupEpoch, + long metadataHash ) { return CoordinatorRecord.record( new ShareGroupMetadataKey() .setGroupId(groupId), new ApiMessageAndVersion( new ShareGroupMetadataValue() - .setEpoch(newGroupEpoch), + .setEpoch(newGroupEpoch) + .setMetadataHash(metadataHash), (short) 0 ) ); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java index 46f7e513def95..035dfdb1d52e9 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java @@ -201,16 +201,16 @@ public Builder withAuthorizerPlugin(Optional> authorizerPlugi } public GroupCoordinatorService build() { - requireNonNull(config, new IllegalArgumentException("Config must be set.")); - requireNonNull(writer, new IllegalArgumentException("Writer must be set.")); - requireNonNull(loader, new IllegalArgumentException("Loader must be set.")); - requireNonNull(time, new IllegalArgumentException("Time must be set.")); - requireNonNull(timer, new IllegalArgumentException("Timer must be set.")); - requireNonNull(coordinatorRuntimeMetrics, new IllegalArgumentException("CoordinatorRuntimeMetrics must be set.")); - requireNonNull(groupCoordinatorMetrics, new IllegalArgumentException("GroupCoordinatorMetrics must be set.")); - requireNonNull(groupConfigManager, new IllegalArgumentException("GroupConfigManager must be set.")); - requireNonNull(persister, new IllegalArgumentException("Persister must be set.")); - requireNonNull(authorizerPlugin, new IllegalArgumentException("Authorizer must be set.")); + requireNonNull(config, "Config must be set."); + requireNonNull(writer, "Writer must be set."); + requireNonNull(loader, "Loader must be set."); + requireNonNull(time, "Time must be set."); + requireNonNull(timer, "Timer must be set."); + requireNonNull(coordinatorRuntimeMetrics, "CoordinatorRuntimeMetrics must be set."); + requireNonNull(groupCoordinatorMetrics, "GroupCoordinatorMetrics must be set."); + requireNonNull(groupConfigManager, "GroupConfigManager must be set."); + requireNonNull(persister, "Persister must be set."); + requireNonNull(authorizerPlugin, "Authorizer must be set."); String logPrefix = String.format("GroupCoordinator id=%d", nodeId); LogContext logContext = new LogContext(String.format("[%s] ", logPrefix)); @@ -288,6 +288,11 @@ public GroupCoordinatorService build() { */ private final Persister persister; + /** + * The timer to schedule tasks. + */ + private final Timer timer; + /** * Boolean indicating whether the coordinator is active or not. */ @@ -305,8 +310,6 @@ public GroupCoordinatorService build() { */ private MetadataImage metadataImage = null; - private Timer timer; - /** * * @param logContext The log context. @@ -314,7 +317,8 @@ public GroupCoordinatorService build() { * @param runtime The runtime. * @param groupCoordinatorMetrics The group coordinator metrics. * @param groupConfigManager The group config manager. - * @param persister The persister + * @param persister The persister. + * @param timer The timer. */ GroupCoordinatorService( LogContext logContext, @@ -863,7 +867,7 @@ public CompletableFuture> runtime.scheduleReadOperation( "consumer-group-describe", topicPartition, - (coordinator, lastCommittedOffset) -> coordinator.consumerGroupDescribe(groupIds, lastCommittedOffset) + (coordinator, lastCommittedOffset) -> coordinator.consumerGroupDescribe(groupList, lastCommittedOffset) ).exceptionally(exception -> handleOperationException( "consumer-group-describe", groupList, @@ -915,7 +919,7 @@ public CompletableFuture> runtime.scheduleReadOperation( "streams-group-describe", topicPartition, - (coordinator, lastCommittedOffset) -> coordinator.streamsGroupDescribe(groupIds, lastCommittedOffset) + (coordinator, lastCommittedOffset) -> coordinator.streamsGroupDescribe(groupList, lastCommittedOffset) ).exceptionally(exception -> handleOperationException( "streams-group-describe", groupList, @@ -967,7 +971,7 @@ public CompletableFuture> shareGroupDescribe( runtime.scheduleReadOperation( "share-group-describe", topicPartition, - (coordinator, lastCommittedOffset) -> coordinator.shareGroupDescribe(groupIds, lastCommittedOffset) + (coordinator, lastCommittedOffset) -> coordinator.shareGroupDescribe(groupList, lastCommittedOffset) ).exceptionally(exception -> handleOperationException( "share-group-describe", groupList, @@ -1972,9 +1976,9 @@ private OffsetFetchResponseData.OffsetFetchResponseGroup handleOffsetFetchExcept } } - private static void requireNonNull(Object obj, RuntimeException throwable) { + private static void requireNonNull(Object obj, String msg) { if (obj == null) { - throw throwable; + throw new IllegalArgumentException(msg); } } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java index 2c03b2a30293c..c94381ebf6535 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java @@ -72,8 +72,10 @@ import org.apache.kafka.common.message.SyncGroupResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.requests.ConsumerGroupHeartbeatResponse; import org.apache.kafka.common.requests.JoinGroupRequest; import org.apache.kafka.common.requests.ShareGroupHeartbeatRequest; +import org.apache.kafka.common.requests.ShareGroupHeartbeatResponse; import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.utils.LogContext; @@ -186,7 +188,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -239,6 +240,11 @@ import static org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers.newShareGroupTargetAssignmentTombstoneRecord; import static org.apache.kafka.coordinator.group.Utils.assignmentToString; import static org.apache.kafka.coordinator.group.Utils.ofSentinel; +import static org.apache.kafka.coordinator.group.Utils.throwIfEmptyString; +import static org.apache.kafka.coordinator.group.Utils.throwIfNotEmptyCollection; +import static org.apache.kafka.coordinator.group.Utils.throwIfNotNull; +import static org.apache.kafka.coordinator.group.Utils.throwIfNull; +import static org.apache.kafka.coordinator.group.Utils.throwIfRegularExpressionIsInvalid; import static org.apache.kafka.coordinator.group.Utils.toConsumerProtocolAssignment; import static org.apache.kafka.coordinator.group.Utils.toTopicPartitions; import static org.apache.kafka.coordinator.group.classic.ClassicGroupMember.EMPTY_ASSIGNMENT; @@ -1412,39 +1418,6 @@ private void removeGroup( groups.remove(groupId); } - /** - * Throws an InvalidRequestException if the value is non-null and empty. - * A string containing only whitespaces is also considered empty. - * - * @param value The value. - * @param error The error message. - * @throws InvalidRequestException - */ - private static void throwIfEmptyString( - String value, - String error - ) throws InvalidRequestException { - if (value != null && value.trim().isEmpty()) { - throw new InvalidRequestException(error); - } - } - - /** - * Throws an InvalidRequestException if the value is null or non-empty. - * - * @param value The value. - * @param error The error message. - * @throws InvalidRequestException - */ - private static void throwIfNotEmptyCollection( - Collection value, - String error - ) throws InvalidRequestException { - if (value == null || !value.isEmpty()) { - throw new InvalidRequestException(error); - } - } - private static void throwIfInvalidTopology( StreamsGroupHeartbeatRequestData.Topology topology ) throws StreamsInvalidTopologyException { @@ -1460,38 +1433,6 @@ private static void throwIfInvalidTopology( } } - /** - * Throws an InvalidRequestException if the value is non-null. - * - * @param value The value. - * @param error The error message. - * @throws InvalidRequestException - */ - private static void throwIfNotNull( - Object value, - String error - ) throws InvalidRequestException { - if (value != null) { - throw new InvalidRequestException(error); - } - } - - /** - * Throws an InvalidRequestException if the value is null. - * - * @param value The value. - * @param error The error message. - * @throws InvalidRequestException - */ - private static void throwIfNull( - Object value, - String error - ) throws InvalidRequestException { - if (value == null) { - throw new InvalidRequestException(error); - } - } - /** * Validates the request. * @@ -1754,12 +1695,12 @@ private void throwIfShareGroupMemberEpochIsInvalid( + "epoch (" + receivedMemberEpoch + ") than the one known by the group coordinator (" + member.memberEpoch() + "). The member must abandon all its partitions and rejoin."); } else if (receivedMemberEpoch < member.memberEpoch()) { - // If the member comes with the previous epoch and has a subset of the current assignment partitions, - // we accept it because the response with the bumped epoch may have been lost. + // If the member comes with the previous epoch, we accept it because the response with the bumped epoch may have been lost. if (receivedMemberEpoch != member.previousMemberEpoch()) { throw new FencedMemberEpochException("The share group member has a smaller member " - + "epoch (" + receivedMemberEpoch + ") than the one known by the group coordinator (" - + member.memberEpoch() + "). The member must abandon all its partitions and rejoin."); + + "epoch (" + receivedMemberEpoch + ") than the one known by the group coordinator (" + + member.memberEpoch() + "), and it does not match the previous member epoch (" + + member.previousMemberEpoch() + "). The member must abandon all its partitions and rejoin."); } } } @@ -1974,24 +1915,6 @@ private void throwIfRebalanceInProgress( } } - /** - * Validates if the provided regular expression is valid. - * - * @param regex The regular expression to validate. - * @throws InvalidRegularExpression if the regular expression is invalid. - */ - private static void throwIfRegularExpressionIsInvalid( - String regex - ) throws InvalidRegularExpression { - try { - Pattern.compile(regex); - } catch (PatternSyntaxException ex) { - throw new InvalidRegularExpression( - String.format("SubscribedTopicRegex `%s` is not a valid regular expression: %s.", - regex, ex.getDescription())); - } - } - /** * Deserialize the subscription in JoinGroupRequestProtocolCollection. * All the protocols have the same subscription, so the method picks a random one. @@ -2011,40 +1934,6 @@ private static ConsumerProtocolSubscription deserializeSubscription( } } - private ConsumerGroupHeartbeatResponseData.Assignment createConsumerGroupResponseAssignment( - ConsumerGroupMember member - ) { - return new ConsumerGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(fromAssignmentMap(member.assignedPartitions())); - } - - private ShareGroupHeartbeatResponseData.Assignment createShareGroupResponseAssignment( - ShareGroupMember member - ) { - return new ShareGroupHeartbeatResponseData.Assignment() - .setTopicPartitions(fromShareGroupAssignmentMap(member.assignedPartitions())); - } - - private List fromAssignmentMap( - Map> assignment - ) { - return assignment.entrySet().stream() - .map(keyValue -> new ConsumerGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(keyValue.getKey()) - .setPartitions(new ArrayList<>(keyValue.getValue()))) - .toList(); - } - - private List fromShareGroupAssignmentMap( - Map> assignment - ) { - return assignment.entrySet().stream() - .map(keyValue -> new ShareGroupHeartbeatResponseData.TopicPartitions() - .setTopicId(keyValue.getKey()) - .setPartitions(new ArrayList<>(keyValue.getValue()))) - .toList(); - } - /** * Handles a regular heartbeat from a Streams group member. * It mainly consists of five parts: @@ -2180,7 +2069,7 @@ private CoordinatorResult stream int groupEpoch = group.groupEpoch(); if (bumpGroupEpoch) { groupEpoch += 1; - records.add(newStreamsGroupEpochRecord(groupId, groupEpoch)); + records.add(newStreamsGroupEpochRecord(groupId, groupEpoch, 0)); log.info("[GroupId {}][MemberId {}] Bumped streams group epoch to {}.", groupId, memberId, groupEpoch); metrics.record(STREAMS_GROUP_REBALANCES_SENSOR_NAME); group.setMetadataRefreshDeadline(currentTimeMs + METADATA_REFRESH_INTERVAL_MS, groupEpoch); @@ -2520,7 +2409,7 @@ private CoordinatorResult // 2. The member's assignment has been updated. boolean isFullRequest = rebalanceTimeoutMs != -1 && (subscribedTopicNames != null || subscribedTopicRegex != null) && ownedTopicPartitions != null; if (memberEpoch == 0 || isFullRequest || hasAssignedPartitionsChanged(member, updatedMember)) { - response.setAssignment(createConsumerGroupResponseAssignment(updatedMember)); + response.setAssignment(ConsumerGroupHeartbeatResponse.createAssignment(updatedMember.assignedPartitions())); } return new CoordinatorResult<>(records, response); @@ -2808,7 +2697,7 @@ private CoordinatorResult( @@ -3516,7 +3404,7 @@ private CoordinatorResult handleRegularExpressionsResul if (bumpGroupEpoch) { int groupEpoch = group.groupEpoch() + 1; - records.add(newConsumerGroupEpochRecord(groupId, groupEpoch)); + records.add(newConsumerGroupEpochRecord(groupId, groupEpoch, 0)); log.info("[GroupId {}] Bumped group epoch to {}.", groupId, groupEpoch); metrics.record(CONSUMER_GROUP_REBALANCES_SENSOR_NAME); group.setMetadataRefreshDeadline( @@ -3832,7 +3720,7 @@ private UpdateSubscriptionMetadataResult updateSubscriptionMetadata( if (bumpGroupEpoch) { groupEpoch += 1; - records.add(newConsumerGroupEpochRecord(groupId, groupEpoch)); + records.add(newConsumerGroupEpochRecord(groupId, groupEpoch, 0)); log.info("[GroupId {}] Bumped group epoch to {}.", groupId, groupEpoch); metrics.record(CONSUMER_GROUP_REBALANCES_SENSOR_NAME); } @@ -4091,9 +3979,13 @@ private CoordinatorResult stream String groupId, String instanceId, String memberId, - int memberEpoch + int memberEpoch, + boolean shutdownApplication ) throws ApiException { StreamsGroup group = streamsGroup(groupId); + if (shutdownApplication) { + group.setShutdownRequestMemberId(memberId); + } StreamsGroupHeartbeatResponseData response = new StreamsGroupHeartbeatResponseData() .setMemberId(memberId) .setMemberEpoch(memberEpoch); @@ -4231,7 +4123,7 @@ private CoordinatorResult consumerGroupFenceMembers( // We bump the group epoch. int groupEpoch = group.groupEpoch() + 1; - records.add(newConsumerGroupEpochRecord(group.groupId(), groupEpoch)); + records.add(newConsumerGroupEpochRecord(group.groupId(), groupEpoch, 0)); log.info("[GroupId {}] Bumped group epoch to {}.", group.groupId(), groupEpoch); for (ConsumerGroupMember member : members) { @@ -4275,7 +4167,7 @@ private CoordinatorResult shareGroupFenceMember( // We bump the group epoch. int groupEpoch = group.groupEpoch() + 1; - records.add(newShareGroupEpochRecord(group.groupId(), groupEpoch)); + records.add(newShareGroupEpochRecord(group.groupId(), groupEpoch, 0)); cancelGroupSessionTimeout(group.groupId(), member.memberId()); @@ -4338,7 +4230,7 @@ private CoordinatorResult streamsGroupFenceMember( // We bump the group epoch. int groupEpoch = group.groupEpoch() + 1; - records.add(newStreamsGroupEpochRecord(group.groupId(), groupEpoch)); + records.add(newStreamsGroupEpochRecord(group.groupId(), groupEpoch, 0)); cancelTimers(group.groupId(), member.memberId()); @@ -4846,7 +4738,8 @@ public CoordinatorResult streams request.groupId(), request.instanceId(), request.memberId(), - request.memberEpoch() + request.memberEpoch(), + request.shutdownApplication() ); } else { return streamsGroupHeartbeat( @@ -5075,9 +4968,13 @@ public List reconcileShareGroupStateInitial private Map attachTopicName(Set topicIds) { TopicsImage topicsImage = metadataImage.topics(); - return topicIds.stream() - .map(topicId -> Map.entry(topicId, topicsImage.getTopic(topicId).name())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + Map finalMap = new HashMap<>(); + for (Uuid topicId : topicIds) { + TopicImage topicImage = topicsImage.getTopic(topicId); + String topicName = (topicImage != null) ? topicImage.name() : ""; + finalMap.put(topicId, topicName); + } + return Collections.unmodifiableMap(finalMap); } private Map>> attachTopicName(Map> initMap) { @@ -5085,7 +4982,8 @@ private Map>> attachTopicName(Map>> finalMap = new HashMap<>(); for (Map.Entry> entry : initMap.entrySet()) { Uuid topicId = entry.getKey(); - String topicName = topicsImage.getTopic(topicId).name(); + TopicImage topicImage = topicsImage.getTopic(topicId); + String topicName = (topicImage != null) ? topicImage.name() : ""; finalMap.put(topicId, Map.entry(topicName, entry.getValue())); } return Collections.unmodifiableMap(finalMap); diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java index 2b50071a7f771..0fa997c557a09 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetMetadataManager.java @@ -461,7 +461,9 @@ public CoordinatorResult commitOffs final OptionalLong expireTimestampMs = expireTimestampMs(request.retentionTimeMs(), currentTimeMs); request.topics().forEach(topic -> { - final OffsetCommitResponseTopic topicResponse = new OffsetCommitResponseTopic().setName(topic.name()); + final OffsetCommitResponseTopic topicResponse = new OffsetCommitResponseTopic() + .setTopicId(topic.topicId()) + .setName(topic.name()); response.topics().add(topicResponse); topic.partitions().forEach(partition -> { @@ -470,8 +472,8 @@ public CoordinatorResult commitOffs .setPartitionIndex(partition.partitionIndex()) .setErrorCode(Errors.OFFSET_METADATA_TOO_LARGE.code())); } else { - log.debug("[GroupId {}] Committing offsets {} for partition {}-{} from member {} with leader epoch {}.", - request.groupId(), partition.committedOffset(), topic.name(), partition.partitionIndex(), + log.debug("[GroupId {}] Committing offsets {} for partition {}-{}-{} from member {} with leader epoch {}.", + request.groupId(), partition.committedOffset(), topic.topicId(), topic.name(), partition.partitionIndex(), request.memberId(), partition.committedLeaderEpoch()); topicResponse.partitions().add(new OffsetCommitResponsePartition() diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java index 5e32d17ae50ff..1736aab9d8869 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java @@ -17,6 +17,8 @@ package org.apache.kafka.coordinator.group; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.InvalidRegularExpression; +import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData; import org.apache.kafka.common.message.ConsumerProtocolAssignment; import org.apache.kafka.common.message.ConsumerProtocolSubscription; @@ -27,7 +29,11 @@ import org.apache.kafka.image.TopicsImage; import org.apache.kafka.server.common.ApiMessageAndVersion; +import com.google.re2j.Pattern; +import com.google.re2j.PatternSyntaxException; + import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -235,4 +241,87 @@ public static ApiMessage messageOrNull(ApiMessageAndVersion apiMessageAndVersion return apiMessageAndVersion.message(); } } + + /** + * Throws an InvalidRequestException if the value is non-null and empty. + * A string containing only whitespaces is also considered empty. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + static void throwIfEmptyString( + String value, + String error + ) throws InvalidRequestException { + if (value != null && value.trim().isEmpty()) { + throw new InvalidRequestException(error); + } + } + + /** + * Throws an InvalidRequestException if the value is null or non-empty. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + static void throwIfNotEmptyCollection( + Collection value, + String error + ) throws InvalidRequestException { + if (value == null || !value.isEmpty()) { + throw new InvalidRequestException(error); + } + } + + /** + * Throws an InvalidRequestException if the value is non-null. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + static void throwIfNotNull( + Object value, + String error + ) throws InvalidRequestException { + if (value != null) { + throw new InvalidRequestException(error); + } + } + + /** + * Throws an InvalidRequestException if the value is null. + * + * @param value The value. + * @param error The error message. + * @throws InvalidRequestException + */ + static void throwIfNull( + Object value, + String error + ) throws InvalidRequestException { + if (value == null) { + throw new InvalidRequestException(error); + } + } + + /** + * Validates if the provided regular expression is valid. + * + * @param regex The regular expression to validate. + * @throws InvalidRegularExpression if the regular expression is invalid. + */ + static void throwIfRegularExpressionIsInvalid( + String regex + ) throws InvalidRegularExpression { + try { + Pattern.compile(regex); + } catch (PatternSyntaxException ex) { + throw new InvalidRegularExpression( + String.format("SubscribedTopicRegex `%s` is not a valid regular expression: %s.", + regex, ex.getDescription())); + } + } } diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java index aea5bb4250026..66d7de9b5fbb3 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java @@ -1210,7 +1210,7 @@ public void createConsumerGroupRecords( records.add(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId(), consumerGroupMember)) ); - records.add(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId(), groupEpoch())); + records.add(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId(), groupEpoch(), 0)); members().forEach((consumerGroupMemberId, consumerGroupMember) -> records.add(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord( diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java index 58df34c2679cc..d686ba1a3427c 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfig.java @@ -32,7 +32,6 @@ 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 { @@ -51,10 +50,6 @@ public class ShareGroupConfig { public static final int SHARE_GROUP_DELIVERY_COUNT_LIMIT_DEFAULT = 5; public static final String SHARE_GROUP_DELIVERY_COUNT_LIMIT_DOC = "The maximum number of delivery attempts for a record delivered to a share group."; - public static final String SHARE_GROUP_MAX_GROUPS_CONFIG = "group.share.max.groups"; - public static final short SHARE_GROUP_MAX_GROUPS_DEFAULT = 10; - public static final String SHARE_GROUP_MAX_GROUPS_DOC = "The maximum number of share groups."; - public static final String SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG = "group.share.record.lock.duration.ms"; public static final int SHARE_GROUP_RECORD_LOCK_DURATION_MS_DEFAULT = 30000; public static final String SHARE_GROUP_RECORD_LOCK_DURATION_MS_DOC = "The record acquisition lock duration in milliseconds for share groups."; @@ -86,7 +81,6 @@ public class ShareGroupConfig { .define(SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG, INT, SHARE_GROUP_RECORD_LOCK_DURATION_MS_DEFAULT, between(1000, 3600000), MEDIUM, SHARE_GROUP_RECORD_LOCK_DURATION_MS_DOC) .define(SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG, INT, SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_DEFAULT, between(1000, 30000), MEDIUM, SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_DOC) .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_GROUP_MAX_SHARE_SESSIONS_CONFIG, INT, SHARE_GROUP_MAX_SHARE_SESSIONS_DEFAULT, atLeast(1), MEDIUM, SHARE_GROUP_MAX_SHARE_SESSIONS_DOC) @@ -95,17 +89,18 @@ public class ShareGroupConfig { private final boolean isShareGroupEnabled; private final int shareGroupPartitionMaxRecordLocks; private final int shareGroupDeliveryCountLimit; - private final short shareGroupMaxGroups; private final int shareGroupRecordLockDurationMs; private final int shareGroupMaxRecordLockDurationMs; private final int shareGroupMinRecordLockDurationMs; private final int shareFetchPurgatoryPurgeIntervalRequests; private final int shareGroupMaxShareSessions; private final String shareGroupPersisterClassName; + private final AbstractConfig config; public ShareGroupConfig(AbstractConfig config) { - // Share groups are enabled in two cases: - // 1. The internal configuration to enable it is explicitly set + this.config = config; + // Share groups are enabled in either of the two following cases: + // 1. The internal configuration to enable it is explicitly set; or // 2. the share rebalance protocol is enabled. Set protocols = config.getList(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG) .stream().map(String::toUpperCase).collect(Collectors.toSet()); @@ -113,7 +108,6 @@ public ShareGroupConfig(AbstractConfig config) { protocols.contains(GroupType.SHARE.name()); shareGroupPartitionMaxRecordLocks = config.getInt(ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG); shareGroupDeliveryCountLimit = config.getInt(ShareGroupConfig.SHARE_GROUP_DELIVERY_COUNT_LIMIT_CONFIG); - shareGroupMaxGroups = config.getShort(ShareGroupConfig.SHARE_GROUP_MAX_GROUPS_CONFIG); shareGroupRecordLockDurationMs = config.getInt(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG); shareGroupMaxRecordLockDurationMs = config.getInt(ShareGroupConfig.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG); shareGroupMinRecordLockDurationMs = config.getInt(ShareGroupConfig.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG); @@ -136,10 +130,6 @@ public int shareGroupDeliveryCountLimit() { return shareGroupDeliveryCountLimit; } - public short shareGroupMaxGroups() { - return shareGroupMaxGroups; - } - public int shareGroupRecordLockDurationMs() { return shareGroupRecordLockDurationMs; } @@ -171,6 +161,9 @@ private void validate() { Utils.require(shareGroupMaxRecordLockDurationMs >= shareGroupRecordLockDurationMs, String.format("%s must be greater than or equal to %s", SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG, SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG)); + Utils.require(shareGroupMaxShareSessions >= config.getInt(GroupCoordinatorConfig.SHARE_GROUP_MAX_SIZE_CONFIG), + String.format("%s must be greater than or equal to %s", + SHARE_GROUP_MAX_SHARE_SESSIONS_CONFIG, GroupCoordinatorConfig.SHARE_GROUP_MAX_SIZE_CONFIG)); } /** diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java index eb1ef06ceac34..ea002d2e1309b 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpers.java @@ -152,7 +152,8 @@ public static CoordinatorRecord newStreamsGroupPartitionMetadataTombstoneRecord( public static CoordinatorRecord newStreamsGroupEpochRecord( String groupId, - int newGroupEpoch + int newGroupEpoch, + long metadataHash ) { Objects.requireNonNull(groupId, "groupId should not be null here"); @@ -161,7 +162,8 @@ public static CoordinatorRecord newStreamsGroupEpochRecord( .setGroupId(groupId), new ApiMessageAndVersion( new StreamsGroupMetadataValue() - .setEpoch(newGroupEpoch), + .setEpoch(newGroupEpoch) + .setMetadataHash(metadataHash), (short) 0 ) ); diff --git a/group-coordinator/src/main/resources/common/message/ConsumerGroupMetadataValue.json b/group-coordinator/src/main/resources/common/message/ConsumerGroupMetadataValue.json index c01cabb521ec6..2a9bc3f4c5723 100644 --- a/group-coordinator/src/main/resources/common/message/ConsumerGroupMetadataValue.json +++ b/group-coordinator/src/main/resources/common/message/ConsumerGroupMetadataValue.json @@ -21,6 +21,13 @@ "flexibleVersions": "0+", "fields": [ { "name": "Epoch", "versions": "0+", "type": "int32", - "about": "The group epoch." } + "about": "The group epoch." }, + // The MetadataHash is added in 4.1 (KIP-1101). It's used to track + // subscribed topics in the group. When subscribed topics change, + // like partition count or rack change, the hash will be different. + // It indicates that the group should be rebalanced. + { "name": "MetadataHash", "versions": "0+", "type": "int64", + "default": 0, "taggedVersions": "0+", "tag": 0, + "about": "The hash of all topics in the group." } ] } diff --git a/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataKey.json b/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataKey.json index e4c3dc1babcfa..46a4a540f4741 100644 --- a/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataKey.json +++ b/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataKey.json @@ -16,6 +16,8 @@ { "apiKey": 4, "type": "coordinator-key", + // This message is replaced by ConsumerGroupMetadataValue#MetadataHash + // in 4.1 (KIP-1101). "name": "ConsumerGroupPartitionMetadataKey", "validVersions": "0", "flexibleVersions": "none", diff --git a/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataValue.json b/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataValue.json index 413ee101b5b96..b75477a83153c 100644 --- a/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataValue.json +++ b/group-coordinator/src/main/resources/common/message/ConsumerGroupPartitionMetadataValue.json @@ -16,6 +16,8 @@ { "apiKey": 4, "type": "coordinator-value", + // This message is replaced by ConsumerGroupMetadataValue#MetadataHash + // in 4.1 (KIP-1101). "name": "ConsumerGroupPartitionMetadataValue", "validVersions": "0", "flexibleVersions": "0+", diff --git a/group-coordinator/src/main/resources/common/message/ShareGroupMetadataValue.json b/group-coordinator/src/main/resources/common/message/ShareGroupMetadataValue.json index 5ff037a2fcc1f..661fdbdde954e 100644 --- a/group-coordinator/src/main/resources/common/message/ShareGroupMetadataValue.json +++ b/group-coordinator/src/main/resources/common/message/ShareGroupMetadataValue.json @@ -22,6 +22,8 @@ "flexibleVersions": "0+", "fields": [ { "name": "Epoch", "type": "int32", "versions": "0+", - "about": "The group epoch." } + "about": "The group epoch." }, + { "name": "MetadataHash", "versions": "0+", "type": "int64", + "about": "The hash of all topics in the group." } ] } diff --git a/group-coordinator/src/main/resources/common/message/StreamsGroupMetadataValue.json b/group-coordinator/src/main/resources/common/message/StreamsGroupMetadataValue.json index 2b4d371570aa4..0d06b0c7f49b4 100644 --- a/group-coordinator/src/main/resources/common/message/StreamsGroupMetadataValue.json +++ b/group-coordinator/src/main/resources/common/message/StreamsGroupMetadataValue.json @@ -22,6 +22,8 @@ "flexibleVersions": "0+", "fields": [ { "name": "Epoch", "versions": "0+", "type": "int32", - "about": "The group epoch." } + "about": "The group epoch." }, + { "name": "MetadataHash", "versions": "0+", "type": "int64", + "about": "The hash of all topics in the group." } ] } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupConfigTest.java index 50205b0a9e948..77014de5bf18a 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupConfigTest.java @@ -41,7 +41,6 @@ public class GroupConfigTest { private static final boolean SHARE_GROUP_ENABLE = true; private static final int SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS = 200; private static final int SHARE_GROUP_DELIVERY_COUNT_LIMIT = 5; - private static final short SHARE_GROUP_MAX_GROUPS = 10; private static final int SHARE_GROUP_RECORD_LOCK_DURATION_MS = 30000; private static final int SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS = 15000; private static final int SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS = 60000; @@ -284,6 +283,6 @@ private GroupCoordinatorConfig createGroupCoordinatorConfig() { private ShareGroupConfig createShareGroupConfig() { return ShareGroupConfigTest.createShareGroupConfig(SHARE_GROUP_ENABLE, SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS, SHARE_GROUP_DELIVERY_COUNT_LIMIT, - SHARE_GROUP_MAX_GROUPS, SHARE_GROUP_RECORD_LOCK_DURATION_MS, SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS, SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS); + SHARE_GROUP_RECORD_LOCK_DURATION_MS, SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS, SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS); } } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java index 2e67c1fa2274d..8bd0987ac4697 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorRecordHelpersTest.java @@ -45,6 +45,7 @@ import org.apache.kafka.coordinator.group.generated.OffsetCommitKey; import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; import org.apache.kafka.coordinator.group.generated.ShareGroupMetadataKey; +import org.apache.kafka.coordinator.group.generated.ShareGroupMetadataValue; import org.apache.kafka.coordinator.group.generated.ShareGroupStatePartitionMetadataKey; import org.apache.kafka.coordinator.group.generated.ShareGroupStatePartitionMetadataValue; import org.apache.kafka.coordinator.group.modern.MemberState; @@ -84,6 +85,7 @@ import static org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochTombstoneRecord; import static org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord; import static org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord; +import static org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers.newShareGroupEpochRecord; import static org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers.newShareGroupEpochTombstoneRecord; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -251,13 +253,15 @@ public void testNewConsumerGroupEpochRecord() { .setGroupId("group-id"), new ApiMessageAndVersion( new ConsumerGroupMetadataValue() - .setEpoch(10), + .setEpoch(10) + .setMetadataHash(10), (short) 0 ) ); assertEquals(expectedRecord, newConsumerGroupEpochRecord( "group-id", + 10, 10 )); } @@ -855,6 +859,26 @@ public void testNewConsumerGroupRegularExpressionTombstone() { assertEquals(expectedRecord, record); } + @Test + public void testNewShareGroupEpochRecord() { + CoordinatorRecord expectedRecord = CoordinatorRecord.record( + new ShareGroupMetadataKey() + .setGroupId("group-id"), + new ApiMessageAndVersion( + new ShareGroupMetadataValue() + .setEpoch(10) + .setMetadataHash(10), + (short) 0 + ) + ); + + assertEquals(expectedRecord, newShareGroupEpochRecord( + "group-id", + 10, + 10 + )); + } + /** * Creates a map of partitions to racks for testing. * diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java index 003d864b93762..56bc21ca2dedc 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java @@ -116,6 +116,7 @@ import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.NullSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; import java.net.InetAddress; @@ -1417,6 +1418,10 @@ public void testConsumerGroupDescribe() throws InterruptedException, ExecutionEx int partitionCount = 2; service.startup(() -> partitionCount); + @SuppressWarnings("unchecked") + ArgumentCaptor>> readOperationCaptor = + ArgumentCaptor.forClass(CoordinatorRuntime.CoordinatorReadOperation.class); + ConsumerGroupDescribeResponseData.DescribedGroup describedGroup1 = new ConsumerGroupDescribeResponseData.DescribedGroup() .setGroupId("group-id-1"); ConsumerGroupDescribeResponseData.DescribedGroup describedGroup2 = new ConsumerGroupDescribeResponseData.DescribedGroup() @@ -1429,14 +1434,14 @@ public void testConsumerGroupDescribe() throws InterruptedException, ExecutionEx when(runtime.scheduleReadOperation( ArgumentMatchers.eq("consumer-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup1))); - CompletableFuture describedGroupFuture = new CompletableFuture<>(); + CompletableFuture> describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( ArgumentMatchers.eq("consumer-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 1)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(describedGroupFuture); CompletableFuture> future = @@ -1445,6 +1450,12 @@ public void testConsumerGroupDescribe() throws InterruptedException, ExecutionEx assertFalse(future.isDone()); describedGroupFuture.complete(List.of(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); + + // Validate that the captured read operations, on the first and the second partition + GroupCoordinatorShard shard = mock(GroupCoordinatorShard.class); + readOperationCaptor.getAllValues().forEach(x -> x.generateResponse(shard, 100)); + verify(shard).consumerGroupDescribe(List.of("group-id-2"), 100); + verify(shard).consumerGroupDescribe(List.of("group-id-1"), 100); } @Test @@ -1545,6 +1556,9 @@ public void testStreamsGroupDescribe() throws InterruptedException, ExecutionExc .build(); int partitionCount = 2; service.startup(() -> partitionCount); + @SuppressWarnings("unchecked") + ArgumentCaptor>> readOperationCaptor = + ArgumentCaptor.forClass(CoordinatorRuntime.CoordinatorReadOperation.class); StreamsGroupDescribeResponseData.DescribedGroup describedGroup1 = new StreamsGroupDescribeResponseData.DescribedGroup() .setGroupId("group-id-1"); @@ -1558,14 +1572,14 @@ public void testStreamsGroupDescribe() throws InterruptedException, ExecutionExc when(runtime.scheduleReadOperation( ArgumentMatchers.eq("streams-group-describe"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup1))); - CompletableFuture describedGroupFuture = new CompletableFuture<>(); + CompletableFuture> describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( ArgumentMatchers.eq("streams-group-describe"), ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 1)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(describedGroupFuture); CompletableFuture> future = @@ -1574,6 +1588,12 @@ public void testStreamsGroupDescribe() throws InterruptedException, ExecutionExc assertFalse(future.isDone()); describedGroupFuture.complete(List.of(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); + + // Validate that the captured read operations, on the first and the second partition + GroupCoordinatorShard shard = mock(GroupCoordinatorShard.class); + readOperationCaptor.getAllValues().forEach(x -> x.generateResponse(shard, 100)); + verify(shard).streamsGroupDescribe(List.of("group-id-2"), 100); + verify(shard).streamsGroupDescribe(List.of("group-id-1"), 100); } @Test @@ -2706,6 +2726,10 @@ public void testShareGroupDescribe() throws InterruptedException, ExecutionExcep int partitionCount = 2; service.startup(() -> partitionCount); + @SuppressWarnings("unchecked") + ArgumentCaptor>> readOperationCaptor = + ArgumentCaptor.forClass(CoordinatorRuntime.CoordinatorReadOperation.class); + ShareGroupDescribeResponseData.DescribedGroup describedGroup1 = new ShareGroupDescribeResponseData.DescribedGroup() .setGroupId("share-group-id-1"); ShareGroupDescribeResponseData.DescribedGroup describedGroup2 = new ShareGroupDescribeResponseData.DescribedGroup() @@ -2718,14 +2742,14 @@ public void testShareGroupDescribe() throws InterruptedException, ExecutionExcep when(runtime.scheduleReadOperation( ArgumentMatchers.eq("share-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(CompletableFuture.completedFuture(List.of(describedGroup1))); - CompletableFuture describedGroupFuture = new CompletableFuture<>(); + CompletableFuture> describedGroupFuture = new CompletableFuture<>(); when(runtime.scheduleReadOperation( ArgumentMatchers.eq("share-group-describe"), ArgumentMatchers.eq(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 1)), - ArgumentMatchers.any() + readOperationCaptor.capture() )).thenReturn(describedGroupFuture); CompletableFuture> future = @@ -2734,6 +2758,12 @@ public void testShareGroupDescribe() throws InterruptedException, ExecutionExcep assertFalse(future.isDone()); describedGroupFuture.complete(List.of(describedGroup2)); assertEquals(expectedDescribedGroups, future.get()); + + // Validate that the captured read operations, on the first and the second partition + GroupCoordinatorShard shard = mock(GroupCoordinatorShard.class); + readOperationCaptor.getAllValues().forEach(x -> x.generateResponse(shard, 100)); + verify(shard).shareGroupDescribe(List.of("share-group-id-2"), 100); + verify(shard).shareGroupDescribe(List.of("share-group-id-1"), 100); } @Test diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java index d9af2b7052898..446af2344ef2b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java @@ -489,7 +489,7 @@ public void testConsumerGroupMemberEpochValidation() { context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, member)); - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 100)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 100, 0)); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 1, 2, 3) @@ -623,7 +623,7 @@ public void testMemberJoinsEmptyConsumerGroup() { fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) )), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), mkTopicAssignment(barTopicId, 0, 1, 2) @@ -721,7 +721,7 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() { fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) )), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), mkTopicAssignment(barTopicId, 0, 1, 2) @@ -839,7 +839,7 @@ memberId3, new MemberAssignmentImpl(mkAssignment( assertUnorderedRecordsEquals( List.of( List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember3)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)), List.of( GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), @@ -947,7 +947,7 @@ public void testLeavingMemberBumpsGroupEpoch() { fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) )), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ); assertRecordsEquals(expectedRecords, result.records()); @@ -1064,7 +1064,7 @@ memberId3, new MemberAssignmentImpl(mkAssignment( assertUnorderedRecordsEquals( List.of( List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember3)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)), List.of( GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1), @@ -1434,7 +1434,7 @@ member2RejoinId, new MemberAssignmentImpl(mkAssignment( fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) )), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, member2RejoinId, mkAssignment( mkTopicAssignment(fooTopicId, 3, 4, 5), mkTopicAssignment(barTopicId, 0, 1, 2) @@ -1627,7 +1627,7 @@ public void testLeavingStaticMemberBumpsGroupEpoch() { fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) )), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ); assertRecordsEquals(expectedRecords, result.records()); @@ -1805,7 +1805,7 @@ public void testConsumerGroupMemberEpochValidationForStaticMember() { context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, member)); - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 100)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 100, 0)); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 1, 2, 3) @@ -2592,7 +2592,7 @@ public void testConsumerGroupStates() { .setState(MemberState.STABLE) .setSubscribedTopicNames(List.of(fooTopicName)) .build())); - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)); assertEquals(ConsumerGroup.ConsumerGroupState.ASSIGNING, context.consumerGroupState(groupId)); @@ -2749,7 +2749,7 @@ public void testSubscriptionMetadataRefreshedAfterGroupIsLoaded() { GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)) ), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5) )), @@ -2873,7 +2873,7 @@ public void testSubscriptionMetadataRefreshedAgainAfterWriteFailure() { GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6)) ), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5) )), @@ -3219,7 +3219,7 @@ public void testSessionTimeoutExpiration() { GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of()), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2, 0) ) ) )), @@ -3280,7 +3280,7 @@ public void testOnLoadedSessionTimeoutExpiration() { GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of()), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ) ) ) @@ -3342,7 +3342,7 @@ public void testSessionTimeoutExpirationForShareMember() { GroupCoordinatorRecordHelpers.newShareGroupTargetAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newShareGroupMemberSubscriptionTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newShareGroupSubscriptionMetadataRecord(groupId, Map.of()), - GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 2) + GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 2, 0) ) ) )), @@ -3401,7 +3401,7 @@ public void testOnLoadedSessionTimeoutExpirationForShareMember() { GroupCoordinatorRecordHelpers.newShareGroupTargetAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newShareGroupMemberSubscriptionTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newShareGroupSubscriptionMetadataRecord(groupId, Map.of()), - GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 11, 0) ) ) ) @@ -3482,7 +3482,7 @@ public void testSessionTimeoutExpirationStaticMember() { GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId), GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of()), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2, 0) ) ) )), @@ -3758,7 +3758,7 @@ memberId2, new MemberAssignmentImpl(mkAssignment( GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId, memberId1), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId1), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId1), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 3) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 3, 0) ) ) )), @@ -8941,13 +8941,13 @@ public void testListGroups() { .setProtocol("range") .setCurrentStateTimestamp(context.time.milliseconds()))); // Create one share group record. - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(shareGroupId, 6)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(shareGroupId, 6, 0)); context.commit(); ClassicGroup classicGroup = context.groupMetadataManager.getOrMaybeCreateClassicGroup(classicGroupId, false); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(consumerGroupId, new ConsumerGroupMember.Builder(memberId1) .setSubscribedTopicNames(List.of(fooTopicName)) .build())); - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(consumerGroupId, 11)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(consumerGroupId, 11, 0)); // Test list group response without a group state or group type filter. Map actualAllGroupMap = @@ -9174,7 +9174,7 @@ public void testConsumerGroupDescribeBeforeAndAfterCommittingOffset() { ConsumerGroupMember.Builder memberBuilder1 = new ConsumerGroupMember.Builder(memberId1) .setSubscribedTopicNames(List.of(topicName)); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(consumerGroupId, memberBuilder1.build())); - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(consumerGroupId, epoch + 1)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(consumerGroupId, epoch + 1, 0)); Map> assignmentMap = Map.of(topicId, Set.of()); @@ -9182,7 +9182,7 @@ public void testConsumerGroupDescribeBeforeAndAfterCommittingOffset() { context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(consumerGroupId, memberBuilder2.build())); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(consumerGroupId, memberId2, assignmentMap)); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(consumerGroupId, memberBuilder2.build())); - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(consumerGroupId, epoch + 2)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(consumerGroupId, epoch + 2, 0)); List actual = context.groupMetadataManager.consumerGroupDescribe(List.of(consumerGroupId), context.lastCommittedOffset); ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup() @@ -9279,7 +9279,7 @@ public void testStreamsGroupDescribeBeforeAndAfterCommittingOffset() { StreamsGroupMember.Builder memberBuilder1 = streamsGroupMemberBuilderWithDefaults(memberId1); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(streamsGroupId, memberBuilder1.build())); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(streamsGroupId, memberBuilder1.build())); - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(streamsGroupId, epoch + 1)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(streamsGroupId, epoch + 1, 0)); TasksTuple assignment = new TasksTuple( Map.of(subtopologyId, Set.of(0, 1)), @@ -9291,7 +9291,7 @@ public void testStreamsGroupDescribeBeforeAndAfterCommittingOffset() { context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(streamsGroupId, memberBuilder2.build())); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(streamsGroupId, memberId2, assignment)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(streamsGroupId, memberBuilder2.build())); - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(streamsGroupId, epoch + 2)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(streamsGroupId, epoch + 2, 0)); List actual = context.groupMetadataManager.streamsGroupDescribe(List.of(streamsGroupId), context.lastCommittedOffset); StreamsGroupDescribeResponseData.DescribedGroup describedGroup = new StreamsGroupDescribeResponseData.DescribedGroup() @@ -10169,7 +10169,7 @@ public void testConsumerGroupHeartbeatWithEmptyClassicGroup() { List.of( GroupCoordinatorRecordHelpers.newGroupMetadataTombstoneRecord(classicGroupId), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(classicGroupId, expectedMember), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(classicGroupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(classicGroupId, 1, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(classicGroupId, memberId, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(classicGroupId, 1), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(classicGroupId, expectedMember) @@ -10333,7 +10333,7 @@ memberId2, new MemberAssignmentImpl(mkAssignment( // Create the new consumer group with member 1. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 0), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember1), @@ -10348,7 +10348,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) )), // Newly joining member 2 bumps the group epoch. A new target assignment is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, assignor.targetPartitions(memberId2)), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, assignor.targetPartitions(memberId1)), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), @@ -10548,7 +10548,7 @@ memberId3, new MemberAssignmentImpl(mkAssignment( GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, expectedMember2.assignedPartitions()), @@ -10567,7 +10567,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) )), // Newly joining member 3 bumps the group epoch. A new target assignment is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, assignor.targetPartitions(memberId1)), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId3, assignor.targetPartitions(memberId3)), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), @@ -10824,7 +10824,7 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() { // Create the new consumer group with the static member. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedClassicMember), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 0, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, expectedClassicMember.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 0), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedClassicMember), @@ -10846,7 +10846,7 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() { GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 1))), // Newly joining static member bumps the group epoch. A new target assignment is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId, mkAssignment(mkTopicAssignment(fooTopicId, 0))), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), @@ -10950,7 +10950,7 @@ public void testConsumerGroupHeartbeatToClassicGroupWithEmptyAssignmentMember() // Create the new consumer group with member 1. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentRecord(groupId, expectedMember1), @@ -10965,7 +10965,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) )), // Newly joining member 2 bumps the group epoch. A new target assignment is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 2), @@ -11335,7 +11335,7 @@ memberId3, new MemberAssignmentImpl(mkAssignment( GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember2), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, expectedMember1.assignedPartitions()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId2, expectedMember2.assignedPartitions()), @@ -11354,7 +11354,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) )), // Newly joining member 3 bumps the group epoch. A new target assignment is computed. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, assignor.targetPartitions(memberId1)), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId3, assignor.targetPartitions(memberId3)), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 2), @@ -12432,7 +12432,7 @@ memberId, new MemberAssignmentImpl(mkAssignment( fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) ))), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)), List.of( GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId, assignor.targetPartitions(memberId)), @@ -12593,7 +12593,7 @@ public void testJoiningConsumerGroupWithNewStaticMember() throws Exception { fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) )), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, newMemberId, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11), @@ -12849,7 +12849,7 @@ memberId2, new MemberAssignmentImpl(mkAssignment( assertUnorderedRecordsEquals( List.of( List.of(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember)), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)), List.of( GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, mkAssignment( mkTopicAssignment(fooTopicId, 0), @@ -13080,7 +13080,7 @@ fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), barTopicName, new TopicMetadata(barTopicId, barTopicName, 1), zarTopicName, new TopicMetadata(zarTopicId, zarTopicName, 1) ))), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)), List.of( GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, mkAssignment( @@ -13315,7 +13315,7 @@ fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2), barTopicName, new TopicMetadata(barTopicId, barTopicName, 1), zarTopicName, new TopicMetadata(zarTopicId, zarTopicName, 1) ))), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)), + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)), List.of( GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, mkAssignment( @@ -14125,7 +14125,7 @@ public void testConsumerGroupMemberUsingClassicProtocolFencedWhenSessionTimeout( GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId), // The group epoch is bumped. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ), timeout.result.records() ); @@ -14190,7 +14190,7 @@ public void testConsumerGroupMemberUsingClassicProtocolFencedWhenJoinTimeout() { GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId), // The group epoch is bumped. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ), timeout.result.records() ); @@ -14400,7 +14400,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) // Update subscription metadata. List.of(GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of())), // Bump the group epoch. - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)) + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)) ); assertUnorderedRecordsEquals(expectedRecords, leaveResult.records()); @@ -14510,7 +14510,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 1) Map.of(fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 2)) ), // Bump the group epoch. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ); assertEquals(expectedRecords, leaveResult.records()); } @@ -14916,8 +14916,8 @@ public void testShareGroupDescribeNoErrors() { )); List groupIds = List.of("group-id-1", "group-id-2"); - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupIds.get(0), 100)); - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupIds.get(1), 15)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupIds.get(0), 100, 0)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupIds.get(1), 15, 0)); Uuid topicId = Uuid.randomUuid(); String topicName = "foo"; @@ -15224,7 +15224,7 @@ public void testShareGroupMemberJoinsEmptyGroupWithAssignments() { fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) )), - GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 1, 0), GroupCoordinatorRecordHelpers.newShareGroupTargetAssignmentRecord(groupId, memberId, mkAssignment( mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5), mkTopicAssignment(barTopicId, 0, 1, 2) @@ -15346,7 +15346,7 @@ public void testShareGroupLeavingMemberBumpsGroupEpoch() { fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) )), - GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 11, 0) ); assertRecordsEquals(expectedRecords, result.records()); @@ -15371,7 +15371,7 @@ public void testShareGroupNewMemberIsRejectedWithMaximumMembersIsReached() { Map.of() )); - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 100)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 100, 0)); Uuid fooTopicId = Uuid.randomUuid(); String fooTopicName = "foo"; @@ -15451,7 +15451,7 @@ public void testShareGroupStates() { .withShareGroup(new ShareGroupBuilder(groupId, 10)) .build(); - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 10)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 10, 0)); assertEquals(ShareGroup.ShareGroupState.EMPTY, context.shareGroupState(groupId)); @@ -15459,7 +15459,7 @@ public void testShareGroupStates() { .setState(MemberState.STABLE) .setSubscribedTopicNames(List.of(fooTopicName)) .build())); - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 11)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 11, 0)); assertEquals(ShareGroup.ShareGroupState.STABLE, context.shareGroupState(groupId)); @@ -15740,7 +15740,7 @@ public void testStreamsGroupMemberEpochValidation() { context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, member)); - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 100)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 100, 0)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord(groupId, topology)); @@ -15927,7 +15927,7 @@ public void testMemberJoinsEmptyStreamsGroup() { fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) )), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), @@ -16010,7 +16010,7 @@ public void testStreamsGroupMemberJoiningWithMissingSourceTopic() { fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6) ) ), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TasksTuple.EMPTY), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 1), StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) @@ -16094,7 +16094,7 @@ public void testStreamsGroupMemberJoiningWithMissingInternalTopic() { StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, Map.of( fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6) )), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TasksTuple.EMPTY), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 1), StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) @@ -16176,7 +16176,7 @@ public void testStreamsGroupMemberJoiningWithIncorrectlyPartitionedTopic() { fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) )), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 1, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TasksTuple.EMPTY), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 1), StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) @@ -16270,7 +16270,7 @@ public void testStreamsGroupMemberJoiningWithStaleTopology() { fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, 3) )), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TasksTuple.EMPTY), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentEpochRecord(groupId, 11), StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentRecord(groupId, expectedMember) @@ -16371,6 +16371,97 @@ public void testStreamsGroupMemberRequestingShutdownApplication() { assertRecordsEquals(List.of(), result2.records()); } + @Test + public void testStreamsGroupMemberRequestingShutdownApplicationUponLeaving() { + String groupId = "fooup"; + String memberId1 = Uuid.randomUuid().toString(); + String memberId2 = Uuid.randomUuid().toString(); + String subtopology1 = "subtopology1"; + String fooTopicName = "foo"; + Uuid fooTopicId = Uuid.randomUuid(); + Topology topology = new Topology().setSubtopologies(List.of( + new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName)) + )); + + MockTaskAssignor assignor = new MockTaskAssignor("sticky"); + GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder() + .withStreamsGroupTaskAssignors(List.of(assignor)) + .withMetadataImage(new MetadataImageBuilder() + .addTopic(fooTopicId, fooTopicName, 2) + .build()) + .withStreamsGroup(new StreamsGroupBuilder(groupId, 10) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId1) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .build()) + .withMember(streamsGroupMemberBuilderWithDefaults(memberId2) + .setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE) + .setMemberEpoch(10) + .setPreviousMemberEpoch(9) + .build()) + .withTargetAssignmentEpoch(10) + .withTopology(StreamsTopology.fromHeartbeatRequest(topology)) + .withPartitionMetadata(Map.of( + fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6) + )) + ) + .build(); + + CoordinatorResult result1 = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId1) + .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH) + .setShutdownApplication(true) + ); + + String statusDetail = String.format("Streams group member %s encountered a fatal error and requested a shutdown for the entire application.", memberId1); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId1) + .setMemberEpoch(LEAVE_GROUP_MEMBER_EPOCH), + result1.response().data() + ); + assertRecordsEquals( + List.of( + StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord(groupId, memberId1), + StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord(groupId, memberId1), + StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord(groupId, memberId1), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11) + ), + result1.records() + ); + + for (CoordinatorRecord record : result1.records()) { + context.replay(record); + } + assignor.prepareGroupAssignment( + Map.of(memberId1, TasksTuple.EMPTY) + ); + + CoordinatorResult result2 = context.streamsGroupHeartbeat( + new StreamsGroupHeartbeatRequestData() + .setGroupId(groupId) + .setMemberId(memberId2) + .setMemberEpoch(10) + ); + + assertResponseEquals( + new StreamsGroupHeartbeatResponseData() + .setMemberId(memberId2) + .setMemberEpoch(12) + .setHeartbeatIntervalMs(5000) + .setStatus(List.of( + new StreamsGroupHeartbeatResponseData.Status() + .setStatusCode(Status.SHUTDOWN_APPLICATION.code()) + .setStatusDetail(statusDetail) + )), + result2.response().data() + ); + } + @Test public void testStreamsUpdatingMemberMetadataTriggersNewTargetAssignment() { String groupId = "fooup"; @@ -16457,7 +16548,7 @@ public void testStreamsUpdatingMemberMetadataTriggersNewTargetAssignment() { List expectedRecords = List.of( StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, expectedMember), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), @@ -16559,7 +16650,7 @@ public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() { fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(barTopicId, barTopicName, changedPartitionCount) )), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5), @@ -16733,7 +16824,7 @@ public void testStreamsLeavingMemberRemovesMemberAndBumpsGroupEpoch() { StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord(groupId, memberId2), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord(groupId, memberId2), StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord(groupId, memberId2), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11) + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11, 0) ); assertRecordsEquals(expectedRecords, result.records()); @@ -17278,7 +17369,7 @@ public void testStreamsStreamsGroupStates() { context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecord(groupId, topology)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberRecord(groupId, streamsGroupMemberBuilderWithDefaults(memberId1) .build())); - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11, 0)); assertEquals(StreamsGroupState.NOT_READY, context.streamsGroupState(groupId)); @@ -17441,7 +17532,7 @@ public void testStreamsPartitionMetadataRefreshedAfterGroupIsLoaded() { StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, Map.of(fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6)) ), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) @@ -17557,7 +17648,7 @@ public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() { StreamsCoordinatorRecordHelpers.newStreamsGroupPartitionMetadataRecord(groupId, Map.of(fooTopicName, new org.apache.kafka.coordinator.group.streams.TopicMetadata(fooTopicId, fooTopicName, 6)) ), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11), + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 11, 0), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentRecord(groupId, memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE, TaskAssignmentTestUtil.mkTasks(subtopology1, 0, 1, 2, 3, 4, 5) @@ -17702,7 +17793,7 @@ public void testStreamsSessionTimeoutExpiration() { StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord(groupId, memberId), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord(groupId, memberId), StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord(groupId, memberId), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 2) + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 2, 0) ) ) )), @@ -17997,7 +18088,7 @@ public void testStreamsRebalanceTimeoutExpiration() { StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord(groupId, memberId1), StreamsCoordinatorRecordHelpers.newStreamsGroupTargetAssignmentTombstoneRecord(groupId, memberId1), StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord(groupId, memberId1), - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 3) + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, 3, 0) ) ) )), @@ -18448,7 +18539,7 @@ public void testReplayConsumerGroupMemberMetadataTombstone() { // The group still exists but the member is already gone. Replaying the // ConsumerGroupMemberMetadata tombstone should be a no-op. - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord("foo", 10)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord("foo", 10, 0)); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord("foo", "m1")); assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.consumerGroup("foo").getOrMaybeCreateMember("m1", false)); @@ -18464,7 +18555,7 @@ public void testReplayConsumerGroupMetadata() { .build(); // The group is created if it does not exist. - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord("foo", 10)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord("foo", 10, 0)); assertEquals(10, context.groupMetadataManager.consumerGroup("foo").groupEpoch()); } @@ -18576,7 +18667,7 @@ public void testReplayConsumerGroupCurrentMemberAssignmentTombstone() { // The group still exists but the member is already gone. Replaying the // ConsumerGroupCurrentMemberAssignment tombstone should be a no-op. - context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord("foo", 10)); + context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord("foo", 10, 0)); context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord("foo", "m1")); assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.consumerGroup("foo").getOrMaybeCreateMember("m1", false)); @@ -18615,7 +18706,7 @@ public void testReplayStreamsGroupMemberMetadataTombstoneNotExisting() { // The group still exists but the member is already gone. Replaying the // StreamsGroupMemberMetadata tombstone should be a no-op. - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10, 0)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupMemberTombstoneRecord("foo", "m1")); assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.streamsGroup("foo").getMemberOrThrow("m1")); @@ -18671,7 +18762,7 @@ public void testReplayStreamsGroupMetadata() { .build(); // The group is created if it does not exist. - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10, 0)); assertEquals(10, context.groupMetadataManager.streamsGroup("foo").groupEpoch()); } @@ -18902,7 +18993,7 @@ public void testReplayStreamsGroupCurrentMemberAssignmentTombstoneNotExisting() // The group still exists, but the member is already gone. Replaying the // StreamsGroupCurrentMemberAssignment tombstone should be a no-op. - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10, 0)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupCurrentAssignmentTombstoneRecord("foo", "m1")); assertThrows(UnknownMemberIdException.class, () -> context.groupMetadataManager.streamsGroup("foo").getMemberOrThrow("m1")); @@ -18978,7 +19069,7 @@ public void testReplayStreamsGroupTopologyTombstoneNotExists() { // The group still exists, but the member is already gone. Replaying the // StreamsGroupTopology tombstone should be a no-op. - context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10)); + context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord("foo", 10, 0)); context.replay(StreamsCoordinatorRecordHelpers.newStreamsGroupTopologyRecordTombstone("foo")); assertTrue(context.groupMetadataManager.streamsGroup("foo").topology().isEmpty()); @@ -19821,7 +19912,7 @@ public void testConsumerGroupMemberJoinsWithNewRegex() { ) ), // The group epoch is bumped. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) )) ) ), @@ -19937,7 +20028,7 @@ public void testConsumerGroupMemberJoinsWithUpdatedRegex() { ) ), // The group epoch is bumped. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ), task.result.records() ); @@ -19999,7 +20090,7 @@ public void testConsumerGroupMemberJoinsWithRegexAndUpdatesItBeforeResolutionCom // The member subscription is created. GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId, expectedMember1), // The group epoch is bumped. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1), + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 1, 0), // The target assignment is created. GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentRecord(groupId, memberId1, Map.of()), GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 1), @@ -20107,7 +20198,7 @@ public void testConsumerGroupMemberJoinsWithRegexAndUpdatesItBeforeResolutionCom ) ), // The group epoch is bumped. - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 2, 0) ), task.result.records() ); @@ -20249,7 +20340,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 3), foooTopicName, new TopicMetadata(foooTopicId, foooTopicName, 1) ) )), - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)) + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)) ), task.result.records() ); @@ -20386,7 +20477,7 @@ fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6))) context.time.milliseconds() ) ), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) )) ) ), @@ -20476,7 +20567,7 @@ fooTopicName, new TopicMetadata(fooTopicId, fooTopicName, 6), barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) ) ), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 12) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 12, 0) ), context.processTasks().get(0).result.records() ); @@ -20566,7 +20657,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 3))) GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of(barTopicName, new TopicMetadata(barTopicId, barTopicName, 3)) ), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0) ); assertRecordsEquals(expectedRecords, result.records()); @@ -20586,7 +20677,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 3))) GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId2), GroupCoordinatorRecordHelpers.newConsumerGroupRegularExpressionTombstone(groupId, "bar*"), GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataRecord(groupId, Map.of()), - GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 12) + GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 12, 0) ) ) )), @@ -20740,7 +20831,7 @@ barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) barTopicName, new TopicMetadata(barTopicId, barTopicName, 3) ))), // Bumped epoch. - List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11)) + List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, 0)) ), result.records() ); @@ -20772,7 +20863,7 @@ public void testShareGroupDeleteRequestNoDeletingTopics() { MetadataDelta delta = new MetadataDelta(image); context.groupMetadataManager.onNewMetadataImage(image, delta); - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0)); context.replay( GroupCoordinatorRecordHelpers.newShareGroupStatePartitionMetadataRecord( @@ -20839,7 +20930,7 @@ public void testShareGroupDeleteRequestWithAlreadyDeletingTopics() { MetadataDelta delta = new MetadataDelta(image); context.groupMetadataManager.onNewMetadataImage(image, delta); - context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0)); + context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0)); context.replay( GroupCoordinatorRecordHelpers.newShareGroupStatePartitionMetadataRecord( diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java index 6f788d84fd009..382b2a9b0e571 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java @@ -1308,6 +1308,75 @@ public void testConsumerGroupOffsetCommit() { ); } + @Test + public void testConsumerGroupOffsetCommitWithTopicIds() { + Uuid topicId = Uuid.randomUuid(); + OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder().build(); + + // Create an empty group. + ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreatePersistedConsumerGroup( + "foo", + true + ); + + // Add member. + group.updateMember(new ConsumerGroupMember.Builder("member") + .setMemberEpoch(10) + .setPreviousMemberEpoch(10) + .build() + ); + + CoordinatorResult result = context.commitOffset( + new OffsetCommitRequestData() + .setGroupId("foo") + .setMemberId("member") + .setGenerationIdOrMemberEpoch(10) + .setTopics(List.of( + new OffsetCommitRequestData.OffsetCommitRequestTopic() + .setTopicId(topicId) + .setName("bar") + .setPartitions(List.of( + new OffsetCommitRequestData.OffsetCommitRequestPartition() + .setPartitionIndex(0) + .setCommittedOffset(100L) + .setCommittedLeaderEpoch(10) + .setCommittedMetadata("metadata") + )) + )) + ); + + assertEquals( + new OffsetCommitResponseData() + .setTopics(List.of( + new OffsetCommitResponseData.OffsetCommitResponseTopic() + .setTopicId(topicId) + .setName("bar") + .setPartitions(List.of( + new OffsetCommitResponseData.OffsetCommitResponsePartition() + .setPartitionIndex(0) + .setErrorCode(Errors.NONE.code()) + )) + )), + result.response() + ); + + assertEquals( + List.of(GroupCoordinatorRecordHelpers.newOffsetCommitRecord( + "foo", + "bar", + 0, + new OffsetAndMetadata( + 100L, + OptionalInt.of(10), + "metadata", + context.time.milliseconds(), + OptionalLong.empty() + ) + )), + result.records() + ); + } + @Test public void testConsumerGroupOffsetCommitWithOffsetMetadataTooLarge() { OffsetMetadataManagerTestContext context = new OffsetMetadataManagerTestContext.Builder() diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupBuilder.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupBuilder.java index 800073f42bc9f..7da6e5e28dd7b 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupBuilder.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupBuilder.java @@ -108,7 +108,7 @@ public List build(TopicsImage topicsImage) { } // Add group epoch record. - records.add(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, groupEpoch)); + records.add(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, groupEpoch, 0)); // Add target assignment records. assignments.forEach((memberId, assignment) -> diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java index 2ae2996143400..8298358231022 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java @@ -94,7 +94,7 @@ public List build(TopicsImage topicsImage) { } // Add group epoch record. - records.add(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, groupEpoch)); + records.add(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, groupEpoch, 0)); // Add target assignment records. assignments.forEach((memberId, assignment) -> diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java index 0a2f57b6ff660..698dd59380c67 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupConfigTest.java @@ -39,7 +39,6 @@ public void testConfigs() { configs.put(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG, true); configs.put(ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG, 200); configs.put(ShareGroupConfig.SHARE_GROUP_DELIVERY_COUNT_LIMIT_CONFIG, 5); - configs.put(ShareGroupConfig.SHARE_GROUP_MAX_GROUPS_CONFIG, (short) 10); configs.put(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG, 30000); configs.put(ShareGroupConfig.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG, 15000); configs.put(ShareGroupConfig.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG, 60000); @@ -51,7 +50,6 @@ public void testConfigs() { assertTrue(config.isShareGroupEnabled()); assertEquals(200, config.shareGroupPartitionMaxRecordLocks()); assertEquals(5, config.shareGroupDeliveryCountLimit()); - assertEquals(10, config.shareGroupMaxGroups()); assertEquals(30000, config.shareGroupRecordLockDurationMs()); assertEquals(15000, config.shareGroupMinRecordLockDurationMs()); assertEquals(60000, config.shareGroupMaxRecordLockDurationMs()); @@ -89,24 +87,6 @@ public void testInvalidConfigs() { assertEquals("Invalid value 11 for configuration group.share.delivery.count.limit: Value must be no more than 10", assertThrows(ConfigException.class, () -> createConfig(configs)).getMessage()); - configs.clear(); - // test for when SHARE_GROUP_MAX_GROUPS_CONFIG is of incorrect data type - configs.put(ShareGroupConfig.SHARE_GROUP_MAX_GROUPS_CONFIG, 10); - assertEquals("Invalid value 10 for configuration group.share.max.groups: Expected value to be a 16-bit integer (short), but it was a java.lang.Integer", - assertThrows(ConfigException.class, () -> createConfig(configs)).getMessage()); - - configs.clear(); - // test for when SHARE_GROUP_MAX_GROUPS_CONFIG is out of bounds - configs.put(ShareGroupConfig.SHARE_GROUP_MAX_GROUPS_CONFIG, (short) 0); - assertEquals("Invalid value 0 for configuration group.share.max.groups: Value must be at least 1", - assertThrows(ConfigException.class, () -> createConfig(configs)).getMessage()); - - configs.clear(); - // test for when SHARE_GROUP_MAX_GROUPS_CONFIG is out of bounds - configs.put(ShareGroupConfig.SHARE_GROUP_MAX_GROUPS_CONFIG, (short) 110); - assertEquals("Invalid value 110 for configuration group.share.max.groups: Value must be no more than 100", - assertThrows(ConfigException.class, () -> createConfig(configs)).getMessage()); - configs.clear(); // test for when SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG is out of bounds configs.put(ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG, 50); @@ -118,13 +98,19 @@ public void testInvalidConfigs() { configs.put(ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG, 20000); assertEquals("Invalid value 20000 for configuration group.share.partition.max.record.locks: Value must be no more than 10000", assertThrows(ConfigException.class, () -> createConfig(configs)).getMessage()); + + configs.clear(); + // test for when SHARE_GROUP_MAX_SHARE_SESSIONS_CONFIG is less than SHARE_GROUP_MAX_SIZE_CONFIG + configs.put(GroupCoordinatorConfig.SHARE_GROUP_MAX_SIZE_CONFIG, 2000); + configs.put(ShareGroupConfig.SHARE_GROUP_MAX_SHARE_SESSIONS_CONFIG, 1000); + assertEquals("Invalid value 2000 for configuration group.share.max.size: Value must be no more than 1000", + assertThrows(ConfigException.class, () -> createConfig(configs)).getMessage()); } public static ShareGroupConfig createShareGroupConfig( boolean shareGroupEnable, int shareGroupPartitionMaxRecordLocks, int shareGroupDeliveryCountLimit, - short shareGroupsMaxGroups, int shareGroupRecordLockDurationsMs, int shareGroupMinRecordLockDurationMs, int shareGroupMaxRecordLockDurationMs @@ -133,7 +119,6 @@ public static ShareGroupConfig createShareGroupConfig( configs.put(ShareGroupConfig.SHARE_GROUP_ENABLE_CONFIG, shareGroupEnable); configs.put(ShareGroupConfig.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG, shareGroupPartitionMaxRecordLocks); configs.put(ShareGroupConfig.SHARE_GROUP_DELIVERY_COUNT_LIMIT_CONFIG, shareGroupDeliveryCountLimit); - configs.put(ShareGroupConfig.SHARE_GROUP_MAX_GROUPS_CONFIG, shareGroupsMaxGroups); configs.put(ShareGroupConfig.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupRecordLockDurationsMs); configs.put(ShareGroupConfig.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupMinRecordLockDurationMs); configs.put(ShareGroupConfig.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG, shareGroupMaxRecordLockDurationMs); diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java index ab55dd3239b24..0509b74d330c4 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsCoordinatorRecordHelpersTest.java @@ -300,12 +300,13 @@ public void testNewStreamsGroupEpochRecord() { .setGroupId(GROUP_ID), new ApiMessageAndVersion( new StreamsGroupMetadataValue() - .setEpoch(42), + .setEpoch(42) + .setMetadataHash(42), (short) 0 ) ); - assertEquals(expectedRecord, StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(GROUP_ID, 42)); + assertEquals(expectedRecord, StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(GROUP_ID, 42, 42)); } @Test @@ -740,7 +741,7 @@ public void testNewStreamsGroupPartitionMetadataTombstoneRecordNullGroupId() { @Test public void testNewStreamsGroupEpochRecordNullGroupId() { NullPointerException exception = assertThrows(NullPointerException.class, () -> - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(null, 1)); + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(null, 1, 1)); assertEquals("groupId should not be null here", exception.getMessage()); } diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java index b8b52267a3fad..a3dc088badc16 100644 --- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java +++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupBuilder.java @@ -85,7 +85,7 @@ public List build() { // Add group epoch record. records.add( - StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, groupEpoch)); + StreamsCoordinatorRecordHelpers.newStreamsGroupEpochRecord(groupId, groupEpoch, 0)); // Add target assignment records. targetAssignments.forEach((memberId, assignment) -> diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index 5ea4b3b6a4e4c..b367fda8114d5 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -665,15 +665,17 @@ String maybeGenerateElrSafetyRecords(List outputRecords) { * @param updates The user-requested updates. * @param upgradeTypes The user-requested upgrade types. * @param validateOnly True if we should validate the request but not make changes. + * @param currentClaimEpoch the currently claimed epoch * * @return The result. */ ControllerResult updateFeatures( Map updates, Map upgradeTypes, - boolean validateOnly + boolean validateOnly, + int currentClaimEpoch ) { - ControllerResult result = featureControl.updateFeatures(updates, upgradeTypes, validateOnly); + ControllerResult result = featureControl.updateFeatures(updates, upgradeTypes, validateOnly, currentClaimEpoch); if (result.response().isSuccess() && !validateOnly && updates.getOrDefault(EligibleLeaderReplicasVersion.FEATURE_NAME, (short) 0) > 0 diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index c167bb4a31cae..f143047f432ce 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -18,7 +18,9 @@ package org.apache.kafka.controller; import org.apache.kafka.clients.admin.FeatureUpdate; +import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.metadata.FeatureLevelRecord; +import org.apache.kafka.common.metadata.NoOpRecord; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.utils.LogContext; @@ -55,6 +57,8 @@ public static class Builder { private LogContext logContext = null; private SnapshotRegistry snapshotRegistry = null; private QuorumFeatures quorumFeatures = null; + private KRaftVersionAccessor kraftVersionAccessor = null; + private ClusterFeatureSupportDescriber clusterSupportDescriber = new ClusterFeatureSupportDescriber() { @Override public Iterator>> brokerSupported() { @@ -87,6 +91,11 @@ Builder setClusterFeatureSupportDescriber(ClusterFeatureSupportDescriber cluster return this; } + Builder setKRaftVersionAccessor(KRaftVersionAccessor kraftVersionAccessor) { + this.kraftVersionAccessor = kraftVersionAccessor; + return this; + } + public FeatureControlManager build() { if (logContext == null) logContext = new LogContext(); if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext); @@ -97,11 +106,30 @@ public FeatureControlManager build() { MetadataVersion.latestProduction().featureLevel())); quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, List.of(0)); } + if (kraftVersionAccessor == null) { + kraftVersionAccessor = new KRaftVersionAccessor() { + private KRaftVersion version = KRaftVersion.LATEST_PRODUCTION; + + @Override + public KRaftVersion kraftVersion() { + return version; + } + + @Override + public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) { + if (!validateOnly) { + this.version = version; + } + } + }; + } + return new FeatureControlManager( logContext, quorumFeatures, snapshotRegistry, - clusterSupportDescriber + clusterSupportDescriber, + kraftVersionAccessor ); } } @@ -128,23 +156,31 @@ public FeatureControlManager build() { */ private final ClusterFeatureSupportDescriber clusterSupportDescriber; + /** + * The interface for reading and upgrading the kraft version. + */ + private final KRaftVersionAccessor kraftVersionAccessor; + private FeatureControlManager( LogContext logContext, QuorumFeatures quorumFeatures, SnapshotRegistry snapshotRegistry, - ClusterFeatureSupportDescriber clusterSupportDescriber + ClusterFeatureSupportDescriber clusterSupportDescriber, + KRaftVersionAccessor kraftVersionAccessor ) { this.log = logContext.logger(FeatureControlManager.class); this.quorumFeatures = quorumFeatures; this.finalizedVersions = new TimelineHashMap<>(snapshotRegistry, 0); this.metadataVersion = new TimelineObject<>(snapshotRegistry, Optional.empty()); this.clusterSupportDescriber = clusterSupportDescriber; + this.kraftVersionAccessor = kraftVersionAccessor; } ControllerResult updateFeatures( Map updates, Map upgradeTypes, - boolean validateOnly + boolean validateOnly, + int currentClaimedEpoch ) { List records = BoundedList.newArrayBacked(MAX_RECORDS_PER_USER_OP); @@ -154,8 +190,15 @@ ControllerResult updateFeatures( proposedUpdatedVersions.putAll(updates); for (Entry entry : updates.entrySet()) { - ApiError error = updateFeature(entry.getKey(), entry.getValue(), - upgradeTypes.getOrDefault(entry.getKey(), FeatureUpdate.UpgradeType.UPGRADE), records, proposedUpdatedVersions); + ApiError error = updateFeature( + entry.getKey(), + entry.getValue(), + upgradeTypes.getOrDefault(entry.getKey(), FeatureUpdate.UpgradeType.UPGRADE), + records, + proposedUpdatedVersions, + validateOnly, + currentClaimedEpoch + ); if (!error.error().equals(Errors.NONE)) { return ControllerResult.of(List.of(), error); } @@ -181,12 +224,15 @@ private MetadataVersion metadataVersionOrThrow(long epoch) { new IllegalStateException("Unknown metadata version for FeatureControlManager")); } + @SuppressWarnings({ "CyclomaticComplexity" }) private ApiError updateFeature( String featureName, short newVersion, FeatureUpdate.UpgradeType upgradeType, List records, - Map proposedUpdatedVersions + Map proposedUpdatedVersions, + boolean validateOnly, + int currentClaimedEpoch ) { if (upgradeType.equals(FeatureUpdate.UpgradeType.UNKNOWN)) { return invalidUpdateVersion(featureName, newVersion, @@ -196,6 +242,8 @@ private ApiError updateFeature( final short currentVersion; if (featureName.equals(MetadataVersion.FEATURE_NAME)) { currentVersion = metadataVersionOrThrow().featureLevel(); + } else if (featureName.equals(KRaftVersion.FEATURE_NAME)) { + currentVersion = kraftVersionAccessor.kraftVersion().featureLevel(); } else { currentVersion = finalizedVersions.getOrDefault(featureName, (short) 0); } @@ -225,6 +273,34 @@ private ApiError updateFeature( if (featureName.equals(MetadataVersion.FEATURE_NAME)) { // Perform additional checks if we're updating metadata.version return updateMetadataVersion(newVersion, upgradeType.equals(FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), records::add); + } else if (featureName.equals(KRaftVersion.FEATURE_NAME)) { + if (upgradeType.equals(FeatureUpdate.UpgradeType.UPGRADE)) { + try { + kraftVersionAccessor.upgradeKRaftVersion( + currentClaimedEpoch, + KRaftVersion.fromFeatureLevel(newVersion), + validateOnly + ); + /* Add the noop record so that there is at least one offset to wait on to + * complete the upgrade RPC + */ + records.add(new ApiMessageAndVersion(new NoOpRecord(), (short) 0)); + return ApiError.NONE; + } catch (ApiException e) { + return ApiError.fromThrowable(e); + } catch (IllegalArgumentException e) { + return invalidUpdateVersion(featureName, newVersion, e.getMessage()); + } + } else if (newVersion != currentVersion) { + return invalidUpdateVersion( + featureName, + newVersion, + "Can't downgrade the version of this feature." + ); + } else { + // Version didn't change + return ApiError.NONE; + } } else { // Validate dependencies for features that are not metadata.version try { diff --git a/metadata/src/main/java/org/apache/kafka/controller/KRaftVersionAccessor.java b/metadata/src/main/java/org/apache/kafka/controller/KRaftVersionAccessor.java new file mode 100644 index 0000000000000..b07d2c6398c73 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/KRaftVersionAccessor.java @@ -0,0 +1,42 @@ +/* + * 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.controller; + +import org.apache.kafka.server.common.KRaftVersion; + +/** + * Type for upgrading and reading the kraft version. + */ +interface KRaftVersionAccessor { + /** + * Returns the latest kraft version. + * + * The latest version may be uncommitted. + */ + KRaftVersion kraftVersion(); + + /** + * Upgrade the kraft version. + * + * @param epoch the current epoch + * @param newVersion the new kraft version to upgrade to + * @param validateOnly whether to just validate the change and not persist it + * @throws ApiException when the upgrade fails to validate + */ + void upgradeKRaftVersion(int epoch, KRaftVersion newVersion, boolean validateOnly); +} diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index bf2e26834ca6c..b03501eb7bb67 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -1519,6 +1519,7 @@ private QuorumController( setQuorumFeatures(quorumFeatures). setSnapshotRegistry(snapshotRegistry). setClusterFeatureSupportDescriber(clusterSupportDescriber). + setKRaftVersionAccessor(new RaftClientKRaftVersionAccessor(raftClient)). build(); this.clusterControl = new ClusterControlManager.Builder(). setLogContext(logContext). @@ -2041,7 +2042,7 @@ public CompletableFuture updateFeatures( upgradeTypes.put(featureName, FeatureUpdate.UpgradeType.fromCode(featureUpdate.upgradeType())); updates.put(featureName, featureUpdate.maxVersionLevel()); }); - return configurationControl.updateFeatures(updates, upgradeTypes, request.validateOnly()); + return configurationControl.updateFeatures(updates, upgradeTypes, request.validateOnly(), curClaimEpoch); }).thenApply(result -> { UpdateFeaturesResponseData responseData = new UpdateFeaturesResponseData(); diff --git a/metadata/src/main/java/org/apache/kafka/controller/RaftClientKRaftVersionAccessor.java b/metadata/src/main/java/org/apache/kafka/controller/RaftClientKRaftVersionAccessor.java new file mode 100644 index 0000000000000..93020bf0dd6d8 --- /dev/null +++ b/metadata/src/main/java/org/apache/kafka/controller/RaftClientKRaftVersionAccessor.java @@ -0,0 +1,37 @@ +/* + * 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.controller; + +import org.apache.kafka.raft.RaftClient; +import org.apache.kafka.server.common.KRaftVersion; + +public final class RaftClientKRaftVersionAccessor implements KRaftVersionAccessor { + private final RaftClient raftClient; + + public RaftClientKRaftVersionAccessor(RaftClient raftClient) { + this.raftClient = raftClient; + } + + public KRaftVersion kraftVersion() { + return raftClient.kraftVersion(); + } + + public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) { + raftClient.upgradeKRaftVersion(epoch, version, validateOnly); + } +} diff --git a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java index 2402437c663e4..786f4c1e31eb0 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java @@ -36,7 +36,7 @@ */ public class BootstrapMetadata { private final List records; - private final MetadataVersion metadataVersion; + private final short metadataVersionLevel; private final String source; public static BootstrapMetadata fromVersions( @@ -66,7 +66,7 @@ public static BootstrapMetadata fromVersions( setFeatureLevel(level), (short) 0)); } } - return new BootstrapMetadata(records, metadataVersion, source); + return new BootstrapMetadata(records, metadataVersion.featureLevel(), source); } public static BootstrapMetadata fromVersion(MetadataVersion metadataVersion, String source) { @@ -74,28 +74,28 @@ public static BootstrapMetadata fromVersion(MetadataVersion metadataVersion, Str new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(metadataVersion.featureLevel()), (short) 0)); - return new BootstrapMetadata(records, metadataVersion, source); + return new BootstrapMetadata(records, metadataVersion.featureLevel(), source); } public static BootstrapMetadata fromRecords(List records, String source) { - MetadataVersion metadataVersion = null; + Optional metadataVersionLevel = Optional.empty(); for (ApiMessageAndVersion record : records) { - Optional version = recordToMetadataVersion(record.message()); - if (version.isPresent()) { - metadataVersion = version.get(); + Optional level = recordToMetadataVersionLevel(record.message()); + if (level.isPresent()) { + metadataVersionLevel = level; } } - if (metadataVersion == null) { + if (metadataVersionLevel.isEmpty()) { throw new RuntimeException("No FeatureLevelRecord for " + MetadataVersion.FEATURE_NAME + " was found in the bootstrap metadata from " + source); } - return new BootstrapMetadata(records, metadataVersion, source); + return new BootstrapMetadata(records, metadataVersionLevel.get(), source); } - public static Optional recordToMetadataVersion(ApiMessage record) { + public static Optional recordToMetadataVersionLevel(ApiMessage record) { if (record instanceof FeatureLevelRecord featureLevel) { if (featureLevel.name().equals(MetadataVersion.FEATURE_NAME)) { - return Optional.of(MetadataVersion.fromFeatureLevel(featureLevel.featureLevel())); + return Optional.of(featureLevel.featureLevel()); } } return Optional.empty(); @@ -103,11 +103,11 @@ public static Optional recordToMetadataVersion(ApiMessage recor BootstrapMetadata( List records, - MetadataVersion metadataVersion, + short metadataVersionLevel, String source ) { this.records = Objects.requireNonNull(records); - this.metadataVersion = metadataVersion; + this.metadataVersionLevel = metadataVersionLevel; Objects.requireNonNull(source); this.source = source; } @@ -117,7 +117,7 @@ public List records() { } public MetadataVersion metadataVersion() { - return metadataVersion; + return MetadataVersion.fromFeatureLevel(metadataVersionLevel); } public String source() { @@ -163,7 +163,7 @@ public BootstrapMetadata copyWithFeatureRecord(String featureName, short level) @Override public int hashCode() { - return Objects.hash(records, metadataVersion, source); + return Objects.hash(records, metadataVersionLevel, source); } @Override @@ -171,14 +171,14 @@ public boolean equals(Object o) { if (o == null || !o.getClass().equals(this.getClass())) return false; BootstrapMetadata other = (BootstrapMetadata) o; return Objects.equals(records, other.records) && - metadataVersion.equals(other.metadataVersion) && + metadataVersionLevel == other.metadataVersionLevel && source.equals(other.source); } @Override public String toString() { return "BootstrapMetadata(records=" + records.toString() + - ", metadataVersion=" + metadataVersion + + ", metadataVersionLevel=" + metadataVersionLevel + ", source=" + source + ")"; } diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 6954487c6883a..2c93d1100ecae 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -489,7 +489,8 @@ public void testRejectMinIsrChangeWhenElrEnabled(boolean removal) { EligibleLeaderReplicasVersion.ELRV_1.featureLevel()), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), - false); + false, + 0); assertNotNull(result.response()); assertEquals(Errors.NONE, result.response().error()); RecordTestUtils.replayAll(manager, result.records()); @@ -540,7 +541,8 @@ public void testElrUpgrade(boolean isMetadataVersionElrEnabled) { EligibleLeaderReplicasVersion.ELRV_1.featureLevel()), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), - false); + false, + 0); assertNotNull(result.response()); if (isMetadataVersionElrEnabled) { assertEquals(Errors.NONE, result.response().error()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index f494ebddcf8dc..eac143209dd6b 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -107,10 +107,10 @@ public void testUpdateFeatures() { "Invalid update version 3 for feature " + TestFeatureVersion.FEATURE_NAME + ". Local controller 0 only supports versions 0-2")), manager.updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 3), Map.of(TestFeatureVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), - false)); + false, 0)); ControllerResult result = manager.updateFeatures( updateMap(TestFeatureVersion.FEATURE_NAME, 1, "bar", 1), Map.of(), - false); + false, 0); ApiError expectedError = new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 1 for feature bar. Local controller 0 does not support this feature."); List expectedMessages = new ArrayList<>(); @@ -119,7 +119,7 @@ public void testUpdateFeatures() { result = manager.updateFeatures( updateMap(TestFeatureVersion.FEATURE_NAME, 1), Map.of(), - false); + false, 0); expectedError = ApiError.NONE; assertEquals(expectedError, result.response()); expectedMessages = new ArrayList<>(); @@ -208,10 +208,10 @@ public void testUpdateFeaturesErrorCases() { "Invalid update version 3 for feature foo. Broker 5 does not support this feature.")), manager.updateFeatures(updateMap("foo", 3), Map.of("foo", FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), - false)); + false, 0)); ControllerResult result = manager.updateFeatures( - updateMap(TransactionVersion.FEATURE_NAME, 2), Map.of(), false); + updateMap(TransactionVersion.FEATURE_NAME, 2), Map.of(), false, 0); assertEquals(ApiError.NONE, result.response()); manager.replay((FeatureLevelRecord) result.records().get(0).message()); snapshotRegistry.idempotentCreateSnapshot(3); @@ -219,7 +219,7 @@ public void testUpdateFeaturesErrorCases() { assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 1 for feature " + TransactionVersion.FEATURE_NAME + "." + " Can't downgrade the version of this feature without setting the upgrade type to either safe or unsafe downgrade.")), - manager.updateFeatures(updateMap(TransactionVersion.FEATURE_NAME, 1), Map.of(), false)); + manager.updateFeatures(updateMap(TransactionVersion.FEATURE_NAME, 1), Map.of(), false, 0)); assertEquals( ControllerResult.atomicOf( @@ -236,7 +236,8 @@ public void testUpdateFeaturesErrorCases() { manager.updateFeatures( updateMap(TransactionVersion.FEATURE_NAME, 1), Map.of(TransactionVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), - false) + false, + 0) ); } @@ -251,7 +252,7 @@ public void testReplayRecords() { build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); ControllerResult result = manager. - updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Map.of(), false); + updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Map.of(), false, 0); RecordTestUtils.replayAll(manager, result.records()); assertEquals(MetadataVersion.MINIMUM_VERSION, manager.metadataVersionOrThrow()); assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get(TestFeatureVersion.FEATURE_NAME)); @@ -289,7 +290,8 @@ public void testCannotDowngradeToHigherVersion() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), - true)); + true, + 0)); } @Test @@ -301,7 +303,8 @@ public void testCannotUnsafeDowngradeToHigherVersion() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), - true)); + true, + 0)); } @Test @@ -318,7 +321,8 @@ public void testCannotUpgradeToLowerVersion() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), - true)); + true, + 0)); } @Test @@ -328,7 +332,8 @@ public void testCanUpgradeToHigherVersion() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), - true)); + true, + 0)); } @Test @@ -340,7 +345,8 @@ public void testCannotUseSafeDowngradeIfMetadataChanged() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), - true)); + true, + 0)); } @Test @@ -349,9 +355,10 @@ public void testUnsafeDowngradeIsTemporarilyDisabled() { assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid metadata.version 7. Unsafe metadata downgrade is not supported in this version.")), manager.updateFeatures( - Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), - Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), - true)); + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), + true, + 0)); } @Disabled @@ -362,7 +369,8 @@ public void testCanUseUnsafeDowngradeIfMetadataChanged() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), - true)); + true, + 0)); } @Test @@ -376,7 +384,8 @@ public void testCanUseSafeDowngradeIfMetadataDidNotChange() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_4_IV0.featureLevel()), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), - true)); + true, + 0)); } @Test @@ -391,7 +400,8 @@ public void testCannotDowngradeBeforeMinimumKraftVersion() { manager.updateFeatures( Map.of(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL), Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), - true)); + true, + 0)); } @Test @@ -410,7 +420,8 @@ public void testCreateFeatureLevelRecords() { ControllerResult result = manager.updateFeatures( Map.of(Feature.TEST_VERSION.featureName(), (short) 1), Map.of(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE), - false); + false, + 0); assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new FeatureLevelRecord().setName(Feature.TEST_VERSION.featureName()).setFeatureLevel((short) 1), (short) 0)), ApiError.NONE), result); @@ -420,7 +431,8 @@ public void testCreateFeatureLevelRecords() { ControllerResult result2 = manager.updateFeatures( Map.of(Feature.TEST_VERSION.featureName(), (short) 0), Map.of(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), - false); + false, + 0); assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new FeatureLevelRecord().setName(Feature.TEST_VERSION.featureName()).setFeatureLevel((short) 0), (short) 0)), ApiError.NONE), result2); @@ -444,7 +456,8 @@ public void testUpgradeElrFeatureLevel() { ControllerResult result = manager.updateFeatures( Map.of(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), (short) 1), Map.of(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE), - false); + false, + 0); assertTrue(result.response().isSuccess()); assertEquals(List.of(new ApiMessageAndVersion( new FeatureLevelRecord(). diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java index d866c1b00ba27..9647c4b031f67 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotEmitterTest.java @@ -126,6 +126,11 @@ public KRaftVersion kraftVersion() { return KRaftVersion.KRAFT_VERSION_0; } + @Override + public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) { + // nothing to do + } + @Override public void close() throws Exception { // nothing to do diff --git a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java index 5d19c1068c602..858e823b7e0f6 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java @@ -51,13 +51,13 @@ public void testFromVersion() { new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). setFeatureLevel((short) 7), (short) 0)), - IBP_3_3_IV3, "foo"), + IBP_3_3_IV3.featureLevel(), "foo"), BootstrapMetadata.fromVersion(IBP_3_3_IV3, "foo")); } @Test public void testFromRecordsList() { - assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV3, "bar"), + assertEquals(new BootstrapMetadata(SAMPLE_RECORDS1, IBP_3_3_IV3.featureLevel(), "bar"), BootstrapMetadata.fromRecords(SAMPLE_RECORDS1, "bar")); } @@ -128,10 +128,10 @@ public void testFeatureLevelForFeature() { @Test public void testFromRecordsListWithOldMetadataVersion() { - RuntimeException exception = assertThrows(RuntimeException.class, - () -> BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux")); + BootstrapMetadata bootstrapMetadata = BootstrapMetadata.fromRecords(RECORDS_WITH_OLD_METADATA_VERSION, "quux"); assertEquals("No MetadataVersion with feature level 1. Valid feature levels are from " + MetadataVersion.MINIMUM_VERSION.featureLevel() - + " to " + MetadataVersion.latestTesting().featureLevel() + ".", - exception.getMessage()); + + " to " + MetadataVersion.latestTesting().featureLevel() + ".", + assertThrows(RuntimeException.class, + () -> bootstrapMetadata.metadataVersion()).getMessage()); } } diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index 8e403b9c9e1f9..0d5eb8dba0761 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -415,7 +415,7 @@ void beginShutdown() { /** * The latest kraft version used by this local log manager. */ - private final KRaftVersion lastKRaftVersion; + private KRaftVersion lastKRaftVersion; /** * Whether this LocalLogManager has been shut down. @@ -789,4 +789,11 @@ public void throwOnNextAppend() { public KRaftVersion kraftVersion() { return lastKRaftVersion; } + + @Override + public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) { + if (!validateOnly) { + lastKRaftVersion = version; + } + } } diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 4d2357fdef58e..6ad6d4f3156f4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -30,6 +30,8 @@ import java.util.Set; public class FollowerState implements EpochState { + private final Logger log; + private final int fetchTimeoutMs; private final int epoch; private final int leaderId; @@ -38,19 +40,23 @@ public class FollowerState implements EpochState { private final Set voters; // Used for tracking the expiration of both the Fetch and FetchSnapshot requests private final Timer fetchTimer; - /* Used to track if the replica has fetched successfully from the leader at least once since the transition to - * follower in this epoch. If the replica has not yet fetched successfully, it may be able to grant PreVotes. + // Used to track when to send another update voter request + private final Timer updateVoterPeriodTimer; + + /* Used to track if the replica has fetched successfully from the leader at least once since + * the transition to follower in this epoch. If the replica has not yet fetched successfully, + * it may be able to grant PreVotes. */ - private boolean hasFetchedFromLeader; + private boolean hasFetchedFromLeader = false; private Optional highWatermark; - /* Used to track the currently fetching snapshot. When fetching snapshot regular - * Fetch request are paused + /* For kraft.version 0, track if the leader has received updated voter information from this + * follower. + */ + private boolean hasUpdatedLeader = false; + /* Used to track the currently fetching snapshot. When fetching snapshot regular Fetch request + * are paused */ private Optional fetchingSnapshot = Optional.empty(); - // Used to throttle update voter request and allow for Fetch/FetchSnapshot requests - private final Timer updateVoterPeriodTimer; - - private final Logger log; public FollowerState( Time time, @@ -73,7 +79,6 @@ public FollowerState( this.updateVoterPeriodTimer = time.timer(updateVoterPeriodMs()); this.highWatermark = highWatermark; this.log = logContext.logger(FollowerState.class); - this.hasFetchedFromLeader = false; } @Override @@ -140,7 +145,7 @@ public void overrideFetchTimeout(long currentTimeMs, long timeoutMs) { private long updateVoterPeriodMs() { // Allow for a few rounds of fetch request before attempting to update // the voter state - return fetchTimeoutMs * 3L; + return fetchTimeoutMs; } public boolean hasUpdateVoterPeriodExpired(long currentTimeMs) { @@ -158,6 +163,14 @@ public void resetUpdateVoterPeriod(long currentTimeMs) { updateVoterPeriodTimer.reset(updateVoterPeriodMs()); } + public boolean hasUpdatedLeader() { + return hasUpdatedLeader; + } + + public void setHasUpdatedLeader() { + this.hasUpdatedLeader = true; + } + public boolean updateHighWatermark(OptionalLong newHighWatermark) { if (newHighWatermark.isEmpty() && highWatermark.isPresent()) { throw new IllegalArgumentException( diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index e46a2a1ff837d..5005f509cbd1f 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -77,10 +77,12 @@ import org.apache.kafka.raft.internals.DefaultRequestSender; import org.apache.kafka.raft.internals.FuturePurgatory; import org.apache.kafka.raft.internals.KRaftControlRecordStateMachine; +import org.apache.kafka.raft.internals.KRaftVersionUpgrade; import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.raft.internals.MemoryBatchReader; import org.apache.kafka.raft.internals.RecordsBatchReader; import org.apache.kafka.raft.internals.RemoveVoterHandler; +import org.apache.kafka.raft.internals.RequestSendResult; import org.apache.kafka.raft.internals.ThresholdPurgatory; import org.apache.kafka.raft.internals.UpdateVoterHandler; import org.apache.kafka.server.common.KRaftVersion; @@ -163,7 +165,7 @@ * as FileRecords, but we use {@link UnalignedRecords} in FetchSnapshotResponse because the records * are not necessarily offset-aligned. */ -@SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity", "ParameterNumber", "NPathComplexity" }) +@SuppressWarnings({ "ClassDataAbstractionCoupling", "ClassFanOutComplexity", "ParameterNumber", "NPathComplexity", "JavaNCSS" }) public final class KafkaRaftClient implements RaftClient { private static final int RETRY_BACKOFF_BASE_MS = 100; private static final int MAX_NUMBER_OF_BATCHES = 10; @@ -592,7 +594,8 @@ public void initialize( this.updateVoterHandler = new UpdateVoterHandler( nodeId, partitionState, - channel.listenerName() + channel.listenerName(), + logContext ); } @@ -653,7 +656,7 @@ private void onBecomeLeader(long currentTimeMs) { // The high watermark can only be advanced once we have written a record // from the new leader's epoch. Hence, we write a control message immediately // to ensure there is no delay committing pending data. - state.appendStartOfEpochControlRecords(quorum.localVoterNodeOrThrow(), currentTimeMs); + state.appendStartOfEpochControlRecords(currentTimeMs); resetConnections(); kafkaRaftMetrics.maybeUpdateElectionLatency(currentTimeMs); @@ -2101,8 +2104,7 @@ private boolean handleFetchSnapshotResponse( } FollowerState state = quorum.followerStateOrThrow(); - - if (Errors.forCode(partitionSnapshot.errorCode()) == Errors.SNAPSHOT_NOT_FOUND || + if (error == Errors.SNAPSHOT_NOT_FOUND || partitionSnapshot.snapshotId().endOffset() < 0 || partitionSnapshot.snapshotId().epoch() < 0) { @@ -2118,6 +2120,8 @@ private boolean handleFetchSnapshotResponse( state.setFetchingSnapshot(Optional.empty()); state.resetFetchTimeoutForSuccessfulFetch(currentTimeMs); return true; + } else if (error != Errors.NONE) { + return handleUnexpectedError(error, responseMetadata); } OffsetAndEpoch snapshotId = new OffsetAndEpoch( @@ -2125,15 +2129,11 @@ private boolean handleFetchSnapshotResponse( partitionSnapshot.snapshotId().epoch() ); - RawSnapshotWriter snapshot; - if (state.fetchingSnapshot().isPresent()) { - snapshot = state.fetchingSnapshot().get(); - } else { - throw new IllegalStateException( + RawSnapshotWriter snapshot = state.fetchingSnapshot().orElseThrow( + () -> new IllegalStateException( String.format("Received unexpected fetch snapshot response: %s", partitionSnapshot) - ); - } - + ) + ); if (!snapshot.snapshotId().equals(snapshotId)) { throw new IllegalStateException( String.format( @@ -2142,8 +2142,7 @@ private boolean handleFetchSnapshotResponse( snapshotId ) ); - } - if (snapshot.sizeInBytes() != partitionSnapshot.position()) { + } else if (snapshot.sizeInBytes() != partitionSnapshot.position()) { throw new IllegalStateException( String.format( "Received fetch snapshot response with an invalid position. Expected %d; Received %d", @@ -2413,7 +2412,7 @@ private boolean handleUpdateVoterResponse( int responseEpoch = data.currentLeader().leaderEpoch(); final Endpoints leaderEndpoints; - if (responseLeaderId.isPresent() && data.currentLeader().host().isEmpty()) { + if (responseLeaderId.isPresent() && !data.currentLeader().host().isEmpty()) { leaderEndpoints = Endpoints.fromInetSocketAddresses( Map.of( channel.listenerName(), @@ -2435,8 +2434,18 @@ private boolean handleUpdateVoterResponse( responseMetadata.source(), currentTimeMs ); + if (handled.isPresent()) { + return handled.get(); + } else if (error == Errors.NONE || error == Errors.UNSUPPORTED_VERSION) { + FollowerState follower = quorum.followerStateOrThrow(); + follower.setHasUpdatedLeader(); + // Treat update voter similar to fetch and fetch snapshot, and reset the timer + follower.resetFetchTimeoutForSuccessfulFetch(currentTimeMs); - return handled.orElse(true); + return true; + } else { + return handleUnexpectedError(error, responseMetadata); + } } private boolean hasConsistentLeader(int epoch, OptionalInt leaderId) { @@ -2746,17 +2755,27 @@ private void handleInboundMessage(RaftMessage message, long currentTimeMs) { } /** - * Attempt to send a request. Return the time to wait before the request can be retried. + * Attempt to send a request. + * + * Return if the request was sent and the time to wait before the request can be retried. + * + * @param currentTimeMs the current time + * @param destination the node receiving the request + * @param requestSupplier the function that creates the request + * @return the first element in the pair indicates if the request was sent; the second element + * in the pair indicates the time to wait before retrying. */ - private long maybeSendRequest( + private RequestSendResult maybeSendRequest( long currentTimeMs, Node destination, Supplier requestSupplier ) { + var requestSent = false; + if (requestManager.isBackingOff(destination, currentTimeMs)) { long remainingBackoffMs = requestManager.remainingBackoffMs(destination, currentTimeMs); logger.debug("Connection for {} is backing off for {} ms", destination, remainingBackoffMs); - return remainingBackoffMs; + return RequestSendResult.of(requestSent, remainingBackoffMs); } if (requestManager.isReady(destination, currentTimeMs)) { @@ -2788,10 +2807,14 @@ private long maybeSendRequest( requestManager.onRequestSent(destination, correlationId, currentTimeMs); channel.send(requestMessage); + requestSent = true; logger.trace("Sent outbound request: {}", requestMessage); } - return requestManager.remainingRequestTimeMs(destination, currentTimeMs); + return RequestSendResult.of( + requestSent, + requestManager.remainingRequestTimeMs(destination, currentTimeMs) + ); } private EndQuorumEpochRequestData buildEndQuorumEpochRequest( @@ -2813,7 +2836,8 @@ private long maybeSendRequests( ) { long minBackoffMs = Long.MAX_VALUE; for (Node destination : destinations) { - long backoffMs = maybeSendRequest(currentTimeMs, destination, requestSupplier); + long backoffMs = maybeSendRequest(currentTimeMs, destination, requestSupplier) + .timeToWaitMs(); if (backoffMs < minBackoffMs) { minBackoffMs = backoffMs; } @@ -2833,7 +2857,7 @@ private long maybeSendRequest( currentTimeMs, destinationSupplier.apply(voter.id()), () -> requestSupplier.apply(voter) - ); + ).timeToWaitMs(); minBackoffMs = Math.min(minBackoffMs, backoffMs); } return minBackoffMs; @@ -2884,11 +2908,13 @@ private FetchRequestData buildFetchRequest() { private long maybeSendFetchToAnyBootstrap(long currentTimeMs) { Optional readyNode = requestManager.findReadyBootstrapServer(currentTimeMs); - return readyNode.map(node -> maybeSendRequest( - currentTimeMs, - node, - this::buildFetchRequest - )).orElseGet(() -> requestManager.backoffBeforeAvailableBootstrapServer(currentTimeMs)); + return readyNode.map( + node -> maybeSendRequest( + currentTimeMs, + node, + this::buildFetchRequest + ).timeToWaitMs() + ).orElseGet(() -> requestManager.backoffBeforeAvailableBootstrapServer(currentTimeMs)); } private FetchSnapshotRequestData buildFetchSnapshotRequest(OffsetAndEpoch snapshotId, long snapshotSize) { @@ -2937,7 +2963,12 @@ private void appendBatch( future.whenComplete((commitTimeMs, exception) -> { if (exception != null) { - logger.debug("Failed to commit {} records up to last offset {}", batch.numRecords, offsetAndEpoch, exception); + logger.debug( + "Failed to commit {} records up to last offset {}", + batch.numRecords, + offsetAndEpoch, + exception + ); } else { long elapsedTime = Math.max(0, commitTimeMs - appendTimeMs); double elapsedTimePerRecord = (double) elapsedTime / batch.numRecords; @@ -2974,6 +3005,7 @@ private long maybeAppendBatches( iterator.next().release(); } } + } return state.accumulator().timeUntilDrain(currentTimeMs); @@ -3180,6 +3212,25 @@ private long pollFollower(long currentTimeMs) { } } + private boolean shouldSendUpdateVoteRequest(FollowerState state) { + var version = partitionState.lastKraftVersion(); + /* When the cluster supports reconfiguration, send an updated voter configuration if the + * one in the log doesn't match the local configuration. + */ + var sendWhenReconfigSupported = version.isReconfigSupported() && + partitionState.lastVoterSet().voterNodeNeedsUpdate(quorum.localVoterNodeOrThrow()); + + /* When the cluster doesn't support reconfiguration, the voter needs to send its voter + * information to every new leader. This is because leaders don't persist voter information + * when reconfiguration has not been enabled. The updated voter information is required to + * be able to upgrade the cluster from kraft.version 0. + */ + var sendWhenReconfigNotSupported = !version.isReconfigSupported() && + !state.hasUpdatedLeader(); + + return sendWhenReconfigSupported || sendWhenReconfigNotSupported; + } + private long pollFollowerAsVoter(FollowerState state, long currentTimeMs) { GracefulShutdown shutdown = this.shutdown.get(); final long backoffMs; @@ -3192,13 +3243,21 @@ private long pollFollowerAsVoter(FollowerState state, long currentTimeMs) { transitionToProspective(currentTimeMs); backoffMs = 0; } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) { - if (partitionState.lastKraftVersion().isReconfigSupported() && - partitionState.lastVoterSet().voterNodeNeedsUpdate(quorum.localVoterNodeOrThrow())) { - backoffMs = maybeSendUpdateVoterRequest(state, currentTimeMs); + final boolean resetUpdateVoterTimer; + if (shouldSendUpdateVoteRequest(state)) { + var sendResult = maybeSendUpdateVoterRequest(state, currentTimeMs); + // Update the request timer if the request was sent + resetUpdateVoterTimer = sendResult.requestSent(); + backoffMs = sendResult.timeToWaitMs(); } else { - backoffMs = maybeSendFetchOrFetchSnapshot(state, currentTimeMs); + // Reset the update voter timer since there was no need to update the voter + resetUpdateVoterTimer = true; + backoffMs = maybeSendFetchToBestNode(state, currentTimeMs); + } + + if (resetUpdateVoterTimer) { + state.resetUpdateVoterPeriod(currentTimeMs); } - state.resetUpdateVoterPeriod(currentTimeMs); } else { backoffMs = maybeSendFetchToBestNode(state, currentTimeMs); } @@ -3256,7 +3315,7 @@ private long maybeSendFetchOrFetchSnapshot(FollowerState state, long currentTime currentTimeMs, state.leaderNode(channel.listenerName()), requestSupplier - ); + ).timeToWaitMs(); } private UpdateRaftVoterRequestData buildUpdateVoterRequest() { @@ -3269,7 +3328,7 @@ private UpdateRaftVoterRequestData buildUpdateVoterRequest() { ); } - private long maybeSendUpdateVoterRequest(FollowerState state, long currentTimeMs) { + private RequestSendResult maybeSendUpdateVoterRequest(FollowerState state, long currentTimeMs) { return maybeSendRequest( currentTimeMs, state.leaderNode(channel.listenerName()), @@ -3568,7 +3627,7 @@ public void resign(int epoch) { return; } - LeaderState leaderState = leaderStateOpt.get(); + LeaderState leaderState = leaderStateOpt.get(); if (leaderState.epoch() != epoch) { logger.debug("Ignoring call to resign from epoch {} since it is smaller than the " + "current epoch {}", epoch, leaderState.epoch()); @@ -3621,7 +3680,35 @@ public long logEndOffset() { @Override public KRaftVersion kraftVersion() { - return partitionState.lastKraftVersion(); + if (!isInitialized()) { + throw new IllegalStateException("Cannot read the kraft version before the replica has been initialized"); + } + + return quorum + .maybeLeaderState() + .flatMap(LeaderState::requestedKRaftVersion) + .map(KRaftVersionUpgrade.Version::kraftVersion) + .orElseGet(partitionState::lastKraftVersion); + } + + @Override + public void upgradeKRaftVersion(int epoch, KRaftVersion version, boolean validateOnly) { + if (!isInitialized()) { + throw new IllegalStateException("Cannot update the kraft version before the replica has been initialized"); + } + + LeaderState leaderState = quorum.maybeLeaderState().orElseThrow( + () -> new NotLeaderException("Upgrade kraft version failed because the replica is not the current leader") + ); + + leaderState.maybeAppendUpgradedKRaftVersion( + epoch, + version, + partitionState.lastKraftVersion(), + partitionState.lastVoterSet(), + validateOnly, + time.milliseconds() + ); } @Override diff --git a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java index a8793a8179ccd..7c379275a0e50 100644 --- a/raft/src/main/java/org/apache/kafka/raft/LeaderState.java +++ b/raft/src/main/java/org/apache/kafka/raft/LeaderState.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.errors.InvalidUpdateVersionException; import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage.Voter; @@ -27,14 +29,17 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Timer; +import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.raft.internals.AddVoterHandlerState; import org.apache.kafka.raft.internals.BatchAccumulator; +import org.apache.kafka.raft.internals.KRaftVersionUpgrade; import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.raft.internals.RemoveVoterHandlerState; import org.apache.kafka.server.common.KRaftVersion; import org.slf4j.Logger; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -45,6 +50,7 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -58,11 +64,10 @@ public class LeaderState implements EpochState { static final long OBSERVER_SESSION_TIMEOUT_MS = 300_000L; static final double CHECK_QUORUM_TIMEOUT_FACTOR = 1.5; - private final ReplicaKey localReplicaKey; + private final VoterSet.VoterNode localVoterNode; private final int epoch; private final long epochStartOffset; private final Set grantingVoters; - private final Endpoints localListeners; private final VoterSet voterSetAtEpochStart; // This field is non-empty if the voter set at epoch start came from a snapshot or log segment private final OptionalLong offsetOfVotersAtEpochStart; @@ -87,9 +92,25 @@ public class LeaderState implements EpochState { // This is volatile because resignation can be requested from an external thread. private volatile boolean resignRequested = false; + /* Used to coordinate the upgrade of the kraft.version from 0 to 1. The upgrade is triggered by + * the clients to RaftClient. + * 1. if the kraft version is 0, the starting state is the Voters type. The voter set is the voters in + * the static voter set with the leader updated. See KRaftVersionUpgrade for details on the + * Voters type. + * 2. as the leader receives UpdateRaftVoter requests, it updates the associated Voters type. Only + * after all of the voters have been updated will an upgrade successfully complete. + * 3. a client of RaftClient triggers the upgrade and transition this state to the Version + * type. See KRaftVersionUpgrade for details on the Version type. + * + * All transition are coordinated using optimistic locking by always calling AtomicReference#compareAndSet + */ + private final AtomicReference kraftVersionUpgradeState = new AtomicReference<>( + KRaftVersionUpgrade.empty() + ); + protected LeaderState( Time time, - ReplicaKey localReplicaKey, + VoterSet.VoterNode localVoterNode, int epoch, long epochStartOffset, VoterSet voterSetAtEpochStart, @@ -97,18 +118,30 @@ protected LeaderState( KRaftVersion kraftVersionAtEpochStart, Set grantingVoters, BatchAccumulator accumulator, - Endpoints localListeners, int fetchTimeoutMs, LogContext logContext, KafkaRaftMetrics kafkaRaftMetrics ) { - this.localReplicaKey = localReplicaKey; + if (localVoterNode.voterKey().directoryId().isEmpty()) { + throw new IllegalArgumentException( + String.format("Unknown local replica directory id: %s", localVoterNode) + ); + } else if (!voterSetAtEpochStart.isVoter(localVoterNode.voterKey())) { + throw new IllegalArgumentException( + String.format( + "Local replica %s is not a voter in %s", + localVoterNode, + voterSetAtEpochStart + ) + ); + } + + this.localVoterNode = localVoterNode; this.epoch = epoch; this.epochStartOffset = epochStartOffset; - this.localListeners = localListeners; for (VoterSet.VoterNode voterNode: voterSetAtEpochStart.voterNodes()) { - boolean hasAcknowledgedLeader = voterNode.isVoter(localReplicaKey); + boolean hasAcknowledgedLeader = voterNode.isVoter(localVoterNode.voterKey()); this.voterStates.put( voterNode.voterKey().id(), new ReplicaState(voterNode.voterKey(), hasAcknowledgedLeader, voterNode.listeners()) @@ -128,6 +161,21 @@ protected LeaderState( kafkaRaftMetrics.addLeaderMetrics(); this.kafkaRaftMetrics = kafkaRaftMetrics; + + if (!kraftVersionAtEpochStart.isReconfigSupported()) { + var updatedVoters = voterSetAtEpochStart + .updateVoterIgnoringDirectoryId(localVoterNode) + .orElseThrow( + () -> new IllegalStateException( + String.format( + "Unable to update voter set %s with the latest leader information %s", + voterSetAtEpochStart, + localVoterNode + ) + ) + ); + kraftVersionUpgradeState.set(new KRaftVersionUpgrade.Voters(updatedVoters)); + } } public long timeUntilBeginQuorumEpochTimerExpires(long currentTimeMs) { @@ -185,7 +233,7 @@ public void updateCheckQuorumForFollowingVoter(ReplicaKey replicaKey, long curre // majority, but the leader will never be a member of the fetchedVoters. // If the leader is not in the voter set, it is not in the majority. Then, the // majority can only be composed of fetched voters. - if (voterStates.containsKey(localReplicaKey.id())) { + if (voterStates.containsKey(localVoterNode.voterKey().id())) { majority = majority - 1; } @@ -197,7 +245,7 @@ public void updateCheckQuorumForFollowingVoter(ReplicaKey replicaKey, long curre } private void updateFetchedVoters(ReplicaKey replicaKey) { - if (replicaKey.id() == localReplicaKey.id()) { + if (replicaKey.id() == localVoterNode.voterKey().id()) { throw new IllegalArgumentException("Received a FETCH/FETCH_SNAPSHOT request from the leader itself."); } @@ -293,25 +341,32 @@ private static List convertToVoters(Set voterIds) { .collect(Collectors.toList()); } - public void appendStartOfEpochControlRecords(VoterSet.VoterNode localVoterNode, long currentTimeMs) { - if (!localReplicaKey.equals(localVoterNode.voterKey())) { - throw new IllegalArgumentException( - String.format( - "Replica key %s didn't match the local key %s", - localVoterNode.voterKey(), - localReplicaKey - ) - ); - } else if (!localListeners.equals(localVoterNode.listeners())) { - throw new IllegalArgumentException( - String.format( - "Listeners %s didn't match the local listeners %s", - localVoterNode.listeners(), - localListeners - ) - ); - } + private static MemoryRecordsBuilder createControlRecordsBuilder( + long baseOffset, + int epoch, + Compression compression, + ByteBuffer buffer, + long currentTimeMs + ) { + return new MemoryRecordsBuilder( + buffer, + RecordBatch.CURRENT_MAGIC_VALUE, + compression, + TimestampType.CREATE_TIME, + baseOffset, + currentTimeMs, + RecordBatch.NO_PRODUCER_ID, + RecordBatch.NO_PRODUCER_EPOCH, + RecordBatch.NO_SEQUENCE, + false, // isTransactional + true, // isControlBatch + epoch, + buffer.capacity() + ); + } + + public void appendStartOfEpochControlRecords(long currentTimeMs) { List voters = convertToVoters(voterStates.keySet()); List grantingVoters = convertToVoters(this.grantingVoters()); @@ -322,20 +377,12 @@ public void appendStartOfEpochControlRecords(VoterSet.VoterNode localVoterNode, .setGrantingVoters(grantingVoters); accumulator.appendControlMessages((baseOffset, epoch, compression, buffer) -> { - try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder( - buffer, - RecordBatch.CURRENT_MAGIC_VALUE, - compression, - TimestampType.CREATE_TIME, + try (MemoryRecordsBuilder builder = createControlRecordsBuilder( baseOffset, - currentTimeMs, - RecordBatch.NO_PRODUCER_ID, - RecordBatch.NO_PRODUCER_EPOCH, - RecordBatch.NO_SEQUENCE, - false, // isTransactional - true, // isControlBatch epoch, - buffer.capacity() + compression, + buffer, + currentTimeMs ) ) { builder.appendLeaderChangeMessage(currentTimeMs, leaderChangeMessage); @@ -395,6 +442,21 @@ public long appendVotersRecord(VoterSet voters, long currentTimeMs) { ); } + public boolean compareAndSetVolatileVoters( + KRaftVersionUpgrade.Voters oldVoters, + KRaftVersionUpgrade.Voters newVoters + ) { + return kraftVersionUpgradeState.compareAndSet(oldVoters, newVoters); + } + + public Optional volatileVoters() { + return kraftVersionUpgradeState.get().toVoters(); + } + + public Optional requestedKRaftVersion() { + return kraftVersionUpgradeState.get().toVersion(); + } + public boolean isResignRequested() { return resignRequested; } @@ -416,6 +478,185 @@ public void requestResign() { this.resignRequested = true; } + /** + * Upgrade the kraft version. + * + * This methods upgrades the kraft version to {@code newVersion}. If the version is already + * {@code newVersion}, this is a noop operation. + * + * KRaft only supports upgrades, so {@code newVersion} must be greater than or equal to curent + * kraft version {@code persistedVersion}. + * + * For the upgrade to succeed all of the voters in the voter set must support the new kraft + * version. The upgrade from kraft version 0 to kraft version 1 generate one control batch + * with one control record setting the kraft version to 1 and one voters record setting the + * updated voter set. + * + * When {@code validateOnly} is true only the validation is perform and the control records are + * not generated. + * + * @param currentEpoch the current epoch + * @param newVersion the new kraft version + * @param persistedVersion the kraft version persisted to disk + * @param persistedVoters the set of voters persisted to disk + * @param validateOnly determine if only validation should be performed + * @param currentTimeMs the current time + */ + public boolean maybeAppendUpgradedKRaftVersion( + int currentEpoch, + KRaftVersion newVersion, + KRaftVersion persistedVersion, + VoterSet persistedVoters, + boolean validateOnly, + long currentTimeMs + ) { + validateEpoch(currentEpoch); + + var pendingVersion = kraftVersionUpgradeState.get().toVersion(); + if (pendingVersion.isPresent()) { + if (pendingVersion.get().kraftVersion().equals(newVersion)) { + // The version match; upgrade is a noop + return false; + } else { + throw new InvalidUpdateVersionException( + String.format( + "Invalid concurrent upgrade of %s from version %s to %s", + KRaftVersion.FEATURE_NAME, + pendingVersion.get(), + newVersion + ) + ); + } + } else if (persistedVersion.equals(newVersion)) { + return false; + } else if (persistedVersion.isMoreThan(newVersion)) { + throw new InvalidUpdateVersionException( + String.format( + "Invalid upgrade of %s from version %s to %s because the new version is a downgrade", + KRaftVersion.FEATURE_NAME, + persistedVersion, + newVersion + ) + ); + } + + // Upgrade to kraft.verion 1 is only supported; this needs to change when kraft.version 2 is added + var inMemoryVoters = kraftVersionUpgradeState.get().toVoters().orElseThrow(() -> + new InvalidUpdateVersionException( + String.format( + "Invalid upgrade of %s from version %s to %s", + KRaftVersion.FEATURE_NAME, + persistedVersion, + newVersion + ) + ) + ); + if (!inMemoryVoters.voters().voterIds().equals(persistedVoters.voterIds())) { + throw new IllegalStateException( + String.format( + "Unable to update %s to %s due to missing voters %s compared to %s", + KRaftVersion.FEATURE_NAME, + newVersion, + inMemoryVoters.voters().voterIds(), + persistedVoters.voterIds() + ) + ); + } else if (!inMemoryVoters.voters().supportsVersion(newVersion)) { + log.info("Not all voters support kraft version {}: {}", newVersion, inMemoryVoters.voters()); + throw new InvalidUpdateVersionException( + String.format( + "Invalid upgrade of %s to %s because not all of the voters support it", + KRaftVersion.FEATURE_NAME, + newVersion + ) + ); + } else if ( + inMemoryVoters + .voters() + .voterKeys() + .stream() + .anyMatch(voterKey -> voterKey.directoryId().isEmpty()) + ) { + throw new IllegalStateException( + String.format( + "Directory id must be known for all of the voters: %s", + inMemoryVoters.voters() + ) + ); + } + + if (!validateOnly) { + /* Note that this only supports upgrades from kraft.version 0 to kraft.version 1. When + * kraft.version 2 is added, this logic needs to be revisited + */ + var successful = kraftVersionUpgradeState.compareAndSet( + inMemoryVoters, + new KRaftVersionUpgrade.Version(newVersion) + ); + if (!successful) { + throw new InvalidUpdateVersionException( + String.format( + "Unable to upgrade version for %s to %s due to changing voters", + KRaftVersion.FEATURE_NAME, + newVersion + ) + ); + } + + // All of the validations succeeded; create control records for the upgrade + accumulator.appendControlMessages((baseOffset, batchEpoch, compression, buffer) -> { + try (MemoryRecordsBuilder builder = createControlRecordsBuilder( + baseOffset, + batchEpoch, + compression, + buffer, + currentTimeMs + ) + ) { + log.info("Appended kraft.version {} to the batch accumulator", newVersion); + builder.appendKRaftVersionMessage( + currentTimeMs, + new KRaftVersionRecord() + .setVersion(newVersion.kraftVersionRecordVersion()) + .setKRaftVersion(newVersion.featureLevel()) + ); + + if (!inMemoryVoters.voters().equals(persistedVoters)) { + log.info("Appended voter set {} to the batch accumulator", inMemoryVoters.voters()); + builder.appendVotersMessage( + currentTimeMs, + inMemoryVoters.voters().toVotersRecord(newVersion.votersRecordVersion()) + ); + } + + return builder.build(); + } + }); + } + + return true; + } + + private void validateEpoch(int currentEpoch) { + if (currentEpoch < epoch()) { + throw new NotLeaderException( + String.format( + "Upgrade kraft version failed because the given epoch %s is stale. Current leader epoch is %s", + currentEpoch, + epoch() + ) + ); + } else if (currentEpoch > epoch()) { + throw new IllegalArgumentException( + String.format( + "Attempt to append from epoch %s which is larger than the current epoch of %s", + currentEpoch, + epoch() + ) + ); + } + } + @Override public Optional highWatermark() { return highWatermark; @@ -423,7 +664,7 @@ public Optional highWatermark() { @Override public ElectionState election() { - return ElectionState.withElectedLeader(epoch, localReplicaKey.id(), Optional.empty(), voterStates.keySet()); + return ElectionState.withElectedLeader(epoch, localVoterNode.voterKey().id(), Optional.empty(), voterStates.keySet()); } @Override @@ -433,7 +674,7 @@ public int epoch() { @Override public Endpoints leaderEndpoints() { - return localListeners; + return localVoterNode.listeners(); } Map voterStates() { @@ -557,7 +798,7 @@ public boolean updateLocalState( LogOffsetMetadata endOffsetMetadata, VoterSet lastVoterSet ) { - ReplicaState state = getOrCreateReplicaState(localReplicaKey); + ReplicaState state = getOrCreateReplicaState(localVoterNode.voterKey()); state.endOffset.ifPresent(currentEndOffset -> { if (currentEndOffset.offset() > endOffsetMetadata.offset()) { throw new IllegalStateException("Detected non-monotonic update of local " + @@ -588,7 +829,7 @@ public boolean updateReplicaState( // the fetch is from non-replica. For example, a consumer. if (replicaKey.id() < 0) { return false; - } else if (replicaKey.id() == localReplicaKey.id()) { + } else if (replicaKey.id() == localVoterNode.voterKey().id()) { throw new IllegalStateException( String.format("Remote replica ID %s matches the local leader ID", replicaKey) ); @@ -603,7 +844,7 @@ public boolean updateReplicaState( } }); - Optional leaderEndOffsetOpt = getOrCreateReplicaState(localReplicaKey).endOffset; + Optional leaderEndOffsetOpt = getOrCreateReplicaState(localVoterNode.voterKey()).endOffset; state.updateFollowerState( currentTimeMs, @@ -617,7 +858,7 @@ public boolean updateReplicaState( public List nonLeaderVotersByDescendingFetchOffset() { return followersByDescendingFetchOffset() - .filter(state -> !state.matchesKey(localReplicaKey)) + .filter(state -> !state.matchesKey(localVoterNode.voterKey())) .map(state -> state.replicaKey) .collect(Collectors.toList()); } @@ -671,7 +912,7 @@ public Optional getReplicaState(ReplicaKey replicaKey) { private void clearInactiveObservers(final long currentTimeMs) { observerStates.entrySet().removeIf(integerReplicaStateEntry -> currentTimeMs - integerReplicaStateEntry.getValue().lastFetchTimestamp >= OBSERVER_SESSION_TIMEOUT_MS && - !integerReplicaStateEntry.getKey().equals(localReplicaKey) + !integerReplicaStateEntry.getKey().equals(localVoterNode.voterKey()) ); kafkaRaftMetrics.updateNumObservers(observerStates.size()); } @@ -864,8 +1105,8 @@ public boolean canGrantVote(ReplicaKey replicaKey, boolean isLogUpToDate, boolea @Override public String toString() { return String.format( - "Leader(localReplicaKey=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)", - localReplicaKey, + "Leader(localVoterNode=%s, epoch=%d, epochStartOffset=%d, highWatermark=%s, voterStates=%s)", + localVoterNode, epoch, epochStartOffset, highWatermark, diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java index 32c3a1b9152f5..d0450a9b0470c 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumState.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumState.java @@ -716,7 +716,7 @@ public LeaderState transitionToLeader(long epochStartOffset, BatchAccumul LeaderState state = new LeaderState<>( time, - ReplicaKey.of(localIdOrThrow(), localDirectoryId), + localVoterNodeOrThrow(), epoch(), epochStartOffset, partitionState.lastVoterSet(), @@ -724,7 +724,6 @@ public LeaderState transitionToLeader(long epochStartOffset, BatchAccumul partitionState.lastKraftVersion(), candidateState.epochElection().grantingVoters(), accumulator, - localListeners, fetchTimeoutMs, logContext, kafkaRaftMetrics diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index a7205b746cd2b..90131ce457573 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -228,7 +228,7 @@ default void beginShutdown() {} * Returns the current end of the log. This method is thread-safe. * * @return the log end offset, which is one greater than the offset of the last record written, - * or 0 if there have not been any records written. + * or 0 if there have not been any records written. */ long logEndOffset(); @@ -238,4 +238,18 @@ default void beginShutdown() {} * @return the current kraft.version. */ KRaftVersion kraftVersion(); + + /** + * Request that the leader to upgrade the kraft version. + * + * @param epoch the current epoch + * @param version the new kraft version to upgrade to + * @param validateOnly whether to just validate the change and not persist it + * @throws ApiException when the upgrade fails to validate + */ + void upgradeKRaftVersion( + int epoch, + KRaftVersion version, + boolean validateOnly + ); } diff --git a/raft/src/main/java/org/apache/kafka/raft/VoterSet.java b/raft/src/main/java/org/apache/kafka/raft/VoterSet.java index cad4dfe008770..23f968cdeeb58 100644 --- a/raft/src/main/java/org/apache/kafka/raft/VoterSet.java +++ b/raft/src/main/java/org/apache/kafka/raft/VoterSet.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.message.VotersRecord; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.common.KRaftVersion; import java.net.InetSocketAddress; import java.util.HashMap; @@ -250,6 +251,26 @@ public Optional updateVoter(VoterNode voter) { return Optional.empty(); } + /** + * Update a voter by only comparing the node id. + * + * This update voter operation doesn't compare the directory id. This is useful when upgrading + * from a voter set that doesn't support directory id to one that supports directory ids. + * + * @param voter the updated voter + * @return a new voter set if the voter was updated, otherwise {@code Optional.empty()} + */ + public Optional updateVoterIgnoringDirectoryId(VoterNode voter) { + if (voters.containsKey(voter.voterKey().id())) { + HashMap newVoters = new HashMap<>(voters); + newVoters.put(voter.voterKey().id(), voter); + + return Optional.of(new VoterSet(newVoters)); + } + + return Optional.empty(); + } + /** * Converts a voter set to a voters record for a given version. * @@ -303,6 +324,19 @@ public boolean hasOverlappingMajority(VoterSet that) { return Utils.diff(HashSet::new, thatReplicaKeys, thisReplicaKeys).size() <= 1; } + /** + * Determines if the voter set supports a given kraft version. + * + * @param version the kraft version to check + * @return true if all of the voter support the given kraft version, false otherwise + */ + public boolean supportsVersion(KRaftVersion version) { + return voters + .values() + .stream() + .allMatch(voter -> voter.supportsVersion(version)); + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -373,15 +407,20 @@ public Endpoints listeners() { return listeners; } - SupportedVersionRange supportedKRaftVersion() { + private SupportedVersionRange supportedKRaftVersion() { return supportedKRaftVersion; } - Optional address(ListenerName listener) { + private Optional address(ListenerName listener) { return listeners.address(listener); } + private boolean supportsVersion(KRaftVersion version) { + return version.featureLevel() >= supportedKRaftVersion.min() && + version.featureLevel() <= supportedKRaftVersion.max(); + } + @Override public boolean equals(Object o) { if (this == o) return true; diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/KRaftVersionUpgrade.java b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftVersionUpgrade.java new file mode 100644 index 0000000000000..22d0e8f894675 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/KRaftVersionUpgrade.java @@ -0,0 +1,65 @@ +/* + * 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.raft.internals; + +import org.apache.kafka.raft.VoterSet; +import org.apache.kafka.server.common.KRaftVersion; + +import java.util.Optional; + +/** + * KRaftVersionUpgrade is a sum types for coordinating the upgrade of the kraft version. + * + * {@code Voters} is used to stored in-memory the latest voter set. {@code Version} is used to + * store in-memory the upgraded kraft version. + * + * This type makes it possible to upgrade the kraft version by only using compare-and-swap and + * avoid blocking locks. + */ +public sealed interface KRaftVersionUpgrade { + public record Empty() implements KRaftVersionUpgrade { + } + + public record Version(KRaftVersion kraftVersion) implements KRaftVersionUpgrade { + } + + public record Voters(VoterSet voters) implements KRaftVersionUpgrade { + } + + public default Optional toVoters() { + if (this instanceof Voters) { + return Optional.of(((Voters) this)); + } else { + return Optional.empty(); + } + } + + public default Optional toVersion() { + if (this instanceof Version) { + return Optional.of(((Version) this)); + } else { + return Optional.empty(); + } + } + + static final KRaftVersionUpgrade EMPTY = new Empty(); + + public static KRaftVersionUpgrade empty() { + return EMPTY; + } + +} diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/RequestSendResult.java b/raft/src/main/java/org/apache/kafka/raft/internals/RequestSendResult.java new file mode 100644 index 0000000000000..891d8e32509f1 --- /dev/null +++ b/raft/src/main/java/org/apache/kafka/raft/internals/RequestSendResult.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft.internals; + +/** + * Type to capture the atempt to send a request. + * + * @param requestSent true if the request was sent + * @param timeToWaitMs the amount of time to wait in milliseconds before attempting to resend the + * request. + */ +public record RequestSendResult(boolean requestSent, long timeToWaitMs) { + public static RequestSendResult of(boolean requestSent, long timeToWaitMs) { + return new RequestSendResult(requestSent, timeToWaitMs); + } +} diff --git a/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java b/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java index 47eec01581aed..6f98ff80f6917 100644 --- a/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java +++ b/raft/src/main/java/org/apache/kafka/raft/internals/UpdateVoterHandler.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.message.UpdateRaftVoterResponseData; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.raft.Endpoints; import org.apache.kafka.raft.LeaderAndEpoch; import org.apache.kafka.raft.LeaderState; @@ -30,6 +31,8 @@ import org.apache.kafka.raft.VoterSet; import org.apache.kafka.server.common.KRaftVersion; +import org.slf4j.Logger; + import java.util.Optional; import java.util.OptionalInt; import java.util.concurrent.CompletableFuture; @@ -44,25 +47,29 @@ * 4. Check that the updated voter still supports the currently finalized kraft.version, otherwise * return the INVALID_REQUEST error. * 5. Check that the updated voter is still listening on the default listener. - * 6. Append the updated VotersRecord to the log. The KRaft internal listener will read this - * uncommitted record from the log and update the voter in the set of voters. + * 6. Update voter set with new voter configuration. + * a. If reconfiguration is supported, append the updated VotersRecord to the log. The KRaft internal listener will read this + * uncommitted record from the log and update the voter in the set of voters. + * b. If reconfiguration is not supported, update the in-memory information for the voter. This will get + * appended to the log when the cluster is upgraded to a kraft version that supports reconfiguration. * 7. Send the UpdateVoter successful response to the voter. - * - * KAFKA-16538 is going to add support for handling this RPC when the kraft.version is 0. */ public final class UpdateVoterHandler { private final OptionalInt localId; private final KRaftControlRecordStateMachine partitionState; private final ListenerName defaultListenerName; + private final Logger log; public UpdateVoterHandler( OptionalInt localId, KRaftControlRecordStateMachine partitionState, - ListenerName defaultListenerName + ListenerName defaultListenerName, + LogContext logContext ) { this.localId = localId; this.partitionState = partitionState; this.defaultListenerName = defaultListenerName; + this.log = logContext.logger(getClass()); } public CompletableFuture handleUpdateVoterRequest( @@ -104,26 +111,42 @@ public CompletableFuture handleUpdateVoterRequest( ); } - // KAFKA-16538 will implement the case when the kraft.version is 0 - // Check that the cluster supports kraft.version >= 1 + // Read the voter set from the log or leader state KRaftVersion kraftVersion = partitionState.lastKraftVersion(); - if (!kraftVersion.isReconfigSupported()) { - return CompletableFuture.completedFuture( - RaftUtil.updateVoterResponse( - Errors.UNSUPPORTED_VERSION, - requestListenerName, - new LeaderAndEpoch( - localId, - leaderState.epoch() - ), - leaderState.leaderEndpoints() - ) - ); - } + final Optional inMemoryVoters; + final Optional voters; + if (kraftVersion.isReconfigSupported()) { + inMemoryVoters = Optional.empty(); - // Check that there are no uncommitted VotersRecord - Optional> votersEntry = partitionState.lastVoterSetEntry(); - if (votersEntry.isEmpty() || votersEntry.get().offset() >= highWatermark.get()) { + // Check that there are no uncommitted VotersRecord + Optional> votersEntry = partitionState.lastVoterSetEntry(); + if (votersEntry.isEmpty() || votersEntry.get().offset() >= highWatermark.get()) { + voters = Optional.empty(); + } else { + voters = votersEntry.map(LogHistory.Entry::value); + } + } else { + inMemoryVoters = leaderState.volatileVoters(); + if (inMemoryVoters.isEmpty()) { + /* This can happen if the remote voter sends an update voter request before the + * updated kraft version has been written to the log + */ + return CompletableFuture.completedFuture( + RaftUtil.updateVoterResponse( + Errors.REQUEST_TIMED_OUT, + requestListenerName, + new LeaderAndEpoch( + localId, + leaderState.epoch() + ), + leaderState.leaderEndpoints() + ) + ); + } + voters = inMemoryVoters.map(KRaftVersionUpgrade.Voters::voters); + } + if (voters.isEmpty()) { + log.info("Unable to read the current voter set with kraft version {}", kraftVersion); return CompletableFuture.completedFuture( RaftUtil.updateVoterResponse( Errors.REQUEST_TIMED_OUT, @@ -136,7 +159,6 @@ public CompletableFuture handleUpdateVoterRequest( ) ); } - // Check that the supported version range is valid if (!validVersionRange(kraftVersion, supportedKraftVersions)) { return CompletableFuture.completedFuture( @@ -168,19 +190,18 @@ public CompletableFuture handleUpdateVoterRequest( } // Update the voter - Optional updatedVoters = votersEntry - .get() - .value() - .updateVoter( - VoterSet.VoterNode.of( - voterKey, - voterEndpoints, - new SupportedVersionRange( - supportedKraftVersions.minSupportedVersion(), - supportedKraftVersions.maxSupportedVersion() - ) + Optional updatedVoters = updateVoters( + voters.get(), + kraftVersion, + VoterSet.VoterNode.of( + voterKey, + voterEndpoints, + new SupportedVersionRange( + supportedKraftVersions.minSupportedVersion(), + supportedKraftVersions.maxSupportedVersion() ) - ); + ) + ); if (updatedVoters.isEmpty()) { return CompletableFuture.completedFuture( RaftUtil.updateVoterResponse( @@ -195,9 +216,80 @@ public CompletableFuture handleUpdateVoterRequest( ); } - leaderState.appendVotersRecord(updatedVoters.get(), currentTimeMs); + return storeUpdatedVoters( + leaderState, + voterKey, + inMemoryVoters, + updatedVoters.get(), + requestListenerName, + currentTimeMs + ); + } + + private boolean validVersionRange( + KRaftVersion finalizedVersion, + UpdateRaftVoterRequestData.KRaftVersionFeature supportedKraftVersions + ) { + return supportedKraftVersions.minSupportedVersion() <= finalizedVersion.featureLevel() && + supportedKraftVersions.maxSupportedVersion() >= finalizedVersion.featureLevel(); + } + + private Optional updateVoters( + VoterSet voters, + KRaftVersion kraftVersion, + VoterSet.VoterNode updatedVoter + ) { + return kraftVersion.isReconfigSupported() ? + voters.updateVoter(updatedVoter) : + voters.updateVoterIgnoringDirectoryId(updatedVoter); + } + + private CompletableFuture storeUpdatedVoters( + LeaderState leaderState, + ReplicaKey voterKey, + Optional inMemoryVoters, + VoterSet newVoters, + ListenerName requestListenerName, + long currentTimeMs + ) { + if (inMemoryVoters.isEmpty()) { + // Since the partition support reconfig then just write the update voter set directly to the log + leaderState.appendVotersRecord(newVoters, currentTimeMs); + } else { + // Store the new voters set in the leader state since it cannot be written to the log + var successful = leaderState.compareAndSetVolatileVoters( + inMemoryVoters.get(), + new KRaftVersionUpgrade.Voters(newVoters) + ); + if (successful) { + log.info( + "Updated in-memory voters from {} to {}", + inMemoryVoters.get().voters(), + newVoters + ); + } else { + log.info( + "Unable to update in-memory voters from {} to {}", + inMemoryVoters.get().voters(), + newVoters + ); + return CompletableFuture.completedFuture( + RaftUtil.updateVoterResponse( + Errors.REQUEST_TIMED_OUT, + requestListenerName, + new LeaderAndEpoch( + localId, + leaderState.epoch() + ), + leaderState.leaderEndpoints() + ) + ); + } + } + + // Reset the check quorum state since the leader received a successful request + leaderState.updateCheckQuorumForFollowingVoter(voterKey, currentTimeMs); - // Reply immediately and don't wait for the change to commit return CompletableFuture.completedFuture( RaftUtil.updateVoterResponse( Errors.NONE, @@ -210,12 +302,4 @@ public CompletableFuture handleUpdateVoterRequest( ) ); } - - private boolean validVersionRange( - KRaftVersion finalizedVersion, - UpdateRaftVoterRequestData.KRaftVersionFeature supportedKraftVersions - ) { - return supportedKraftVersions.minSupportedVersion() <= finalizedVersion.featureLevel() && - supportedKraftVersions.maxSupportedVersion() >= finalizedVersion.featureLevel(); - } } diff --git a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java index 021d6791837e1..3f6fb61894c35 100644 --- a/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/FollowerStateTest.java @@ -140,4 +140,13 @@ public void testLeaderIdAndEndpoint() { assertEquals(leaderId, state.leaderId()); assertEquals(leaderEndpoints, state.leaderEndpoints()); } + + @Test + public void testHasUpdatedLeader() { + FollowerState state = newFollowerState(Set.of(0, 1, 2)); + + assertFalse(state.hasUpdatedLeader()); + state.setHasUpdatedLeader(); + assertTrue(state.hasUpdatedLeader()); + } } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 0c4c1780919b2..dc04f85d0b81b 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -17,7 +17,9 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Node; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.compress.Compression; +import org.apache.kafka.common.errors.InvalidUpdateVersionException; import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData; @@ -46,6 +48,8 @@ import org.apache.kafka.snapshot.SnapshotWriterReaderTest; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.mockito.Mockito; import java.net.InetSocketAddress; @@ -59,17 +63,21 @@ import java.util.OptionalInt; import java.util.OptionalLong; import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.IntStream; import java.util.stream.Stream; import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol; import static org.apache.kafka.snapshot.Snapshots.BOOTSTRAP_SNAPSHOT_ID; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class KafkaRaftClientReconfigTest { + private static final int NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD = 1; @Test public void testLeaderWritesBootstrapRecords() throws Exception { @@ -1695,6 +1703,53 @@ public void testUpdateVoterInvalidClusterId() throws Exception { ); } + @Test + public void testInvalidUpdateVoter() throws Exception { + ReplicaKey local = replicaKey(randomReplicaId(), true); + ReplicaKey follower = replicaKey(local.id() + 1, true); + + VoterSet voters = VoterSetTest.voterSet(Stream.of(local, follower)); + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(voters)) + .withUnknownLeader(3) + .build(); + + context.unattachedToLeader(); + int epoch = context.currentEpoch(); + + // missing directory id + context.deliverRequest( + context.updateVoterRequest( + ReplicaKey.of(follower.id(), Uuid.ZERO_UUID), + Feature.KRAFT_VERSION.supportedVersionRange(), + Endpoints.empty() + ) + ); + context.pollUntilResponse(); + context.assertSentUpdateVoterResponse( + Errors.INVALID_REQUEST, + OptionalInt.of(local.id()), + epoch + ); + + // missing endpoints + context.deliverRequest( + context.updateVoterRequest( + follower, + Feature.KRAFT_VERSION.supportedVersionRange(), + Endpoints.empty() + ) + ); + context.pollUntilResponse(); + context.assertSentUpdateVoterResponse( + Errors.INVALID_REQUEST, + OptionalInt.of(local.id()), + epoch + ); + } + @Test void testUpdateVoterOldEpoch() throws Exception { ReplicaKey local = replicaKey(randomReplicaId(), true); @@ -1834,7 +1889,6 @@ void testUpdateVoterWithoutFencedPreviousLeaders() throws Exception { ); } - // KAFKA-16538 is going to allow UpdateVoter RPC when the kraft.version is 0 @Test void testUpdateVoterWithKraftVersion0() throws Exception { ReplicaKey local = replicaKey(randomReplicaId(), true); @@ -1880,7 +1934,7 @@ void testUpdateVoterWithKraftVersion0() throws Exception { ); context.pollUntilResponse(); context.assertSentUpdateVoterResponse( - Errors.UNSUPPORTED_VERSION, + Errors.NONE, OptionalInt.of(local.id()), epoch ); @@ -2085,8 +2139,8 @@ void testFollowerSendsUpdateVoter() throws Exception { .withLocalListeners(localListeners) .build(); - // waiting for 3 times the fetch timeout sends an update voter - for (int i = 0; i < 3; i++) { + // waiting for FETCH requests until the UpdateRaftVoter request is sent + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); @@ -2130,6 +2184,444 @@ void testFollowerSendsUpdateVoter() throws Exception { context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); } + @ParameterizedTest + @EnumSource(value = Errors.class, names = {"NONE", "UNSUPPORTED_VERSION"}) + void testFollowerSendsUpdateVoterWithKraftVersion0(Errors updateVoterError) throws Exception { + ReplicaKey local = replicaKey(randomReplicaId(), true); + ReplicaKey voter1 = replicaKey(local.id() + 1, true); + ReplicaKey voter2 = replicaKey(local.id() + 2, true); + + VoterSet voters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2)); + int epoch = 4; + + HashMap listenersMap = new HashMap<>(2); + listenersMap.put( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 9990 + local.id()) + ); + listenersMap.put( + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved("localhost", 8990 + local.id()) + ); + Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap); + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withKip853Rpc(true) + .withStaticVoters(voters) + .withElectedLeader(epoch, voter1.id()) + .withLocalListeners(localListeners) + .build(); + + // waiting for FETCH request until the UpdateRaftVoter request is set + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + epoch, + voter1.id(), + MemoryRecords.EMPTY, + 0L, + Errors.NONE + ) + ); + // poll kraft to handle the fetch response + context.client.poll(); + } + + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest( + local, + epoch, + Feature.KRAFT_VERSION.supportedVersionRange(), + localListeners + ); + context.deliverResponse( + updateRequest.correlationId(), + updateRequest.destination(), + context.updateVoterResponse( + updateVoterError, + new LeaderAndEpoch(OptionalInt.of(voter1.id()), epoch) + ) + ); + context.client.poll(); + + // after sending an update voter the next requests should be fetch and no update voter + for (int i = 0; i < 10; i++) { + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + epoch, + voter1.id(), + MemoryRecords.EMPTY, + 0L, + Errors.NONE + ) + ); + // poll kraft to handle the fetch response + context.client.poll(); + } + } + + @ParameterizedTest + @EnumSource(value = Errors.class, names = {"NONE", "UNSUPPORTED_VERSION"}) + void testFollowerSendsUpdateVoterAfterElectionWithKraftVersion0(Errors updateVoterError) throws Exception { + ReplicaKey local = replicaKey(randomReplicaId(), true); + ReplicaKey voter1 = replicaKey(local.id() + 1, true); + ReplicaKey voter2 = replicaKey(local.id() + 2, true); + + VoterSet voters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2)); + int epoch = 4; + + HashMap listenersMap = new HashMap<>(2); + listenersMap.put( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 9990 + local.id()) + ); + listenersMap.put( + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved("localhost", 8990 + local.id()) + ); + Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap); + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withKip853Rpc(true) + .withStaticVoters(voters) + .withElectedLeader(epoch, voter1.id()) + .withLocalListeners(localListeners) + .build(); + + // waiting for FETCH request until the UpdateRaftVoter request is set + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + epoch, + voter1.id(), + MemoryRecords.EMPTY, + 0L, + Errors.NONE + ) + ); + // poll kraft to handle the fetch response + context.client.poll(); + } + + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest( + local, + epoch, + Feature.KRAFT_VERSION.supportedVersionRange(), + localListeners + ); + context.deliverResponse( + updateRequest.correlationId(), + updateRequest.destination(), + context.updateVoterResponse( + updateVoterError, + new LeaderAndEpoch(OptionalInt.of(voter1.id()), epoch) + ) + ); + + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + // Election a new leader causes the replica to resend update voter request + int newEpoch = epoch + 1; + context.deliverRequest(context.beginEpochRequest(newEpoch, voter1.id())); + context.pollUntilResponse(); + + // waiting for FETCH request until the UpdateRaftVoter request is set + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0); + + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + newEpoch, + voter1.id(), + MemoryRecords.EMPTY, + 0L, + Errors.NONE + ) + ); + // poll kraft to handle the fetch response + context.client.poll(); + } + + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + updateRequest = context.assertSentUpdateVoterRequest( + local, + newEpoch, + Feature.KRAFT_VERSION.supportedVersionRange(), + localListeners + ); + context.deliverResponse( + updateRequest.correlationId(), + updateRequest.destination(), + context.updateVoterResponse( + Errors.NONE, + new LeaderAndEpoch(OptionalInt.of(voter1.id()), newEpoch) + ) + ); + + context.pollUntilRequest(); + fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0); + } + + @Test + void testKRaftUpgradeVersion() throws Exception { + var local = replicaKey(randomReplicaId(), true); + var voter1 = replicaKey(local.id() + 1, true); + var voter2 = replicaKey(local.id() + 2, true); + + VoterSet startingVoters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(local.id(), voter1.id(), voter2.id()), false) + ); + + var context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL) + .withStartingVoters(startingVoters, KRaftVersion.KRAFT_VERSION_0) + .build(); + + context.unattachedToLeader(); + var epoch = context.currentEpoch(); + + // Establish a HWM and fence previous leaders + for (var voter : List.of(voter1, voter2)) { + context.deliverRequest( + context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0) + ); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id())); + } + + // Update voters so that they supports kraft version 1 + for (var voter : List.of(voter1, voter2)) { + context.deliverRequest( + context.updateVoterRequest( + voter, + Feature.KRAFT_VERSION.supportedVersionRange(), + startingVoters.listeners(voter.id()) + ) + ); + context.pollUntilResponse(); + context.assertSentUpdateVoterResponse( + Errors.NONE, + OptionalInt.of(local.id()), + epoch + ); + } + + context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false); + assertEquals(KRaftVersion.KRAFT_VERSION_1, context.client.kraftVersion()); + + var localLogEndOffset = context.log.endOffset().offset(); + context.client.poll(); + + // check if leader writes 2 control records to the log; + // one for the kraft version and one for the voter set + var updatedVoters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2)); + var records = context.log.read(localLogEndOffset, Isolation.UNCOMMITTED).records; + var batch = records.batches().iterator().next(); + assertTrue(batch.isControlBatch()); + var recordsIterator = batch.iterator(); + var controlRecord = recordsIterator.next(); + verifyKRaftVersionRecord( + KRaftVersion.KRAFT_VERSION_1.featureLevel(), + controlRecord.key(), + controlRecord.value() + ); + controlRecord = recordsIterator.next(); + verifyVotersRecord(updatedVoters, controlRecord.key(), controlRecord.value()); + } + + @Test + void testUpdateVoterAfterKRaftVersionUpgrade() throws Exception { + var local = replicaKey(randomReplicaId(), true); + var voter1 = replicaKey(local.id() + 1, true); + var voter2 = replicaKey(local.id() + 2, true); + + VoterSet startingVoters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(local.id(), voter1.id(), voter2.id()), false) + ); + + var context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL) + .withStartingVoters(startingVoters, KRaftVersion.KRAFT_VERSION_0) + .build(); + + context.unattachedToLeader(); + var epoch = context.currentEpoch(); + + // Establish a HWM and fence previous leaders + for (var voter : List.of(voter1, voter2)) { + context.deliverRequest( + context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0) + ); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id())); + } + + // Update voters so that they supports kraft version 1 + for (var voter : List.of(voter1, voter2)) { + context.deliverRequest( + context.updateVoterRequest( + voter, + Feature.KRAFT_VERSION.supportedVersionRange(), + startingVoters.listeners(voter.id()) + ) + ); + context.pollUntilResponse(); + context.assertSentUpdateVoterResponse( + Errors.NONE, + OptionalInt.of(local.id()), + epoch + ); + } + + context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false); + assertEquals(KRaftVersion.KRAFT_VERSION_1, context.client.kraftVersion()); + + // Push the control records to the log + context.client.poll(); + // Advance the HWM to the LEO + for (var voter : List.of(voter1, voter2)) { + context.deliverRequest( + context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0) + ); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id())); + } + + // Check that it can still handle update voter request after upgrade + Endpoints newVoter1Listeners = Endpoints.fromInetSocketAddresses( + Map.of( + // first entry + context.channel.listenerName(), + InetSocketAddress.createUnresolved( + "localhost", + 9990 + voter1.id() + ), + // second entry + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved( + "localhost", + 8990 + voter1.id() + ) + ) + ); + context.deliverRequest( + context.updateVoterRequest( + voter1, + Feature.KRAFT_VERSION.supportedVersionRange(), + newVoter1Listeners + ) + ); + context.pollUntilResponse(); + context.assertSentUpdateVoterResponse( + Errors.NONE, + OptionalInt.of(local.id()), + epoch + ); + + // Push the control records to the log + var localLogEndOffset = context.log.endOffset().offset(); + context.client.poll(); + + // check that the leader wrote voters control record to the log; + var records = context.log.read(localLogEndOffset, Isolation.UNCOMMITTED).records; + var batch = records.batches().iterator().next(); + assertTrue(batch.isControlBatch()); + var recordsIterator = batch.iterator(); + var controlRecord = recordsIterator.next(); + assertEquals(ControlRecordType.KRAFT_VOTERS, ControlRecordType.parse(controlRecord.key())); + ControlRecordUtils.deserializeVotersRecord(controlRecord.value()); + } + + @Test + void testInvalidKRaftUpgradeVersion() throws Exception { + var local = replicaKey(randomReplicaId(), true); + var voter1 = replicaKey(local.id() + 1, true); + var voter2 = replicaKey(local.id() + 2, true); + + VoterSet startingVoters = VoterSetTest.voterSet( + VoterSetTest.voterMap(IntStream.of(local.id(), voter1.id(), voter2.id()), false) + ); + + var context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withRaftProtocol(RaftProtocol.KIP_853_PROTOCOL) + .withStartingVoters(startingVoters, KRaftVersion.KRAFT_VERSION_0) + .build(); + + context.unattachedToLeader(); + var epoch = context.currentEpoch(); + + // Upgrade not allowed since none of the remote voters support the new version + assertEquals(KRaftVersion.KRAFT_VERSION_0, context.client.kraftVersion()); + assertThrows( + InvalidUpdateVersionException.class, + () -> context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false) + ); + + // Establish a HWM and fence previous leaders + for (var voter : List.of(voter1, voter2)) { + context.deliverRequest( + context.fetchRequest(epoch, voter, context.log.endOffset().offset(), epoch, 0) + ); + context.pollUntilResponse(); + context.assertSentFetchPartitionResponse(Errors.NONE, epoch, OptionalInt.of(local.id())); + } + + // Update only one of the voters so that they supports kraft version 1 + context.deliverRequest( + context.updateVoterRequest( + voter1, + Feature.KRAFT_VERSION.supportedVersionRange(), + startingVoters.listeners(voter1.id()) + ) + ); + context.pollUntilResponse(); + context.assertSentUpdateVoterResponse( + Errors.NONE, + OptionalInt.of(local.id()), + epoch + ); + + // Upgrade not allowed since one of the voters doesn't support the new version + assertEquals(KRaftVersion.KRAFT_VERSION_0, context.client.kraftVersion()); + assertThrows( + InvalidUpdateVersionException.class, + () -> context.client.upgradeKRaftVersion(epoch, KRaftVersion.KRAFT_VERSION_1, false) + ); + } + @Test void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { ReplicaKey local = replicaKey(randomReplicaId(), true); @@ -2145,8 +2637,8 @@ void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { .withElectedLeader(epoch, voter1.id()) .build(); - // waiting for 3 times the fetch timeout sends an update voter - for (int i = 0; i < 3; i++) { + // waiting for FETCH request until the UpdateRaftVoter request is set + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); @@ -2179,6 +2671,86 @@ void testFollowerSendsUpdateVoterWhenDifferent() throws Exception { assertNotEquals(OptionalLong.of(0L), context.messageQueue.lastPollTimeoutMs()); } + @Test + void testFollowerSendsUpdateVoterIfPendingFetchDuringTimeout() throws Exception { + ReplicaKey local = replicaKey(randomReplicaId(), true); + ReplicaKey voter1 = replicaKey(local.id() + 1, true); + ReplicaKey voter2 = replicaKey(local.id() + 2, true); + + VoterSet voters = VoterSetTest.voterSet(Stream.of(local, voter1, voter2)); + int epoch = 4; + + HashMap listenersMap = new HashMap<>(2); + listenersMap.put( + VoterSetTest.DEFAULT_LISTENER_NAME, + InetSocketAddress.createUnresolved("localhost", 9990 + local.id()) + ); + listenersMap.put( + ListenerName.normalised("ANOTHER_LISTENER"), + InetSocketAddress.createUnresolved("localhost", 8990 + local.id()) + ); + Endpoints localListeners = Endpoints.fromInetSocketAddresses(listenersMap); + + RaftClientTestContext context = new RaftClientTestContext.Builder(local.id(), local.directoryId().get()) + .withKip853Rpc(true) + .withBootstrapSnapshot(Optional.of(voters)) + .withElectedLeader(epoch, voter1.id()) + .withLocalListeners(localListeners) + .build(); + + // waiting up to the last FETCH request before the UpdateRaftVoter request is set + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + epoch, + voter1.id(), + MemoryRecords.EMPTY, + 0L, + Errors.NONE + ) + ); + // poll kraft to handle the fetch response + context.client.poll(); + } + + // expect one last FETCH request + context.pollUntilRequest(); + RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0); + + // don't send a response but increase the time + context.time.sleep(context.requestTimeoutMs() - 1); + context.client.poll(); + assertFalse(context.channel.hasSentRequests()); + + // expect an update voter request after the FETCH rpc completes + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + epoch, + voter1.id(), + MemoryRecords.EMPTY, + 0L, + Errors.NONE + ) + ); + context.pollUntilRequest(); + context.assertSentUpdateVoterRequest( + local, + epoch, + Feature.KRAFT_VERSION.supportedVersionRange(), + localListeners + ); + } + @Test void testUpdateVoterResponseCausesEpochChange() throws Exception { ReplicaKey local = replicaKey(randomReplicaId(), true); @@ -2206,8 +2778,8 @@ void testUpdateVoterResponseCausesEpochChange() throws Exception { .withLocalListeners(localListeners) .build(); - // waiting for 3 times the fetch timeout sends an update voter - for (int i = 0; i < 3; i++) { + // waiting for FETCH request until the UpdateRaftVoter request is set + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); diff --git a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java index f534db5274357..686daa37637da 100644 --- a/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/LeaderStateTest.java @@ -17,10 +17,13 @@ package org.apache.kafka.raft; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.errors.InvalidUpdateVersionException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.raft.internals.BatchAccumulator; +import org.apache.kafka.raft.internals.KRaftVersionUpgrade; import org.apache.kafka.raft.internals.KafkaRaftMetrics; import org.apache.kafka.server.common.KRaftVersion; @@ -46,23 +49,36 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class LeaderStateTest { - private final ReplicaKey localReplicaKey = ReplicaKey.of(0, Uuid.randomUuid()); + private final VoterSet.VoterNode localVoterNode = VoterSetTest.voterNode(ReplicaKey.of(0, Uuid.randomUuid())); private final int epoch = 5; private final LogContext logContext = new LogContext(); - private final BatchAccumulator accumulator = Mockito.mock(BatchAccumulator.class); private final MockTime time = new MockTime(); private final int fetchTimeoutMs = 2000; private final int checkQuorumTimeoutMs = (int) (fetchTimeoutMs * CHECK_QUORUM_TIMEOUT_FACTOR); private final int beginQuorumEpochTimeoutMs = fetchTimeoutMs / 2; - private final KRaftVersion kraftVersion = KRaftVersion.KRAFT_VERSION_1; private LeaderState newLeaderState( VoterSet voters, - long epochStartOffset + long epochStartOffset, + KRaftVersion kraftVersion + ) { + return newLeaderState( + voters, + epochStartOffset, + kraftVersion, + Mockito.mock(BatchAccumulator.class) + ); + } + + private LeaderState newLeaderState( + VoterSet voters, + long epochStartOffset, + KRaftVersion kraftVersion, + BatchAccumulator accumulator ) { return new LeaderState<>( time, - localReplicaKey, + localVoterNode, epoch, epochStartOffset, voters, @@ -70,7 +86,6 @@ private LeaderState newLeaderState( kraftVersion, voters.voterIds(), accumulator, - voters.listeners(localReplicaKey.id()), fetchTimeoutMs, logContext, new KafkaRaftMetrics(new Metrics(), "raft") @@ -80,11 +95,13 @@ private LeaderState newLeaderState( private VoterSet localWithRemoteVoterSet(IntStream remoteIds, boolean withDirectoryId) { Map voters = VoterSetTest.voterMap(remoteIds, withDirectoryId); if (withDirectoryId) { - voters.put(localReplicaKey.id(), VoterSetTest.voterNode(localReplicaKey)); + voters.put(localVoterNode.voterKey().id(), localVoterNode); } else { voters.put( - localReplicaKey.id(), - VoterSetTest.voterNode(ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID)) + localVoterNode.voterKey().id(), + VoterSetTest.voterNode( + ReplicaKey.of(localVoterNode.voterKey().id(), ReplicaKey.NO_DIRECTORY_ID) + ) ); } @@ -93,8 +110,8 @@ private VoterSet localWithRemoteVoterSet(IntStream remoteIds, boolean withDirect private VoterSet localWithRemoteVoterSet(Stream remoteReplicaKeys, boolean withDirectoryId) { ReplicaKey actualLocalVoter = withDirectoryId ? - localReplicaKey : - ReplicaKey.of(localReplicaKey.id(), ReplicaKey.NO_DIRECTORY_ID); + localVoterNode.voterKey() : + ReplicaKey.of(localVoterNode.voterKey().id(), ReplicaKey.NO_DIRECTORY_ID); return VoterSetTest.voterSet( Stream.concat(Stream.of(actualLocalVoter), remoteReplicaKeys) @@ -103,20 +120,23 @@ private VoterSet localWithRemoteVoterSet(Stream remoteReplicaKeys, b @Test public void testRequireNonNullAccumulator() { - VoterSet voterSet = VoterSetTest.voterSet(Stream.of(localReplicaKey)); + VoterSet voterSet = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())); assertThrows( NullPointerException.class, () -> new LeaderState<>( new MockTime(), - localReplicaKey, + voterSet.voterNodes() + .stream() + .filter(node -> node.voterKey().equals(localVoterNode.voterKey())) + .findFirst() + .get(), epoch, 0, voterSet, OptionalLong.of(0), - kraftVersion, + KRaftVersion.KRAFT_VERSION_1, Set.of(), null, - Endpoints.empty(), fetchTimeoutMs, logContext, new KafkaRaftMetrics(new Metrics(), "raft") @@ -131,7 +151,8 @@ public void testFollowerAcknowledgement(boolean withDirectoryId) { ReplicaKey node2 = replicaKey(2, withDirectoryId); LeaderState state = newLeaderState( localWithRemoteVoterSet(Stream.of(node1, node2), withDirectoryId), - 0L + 0L, + KRaftVersion.KRAFT_VERSION_1 ); assertEquals(Set.of(node1, node2), state.nonAcknowledgingVoters()); state.addAcknowledgementFrom(node1.id()); @@ -144,16 +165,21 @@ public void testFollowerAcknowledgement(boolean withDirectoryId) { public void testNonFollowerAcknowledgement() { int nonVoterId = 1; LeaderState state = newLeaderState( - VoterSetTest.voterSet(Stream.of(localReplicaKey)), - 0L + VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())), + 0L, + KRaftVersion.KRAFT_VERSION_1 ); assertThrows(IllegalArgumentException.class, () -> state.addAcknowledgementFrom(nonVoterId)); } @Test public void testUpdateHighWatermarkQuorumSizeOne() { - VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); - LeaderState state = newLeaderState(voters, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())); + LeaderState state = newLeaderState( + voters, + 15L, + KRaftVersion.KRAFT_VERSION_1 + ); assertEquals(Optional.empty(), state.highWatermark()); assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), voters)); @@ -167,8 +193,12 @@ public void testUpdateHighWatermarkQuorumSizeOne() { @Test public void testNonMonotonicLocalEndOffsetUpdate() { - VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); - LeaderState state = newLeaderState(voters, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())); + LeaderState state = newLeaderState( + voters, + 15L, + KRaftVersion.KRAFT_VERSION_1 + ); assertEquals(Optional.empty(), state.highWatermark()); assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), voters)); @@ -181,8 +211,12 @@ public void testNonMonotonicLocalEndOffsetUpdate() { @Test public void testIdempotentEndOffsetUpdate() { - VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); - LeaderState state = newLeaderState(voters, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())); + LeaderState state = newLeaderState( + voters, + 15L, + KRaftVersion.KRAFT_VERSION_1 + ); assertEquals(Optional.empty(), state.highWatermark()); assertTrue(state.updateLocalState(new LogOffsetMetadata(16L), voters)); assertFalse(state.updateLocalState(new LogOffsetMetadata(16L), voters)); @@ -191,8 +225,12 @@ public void testIdempotentEndOffsetUpdate() { @Test public void testUpdateHighWatermarkMetadata() { - VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); - LeaderState state = newLeaderState(voters, 15L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())); + LeaderState state = newLeaderState( + voters, + 15L, + KRaftVersion.KRAFT_VERSION_1 + ); assertEquals(Optional.empty(), state.highWatermark()); LogOffsetMetadata initialHw = new LogOffsetMetadata(16L, Optional.of(new MockOffsetMetadata("bar"))); @@ -210,7 +248,11 @@ public void testUpdateHighWatermarkQuorumSizeTwo(boolean withDirectoryId) { ReplicaKey otherNodeKey = replicaKey(1, withDirectoryId); VoterSet voters = localWithRemoteVoterSet(Stream.of(otherNodeKey), withDirectoryId); - LeaderState state = newLeaderState(voters, 10L); + LeaderState state = newLeaderState( + voters, + 10L, + KRaftVersion.KRAFT_VERSION_1 + ); assertFalse(state.updateLocalState(new LogOffsetMetadata(13L), voters)); assertEquals(Set.of(otherNodeKey), state.nonAcknowledgingVoters()); @@ -231,7 +273,11 @@ public void testUpdateHighWatermarkQuorumSizeThree(boolean withDirectoryId) { ReplicaKey nodeKey2 = replicaKey(2, withDirectoryId); VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId); - LeaderState state = newLeaderState(voters, 10L); + LeaderState state = newLeaderState( + voters, + 10L, + KRaftVersion.KRAFT_VERSION_1 + ); assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), voters)); assertEquals(Set.of(nodeKey1, nodeKey2), state.nonAcknowledgingVoters()); @@ -258,7 +304,11 @@ public void testHighWatermarkDoesIncreaseFromNewVoter() { ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1), true); - LeaderState state = newLeaderState(originalVoters, 5L); + LeaderState state = newLeaderState( + originalVoters, + 5L, + KRaftVersion.KRAFT_VERSION_1 + ); assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(10L))); @@ -288,7 +338,11 @@ public void testHighWatermarkDoesNotDecreaseFromNewVoter() { // start with three voters with HW at 15L VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); - LeaderState state = newLeaderState(originalVoters, 5L); + LeaderState state = newLeaderState( + originalVoters, + 5L, + KRaftVersion.KRAFT_VERSION_1 + ); assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L))); @@ -322,7 +376,11 @@ public void testUpdateHighWatermarkRemovingFollowerFromVoterStates() { ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); - LeaderState state = newLeaderState(originalVoters, 10L); + LeaderState state = newLeaderState( + originalVoters, + 10L, + KRaftVersion.KRAFT_VERSION_1 + ); assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L))); @@ -355,7 +413,11 @@ public void testUpdateHighWatermarkQuorumRemovingLeaderFromVoterStates() { ReplicaKey nodeKey2 = ReplicaKey.of(2, Uuid.randomUuid()); VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); - LeaderState state = newLeaderState(originalVoters, 10L); + LeaderState state = newLeaderState( + originalVoters, + 10L, + KRaftVersion.KRAFT_VERSION_1 + ); assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), originalVoters)); assertTrue(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(15L))); @@ -363,7 +425,7 @@ public void testUpdateHighWatermarkQuorumRemovingLeaderFromVoterStates() { assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); // removing leader should not decrement HW to 10L - VoterSet votersWithoutLeader = originalVoters.removeVoter(localReplicaKey).get(); + VoterSet votersWithoutLeader = originalVoters.removeVoter(localVoterNode.voterKey()).get(); assertFalse(state.updateLocalState(new LogOffsetMetadata(17L), votersWithoutLeader)); assertEquals(Optional.of(new LogOffsetMetadata(15L)), state.highWatermark()); @@ -389,7 +451,11 @@ public void testNonMonotonicHighWatermarkUpdate(boolean withDirectoryId) { ReplicaKey nodeKey1 = replicaKey(1, withDirectoryId); VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1), withDirectoryId); - LeaderState state = newLeaderState(voters, 0L); + LeaderState state = newLeaderState( + voters, + 0L, + KRaftVersion.KRAFT_VERSION_1 + ); state.updateLocalState(new LogOffsetMetadata(10L), voters); state.updateReplicaState(nodeKey1, time.milliseconds(), new LogOffsetMetadata(10L)); @@ -410,7 +476,11 @@ public void testGetNonLeaderFollowersByFetchOffsetDescending(boolean withDirecto long leaderEndOffset = 15L; VoterSet voters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), withDirectoryId); - LeaderState state = newLeaderState(voters, leaderStartOffset); + LeaderState state = newLeaderState( + voters, + leaderStartOffset, + KRaftVersion.KRAFT_VERSION_1 + ); state.updateLocalState(new LogOffsetMetadata(leaderEndOffset), voters); assertEquals(Optional.empty(), state.highWatermark()); @@ -437,7 +507,11 @@ public void testCheckQuorum(boolean withDirectoryId) { Stream.of(nodeKey1, nodeKey2, nodeKey3, nodeKey4), withDirectoryId ); - LeaderState state = newLeaderState(voters, 0L); + LeaderState state = newLeaderState( + voters, + 0L, + KRaftVersion.KRAFT_VERSION_1 + ); assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); int resignLeadershipTimeout = checkQuorumTimeoutMs; @@ -473,7 +547,11 @@ public void testCheckQuorumAfterVoterSetChanges() { ReplicaKey nodeKey3 = ReplicaKey.of(3, Uuid.randomUuid()); VoterSet originalVoters = localWithRemoteVoterSet(Stream.of(nodeKey1, nodeKey2), true); - LeaderState state = newLeaderState(originalVoters, 0L); + LeaderState state = newLeaderState( + originalVoters, + 0L, + KRaftVersion.KRAFT_VERSION_1 + ); assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); // checkQuorum timeout not exceeded, should not expire the timer @@ -498,7 +576,7 @@ public void testCheckQuorumAfterVoterSetChanges() { assertEquals(checkQuorumTimeoutMs, state.timeUntilCheckQuorumExpires(time.milliseconds())); // removing leader from the voter set - VoterSet votersWithoutLeader = votersWithNode3.removeVoter(localReplicaKey).get(); + VoterSet votersWithoutLeader = votersWithNode3.removeVoter(localVoterNode.voterKey()).get(); state.updateLocalState(new LogOffsetMetadata(1L), votersWithoutLeader); time.sleep(checkQuorumTimeoutMs / 2); @@ -517,8 +595,9 @@ public void testCheckQuorumWithOneVoter() { // Only 1 voter quorum LeaderState state = newLeaderState( - VoterSetTest.voterSet(Stream.of(localReplicaKey)), - 0L + VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())), + 0L, + KRaftVersion.KRAFT_VERSION_1 ); assertEquals(Long.MAX_VALUE, state.timeUntilCheckQuorumExpires(time.milliseconds())); @@ -536,11 +615,15 @@ public void testCheckQuorumWithOneVoter() { @Test public void testLeaderEndpoints() { - VoterSet voters = VoterSetTest.voterSet(Stream.of(localReplicaKey)); - LeaderState state = newLeaderState(voters, 0L); + VoterSet voters = VoterSetTest.voterSet(Stream.of(localVoterNode.voterKey())); + LeaderState state = newLeaderState( + voters, + 0L, + KRaftVersion.KRAFT_VERSION_1 + ); assertNotEquals(Endpoints.empty(), state.leaderEndpoints()); - assertEquals(voters.listeners(localReplicaKey.id()), state.leaderEndpoints()); + assertEquals(voters.listeners(localVoterNode.voterKey().id()), state.leaderEndpoints()); } @Test @@ -555,7 +638,11 @@ public void testUpdateVotersFromNoDirectoryIdToDirectoryId() { false ); - LeaderState state = newLeaderState(votersBeforeUpgrade, 0L); + LeaderState state = newLeaderState( + votersBeforeUpgrade, + 0L, + KRaftVersion.KRAFT_VERSION_1 + ); assertFalse(state.updateLocalState(new LogOffsetMetadata(10L), votersBeforeUpgrade)); assertTrue(state.updateReplicaState(nodeKey1, 0L, new LogOffsetMetadata(10L))); @@ -571,18 +658,29 @@ public void testUpdateVotersFromNoDirectoryIdToDirectoryId() { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testGrantVote(boolean isLogUpToDate) { + int[] remoteIds = {1, 2, 3}; LeaderState state = newLeaderState( - VoterSetTest.voterSet(VoterSetTest.voterMap(IntStream.of(1, 2, 3), false)), - 1 + VoterSetTest.voterSet( + VoterSetTest.voterMap( + IntStream.concat(IntStream.of(localVoterNode.voterKey().id()), IntStream.of(remoteIds)), + false + ) + ), + 1, + KRaftVersion.KRAFT_VERSION_1 ); - assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, true)); - - assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); - assertFalse(state.canGrantVote(ReplicaKey.of(3, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate, false)); + IntStream.of(remoteIds).forEach(id -> { + List.of(true, false).forEach(isPrevote -> { + assertFalse( + state.canGrantVote( + ReplicaKey.of(id, ReplicaKey.NO_DIRECTORY_ID), + isLogUpToDate, + isPrevote + ) + ); + }); + }); } @ParameterizedTest @@ -594,7 +692,8 @@ public void testBeginQuorumEpochTimer(boolean withDirectoryId) { VoterSet voters = localWithRemoteVoterSet(IntStream.of(follower1), withDirectoryId); LeaderState state = newLeaderState( voters, - epochStartOffset + epochStartOffset, + KRaftVersion.KRAFT_VERSION_1 ); assertEquals(0, state.timeUntilBeginQuorumEpochTimerExpires(time.milliseconds())); @@ -609,6 +708,165 @@ public void testBeginQuorumEpochTimer(boolean withDirectoryId) { assertEquals(0, state.timeUntilBeginQuorumEpochTimerExpires(time.milliseconds())); } + @Test + public void testVolatileVoters() { + int follower1 = 1; + long epochStartOffset = 10L; + + VoterSet voters = localWithRemoteVoterSet(IntStream.of(follower1), false); + LeaderState state = newLeaderState( + voters, + epochStartOffset, + KRaftVersion.KRAFT_VERSION_0 + ); + + var votersWithLeaderUpdated = state.volatileVoters().get(); + assertEquals( + voters.updateVoterIgnoringDirectoryId(localVoterNode).get(), + votersWithLeaderUpdated.voters() + ); + + var updatedVoters = new KRaftVersionUpgrade.Voters( + votersWithLeaderUpdated + .voters() + .updateVoterIgnoringDirectoryId(VoterSetTest.voterNode(follower1, true)) + .get() + ); + + // Upate in-memory voter and check state + assertTrue( + state.compareAndSetVolatileVoters(votersWithLeaderUpdated, updatedVoters) + ); + assertEquals(updatedVoters, state.volatileVoters().get()); + + // Unable to perform atomic update + assertFalse( + state.compareAndSetVolatileVoters(votersWithLeaderUpdated, updatedVoters) + ); + } + + @Test + public void testInvalidMaybeAppendUpgradedKRaftVersion() { + int follower1 = 1; + int follower2 = 2; + long epochStartOffset = 10L; + + VoterSet persistedVoters = localWithRemoteVoterSet(IntStream.of(follower1, follower2), false); + LeaderState state = newLeaderState( + persistedVoters, + epochStartOffset, + KRaftVersion.KRAFT_VERSION_0 + ); + + // none of the remove voters support kraft version 1 since the starting version is 0. + assertThrows( + InvalidUpdateVersionException.class, + () -> + state.maybeAppendUpgradedKRaftVersion( + epoch, + KRaftVersion.KRAFT_VERSION_1, + KRaftVersion.KRAFT_VERSION_0, + persistedVoters, + false, + time.milliseconds() + ) + ); + + // epoch is less than the leader's epoch + assertThrows( + NotLeaderException.class, + () -> + state.maybeAppendUpgradedKRaftVersion( + epoch - 1, + KRaftVersion.KRAFT_VERSION_1, + KRaftVersion.KRAFT_VERSION_0, + persistedVoters, + false, + time.milliseconds() + ) + ); + + // epoch is greater than the leader's epoch + assertThrows( + IllegalArgumentException.class, + () -> + state.maybeAppendUpgradedKRaftVersion( + epoch + 1, + KRaftVersion.KRAFT_VERSION_1, + KRaftVersion.KRAFT_VERSION_0, + persistedVoters, + false, + time.milliseconds() + ) + ); + + // noop since the upgrade version is already 1 + assertFalse( + state.maybeAppendUpgradedKRaftVersion( + epoch, + KRaftVersion.KRAFT_VERSION_1, + KRaftVersion.KRAFT_VERSION_1, + persistedVoters, + false, + time.milliseconds() + ) + ); + } + + @Test + public void testMaybeAppendUpgradedKRaftVersion() { + int follower1 = 1; + int follower2 = 2; + long epochStartOffset = 10L; + BatchAccumulator accumulator = Mockito.mock(BatchAccumulator.class); + + VoterSet persistedVoters = localWithRemoteVoterSet(IntStream.of(follower1, follower2), false); + LeaderState state = newLeaderState( + persistedVoters, + epochStartOffset, + KRaftVersion.KRAFT_VERSION_0, + accumulator + ); + + var updatedVoters = state.volatileVoters().get().voters(); + updatedVoters = updatedVoters + .updateVoterIgnoringDirectoryId(VoterSetTest.voterNode(follower1, true)) + .get(); + updatedVoters = updatedVoters + .updateVoterIgnoringDirectoryId(VoterSetTest.voterNode(follower2, true)) + .get(); + state.compareAndSetVolatileVoters( + state.volatileVoters().get(), + new KRaftVersionUpgrade.Voters(updatedVoters) + ); + + assertTrue( + state.maybeAppendUpgradedKRaftVersion( + epoch, + KRaftVersion.KRAFT_VERSION_1, + KRaftVersion.KRAFT_VERSION_0, + persistedVoters, + false, + time.milliseconds() + ) + ); + + // Expect control records after upgrading the kraft version. + Mockito.verify(accumulator).appendControlMessages(Mockito.any()); + + // maybe upgrade kraft version should be a noop after an upgrade + assertFalse( + state.maybeAppendUpgradedKRaftVersion( + epoch, + KRaftVersion.KRAFT_VERSION_1, + KRaftVersion.KRAFT_VERSION_0, + persistedVoters, + false, + time.milliseconds() + ) + ); + } + private static class MockOffsetMetadata implements OffsetMetadata { private final String value; diff --git a/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java index 665936d9897d1..985c9b4929ae1 100644 --- a/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/VoterSetTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.feature.SupportedVersionRange; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.server.common.Feature; +import org.apache.kafka.server.common.KRaftVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -144,6 +145,34 @@ void testUpdateVoter() { ); } + @Test + void testUpdateVoterIgnoringDirectoryId() { + Map aVoterMap = voterMap(IntStream.of(1, 2, 3), false); + VoterSet voterSet = VoterSet.fromMap(new HashMap<>(aVoterMap)); + + // Cannot override node id not contianed in the voter set + assertEquals(Optional.empty(), voterSet.updateVoterIgnoringDirectoryId(voterNode(4, true))); + + // Test that it can override voter set with different directory ids + VoterSet.VoterNode newVoter3 = voterNode(3, true); + assertNotEquals(aVoterMap.get(3).voterKey(), newVoter3.voterKey()); + aVoterMap.put(3, newVoter3); + + assertEquals( + Optional.of(VoterSet.fromMap(new HashMap<>(aVoterMap))), + voterSet.updateVoterIgnoringDirectoryId(newVoter3) + ); + + // Test that it can continue to override voter set with different directory ids + newVoter3 = voterNode(3, true); + assertNotEquals(aVoterMap.get(3).voterKey(), newVoter3.voterKey()); + aVoterMap.put(3, newVoter3); + + assertEquals( + Optional.of(VoterSet.fromMap(new HashMap<>(aVoterMap))), + voterSet.updateVoterIgnoringDirectoryId(newVoter3) + ); + } @Test void testCannotRemoveToEmptyVoterSet() { @@ -314,6 +343,17 @@ void testNonOverlappingMajority() { assertMajorities(false, startingVoterSet, replacedVoterSet); } + @Test + void testSupportsVersion() { + VoterSet voterSet = voterSet(voterMap(IntStream.of(1, 2, 3), false)); + assertTrue(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_0)); + assertFalse(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_1)); + + voterSet = voterSet(voterMap(IntStream.of(1, 2, 3), true)); + assertTrue(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_0)); + assertTrue(voterSet.supportsVersion(KRaftVersion.KRAFT_VERSION_1)); + } + private void assertMajorities(boolean overlap, VoterSet a, VoterSet b) { assertEquals( overlap, @@ -373,11 +413,11 @@ public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey) { } public static VoterSet.VoterNode voterNode(ReplicaKey replicaKey, Endpoints endpoints) { - return new VoterSet.VoterNode( - replicaKey, - endpoints, - Feature.KRAFT_VERSION.supportedVersionRange() - ); + var supportedVersionRange = replicaKey.directoryId().isEmpty() ? + new SupportedVersionRange((short) 0) : + Feature.KRAFT_VERSION.supportedVersionRange(); + + return new VoterSet.VoterNode(replicaKey, endpoints, supportedVersionRange); } public static VoterSet voterSet(Map voters) { diff --git a/release/git.py b/release/git.py index 61de830ecea29..9cb106df2fdcb 100644 --- a/release/git.py +++ b/release/git.py @@ -128,8 +128,12 @@ def create_tag(tag, **kwargs): cmd(f"Creating git tag {tag}", ["git", "tag", "-a", tag, "-m", tag], **kwargs) -def push_tag(tag, remote=push_remote_name, **kwargs): +def push_ref(ref, remote=push_remote_name, **kwargs): __defaults(kwargs) - cmd("Pushing tag {tag} to {remote}", f"git push {remote} {tag}") + cmd(f"Pushing ref {ref} to {remote}", f"git push {remote} {ref}") +def merge_ref(ref, **kwargs): + __defaults(kwargs) + cmd(f"Merging ref {ref}", f"git merge {ref}") + diff --git a/release/release.py b/release/release.py index b3a428f651636..d0cba6f178216 100644 --- a/release/release.py +++ b/release/release.py @@ -297,6 +297,7 @@ def delete_gitrefs(): git.commit(f"Bump version to {release_version}") git.create_tag(rc_tag) git.switch_branch(starting_branch) +git.merge_ref(rc_tag) # Note that we don't use tempfile here because mkdtemp causes problems with being able to determine the absolute path to a file. # Instead we rely on a fixed path @@ -367,7 +368,8 @@ def delete_gitrefs(): print(templates.deploy_instructions()) confirm_or_fail("Have you successfully deployed the artifacts?") confirm_or_fail(f"Ok to push RC tag {rc_tag}?") -git.push_tag(rc_tag) +git.push_ref(rc_tag) +git.push_ref(starting_branch) # Move back to starting branch and clean out the temporary release branch (e.g. 1.0.0) we used to generate everything git.reset_hard_head() diff --git a/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java b/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java index bb68bc6911a70..463cc2a015ce3 100644 --- a/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java +++ b/server-common/src/main/java/org/apache/kafka/server/common/KRaftVersion.java @@ -52,7 +52,7 @@ public static KRaftVersion fromFeatureLevel(short version) { case 1: return KRAFT_VERSION_1; default: - throw new RuntimeException("Unknown KRaft feature level: " + (int) version); + throw new IllegalArgumentException("Unknown KRaft feature level: " + (int) version); } } @@ -80,29 +80,40 @@ public Map dependencies() { } } + public boolean isAtLeast(KRaftVersion otherVersion) { + return this.compareTo(otherVersion) >= 0; + } + + public boolean isMoreThan(KRaftVersion otherVersion) { + return this.compareTo(otherVersion) > 0; + } + public short quorumStateVersion() { switch (this) { case KRAFT_VERSION_0: return (short) 0; case KRAFT_VERSION_1: return (short) 1; + default: + throw new IllegalStateException("Unsupported KRaft feature level: " + this); } - throw new IllegalStateException("Unsupported KRaft feature level: " + this); } public short kraftVersionRecordVersion() { switch (this) { case KRAFT_VERSION_1: return (short) 0; + default: + throw new IllegalStateException("Unsupported KRaft feature level: " + this); } - throw new IllegalStateException("Unsupported KRaft feature level: " + this); } public short votersRecordVersion() { switch (this) { case KRAFT_VERSION_1: return (short) 0; + default: + throw new IllegalStateException("Unsupported KRaft feature level: " + this); } - throw new IllegalStateException("Unsupported KRaft feature level: " + this); } } diff --git a/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java b/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java index 1b2a03b0b8bd9..9e451b4b1cd05 100644 --- a/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java +++ b/server-common/src/main/java/org/apache/kafka/server/share/persister/PersisterStateManager.java @@ -1475,8 +1475,7 @@ private static AbstractRequest.Builder coalesceReadSu .map(entry -> new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData() .setTopicId(entry.getKey()) .setPartitions(entry.getValue())) - .collect(Collectors.toList())), - true + .collect(Collectors.toList())) ); } diff --git a/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java b/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java index f850d3703224f..fadf386a0354f 100644 --- a/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java +++ b/server-common/src/test/java/org/apache/kafka/server/common/KRaftVersionTest.java @@ -22,7 +22,9 @@ import org.junit.jupiter.api.Test; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; public final class KRaftVersionTest { @Test @@ -110,4 +112,20 @@ public void tesVotersRecordVersion() { } } } + + @Test + public void testIsAtLeast() { + assertTrue(KRaftVersion.KRAFT_VERSION_0.isAtLeast(KRaftVersion.KRAFT_VERSION_0)); + assertFalse(KRaftVersion.KRAFT_VERSION_0.isAtLeast(KRaftVersion.KRAFT_VERSION_1)); + assertTrue(KRaftVersion.KRAFT_VERSION_1.isAtLeast(KRaftVersion.KRAFT_VERSION_0)); + assertTrue(KRaftVersion.KRAFT_VERSION_1.isAtLeast(KRaftVersion.KRAFT_VERSION_1)); + } + + @Test + public void testIsMoreThan() { + assertFalse(KRaftVersion.KRAFT_VERSION_0.isMoreThan(KRaftVersion.KRAFT_VERSION_0)); + assertFalse(KRaftVersion.KRAFT_VERSION_0.isMoreThan(KRaftVersion.KRAFT_VERSION_1)); + assertTrue(KRaftVersion.KRAFT_VERSION_1.isMoreThan(KRaftVersion.KRAFT_VERSION_0)); + assertFalse(KRaftVersion.KRAFT_VERSION_1.isMoreThan(KRaftVersion.KRAFT_VERSION_1)); + } } diff --git a/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java b/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java index 8406f9efa91c3..fd5e917569aa7 100644 --- a/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java +++ b/server/src/main/java/org/apache/kafka/server/share/fetch/ShareFetch.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.server.storage.log.FetchParams; import org.apache.kafka.storage.log.metrics.BrokerTopicStats; @@ -217,7 +218,8 @@ private synchronized void addErroneousToResponse(Map { brokerTopicStats.allTopicsStats().failedShareFetchRequestRate().mark(); diff --git a/server/src/main/java/org/apache/kafka/server/share/session/LastUsedKey.java b/server/src/main/java/org/apache/kafka/server/share/session/LastUsedKey.java deleted file mode 100644 index e6084f8e9b5f6..0000000000000 --- a/server/src/main/java/org/apache/kafka/server/share/session/LastUsedKey.java +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.server.share.session; - -import java.util.Objects; - -public class LastUsedKey implements Comparable { - private final ShareSessionKey key; - private final long lastUsedMs; - - public LastUsedKey(ShareSessionKey key, long lastUsedMs) { - this.key = key; - this.lastUsedMs = lastUsedMs; - } - - public ShareSessionKey key() { - return key; - } - - public long lastUsedMs() { - return lastUsedMs; - } - - @Override - public int hashCode() { - return Objects.hash(key, lastUsedMs); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - LastUsedKey other = (LastUsedKey) obj; - return lastUsedMs == other.lastUsedMs && Objects.equals(key, other.key); - } - - @Override - public int compareTo(LastUsedKey other) { - int res = Long.compare(lastUsedMs, other.lastUsedMs); - if (res != 0) - return res; - return Integer.compare(key.hashCode(), other.key.hashCode()); - } -} diff --git a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java index 362f32e61975e..5cb800c552453 100644 --- a/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java +++ b/server/src/main/java/org/apache/kafka/server/share/session/ShareSession.java @@ -38,9 +38,7 @@ public enum ModifiedTopicIdPartitionType { private final ShareSessionKey key; private final ImplicitLinkedHashCollection partitionMap; - private final long creationMs; - private long lastUsedMs; // visible for testing public int epoch; // This is used by the ShareSessionCache to store the last known size of this session. @@ -54,17 +52,11 @@ public enum ModifiedTopicIdPartitionType { * * @param key The share session key to identify the share session uniquely. * @param partitionMap The CachedPartitionMap. - * @param creationMs The time in milliseconds when this share session was created. - * @param lastUsedMs The last used time in milliseconds. This should only be updated by - * ShareSessionCache#touch. * @param epoch The share session sequence number. */ - public ShareSession(ShareSessionKey key, ImplicitLinkedHashCollection partitionMap, - long creationMs, long lastUsedMs, int epoch) { + public ShareSession(ShareSessionKey key, ImplicitLinkedHashCollection partitionMap, int epoch) { this.key = key; this.partitionMap = partitionMap; - this.creationMs = creationMs; - this.lastUsedMs = lastUsedMs; this.epoch = epoch; } @@ -76,18 +68,6 @@ public synchronized int cachedSize() { return cachedSize; } - public synchronized void cachedSize(int size) { - cachedSize = size; - } - - public synchronized long lastUsedMs() { - return lastUsedMs; - } - - public synchronized void lastUsedMs(long ts) { - lastUsedMs = ts; - } - public synchronized ImplicitLinkedHashCollection partitionMap() { return partitionMap; } @@ -105,10 +85,6 @@ public synchronized Boolean isEmpty() { return partitionMap.isEmpty(); } - public synchronized LastUsedKey lastUsedKey() { - return new LastUsedKey(key, lastUsedMs); - } - // Update the cached partition data based on the request. public synchronized Map> update( List shareFetchData, @@ -138,6 +114,17 @@ public synchronized Map> up return result; } + /** + * Updates the cached size of the session to represent the current partitionMap size. + * @return The difference between the current cached size and the previously stored cached size. This is required to + * update the total number of share partitions stored in the share session cache. + */ + public synchronized int updateCachedSize() { + var previousSize = cachedSize; + cachedSize = partitionMap.size(); + return previousSize != -1 ? cachedSize - previousSize : cachedSize; + } + public static String partitionsToLogString(Collection partitions, Boolean traceEnabled) { if (traceEnabled) { return String.format("( %s )", String.join(", ", partitions.toString())); @@ -149,8 +136,6 @@ public String toString() { return "ShareSession(" + "key=" + key + ", partitionMap=" + partitionMap + - ", creationMs=" + creationMs + - ", lastUsedMs=" + lastUsedMs + ", epoch=" + epoch + ", cachedSize=" + cachedSize + ")"; diff --git a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java index dc870fc9c252c..0b06ea535be7e 100644 --- a/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java +++ b/server/src/main/java/org/apache/kafka/server/share/session/ShareSessionCache.java @@ -27,7 +27,6 @@ import java.util.HashMap; import java.util.Map; -import java.util.TreeMap; import java.util.concurrent.TimeUnit; /** @@ -53,19 +52,14 @@ public class ShareSessionCache { private final Meter evictionsMeter; private final int maxEntries; - private final long evictionMs; private long numPartitions = 0; // A map of session key to ShareSession. private final Map sessions = new HashMap<>(); - // Maps last used times to sessions. - private final TreeMap lastUsed = new TreeMap<>(); - @SuppressWarnings("this-escape") - public ShareSessionCache(int maxEntries, long evictionMs) { + public ShareSessionCache(int maxEntries) { this.maxEntries = maxEntries; - this.evictionMs = evictionMs; // Register metrics for ShareSessionCache. KafkaMetricsGroup metricsGroup = new KafkaMetricsGroup("kafka.server", "ShareSessionCache"); metricsGroup.newGauge(SHARE_SESSIONS_COUNT, this::size); @@ -108,9 +102,6 @@ public synchronized ShareSession remove(ShareSessionKey key) { * @return The removed session, or None if there was no such session. */ public synchronized ShareSession remove(ShareSession session) { - synchronized (session) { - lastUsed.remove(session.lastUsedKey()); - } ShareSession removeResult = sessions.remove(session.key()); if (removeResult != null) { numPartitions = numPartitions - session.cachedSize(); @@ -119,64 +110,27 @@ public synchronized ShareSession remove(ShareSession session) { } /** - * Update a session's position in the lastUsed tree. + * Update the size of the cache by updating the total number of share partitions. * * @param session The session. - * @param now The current time in milliseconds. */ - public synchronized void touch(ShareSession session, long now) { - synchronized (session) { - // Update the lastUsed map. - lastUsed.remove(session.lastUsedKey()); - session.lastUsedMs(now); - lastUsed.put(session.lastUsedKey(), session); - - int oldSize = session.cachedSize(); - if (oldSize != -1) { - numPartitions = numPartitions - oldSize; - } - session.cachedSize(session.size()); - numPartitions = numPartitions + session.cachedSize(); - } - } - - /** - * Try to evict an entry from the session cache. - *

- * A proposed new element A may evict an existing element B if: - * B is considered "stale" because it has been inactive for a long time. - * - * @param now The current time in milliseconds. - * @return True if an entry was evicted; false otherwise. - */ - public synchronized boolean tryEvict(long now) { - // Try to evict an entry which is stale. - Map.Entry lastUsedEntry = lastUsed.firstEntry(); - if (lastUsedEntry == null) { - return false; - } else if (now - lastUsedEntry.getKey().lastUsedMs() > evictionMs) { - ShareSession session = lastUsedEntry.getValue(); - remove(session); - evictionsMeter.mark(); - return true; - } - return false; + public synchronized void updateNumPartitions(ShareSession session) { + numPartitions += session.updateCachedSize(); } /** * Maybe create a new session and add it to the cache. * @param groupId - The group id in the share fetch request. * @param memberId - The member id in the share fetch request. - * @param now - The current time in milliseconds. * @param partitionMap - The topic partitions to be added to the session. * @return - The session key if the session was created, or null if the session was not created. */ - public synchronized ShareSessionKey maybeCreateSession(String groupId, Uuid memberId, long now, ImplicitLinkedHashCollection partitionMap) { - if (sessions.size() < maxEntries || tryEvict(now)) { + public synchronized ShareSessionKey maybeCreateSession(String groupId, Uuid memberId, ImplicitLinkedHashCollection partitionMap) { + if (sessions.size() < maxEntries) { ShareSession session = new ShareSession(new ShareSessionKey(groupId, memberId), partitionMap, - now, now, ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)); + ShareRequestMetadata.nextEpoch(ShareRequestMetadata.INITIAL_EPOCH)); sessions.put(session.key(), session); - touch(session, now); + updateNumPartitions(session); return session.key(); } return null; diff --git a/server/src/test/java/org/apache/kafka/network/RequestConvertToJsonTest.java b/server/src/test/java/org/apache/kafka/network/RequestConvertToJsonTest.java index 85a34ccb0e07b..be0e4337c1413 100644 --- a/server/src/test/java/org/apache/kafka/network/RequestConvertToJsonTest.java +++ b/server/src/test/java/org/apache/kafka/network/RequestConvertToJsonTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.network.ClientInformation; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; +import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.common.protocol.Readable; @@ -35,7 +36,6 @@ import org.junit.jupiter.api.Test; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -87,8 +87,8 @@ public void testAllApiVersionsResponseHandled() { message = ApiMessageType.fromApiKey(key.id).newResponse(); } - ByteBuffer bytes = MessageUtil.toByteBufferAccessor(message, version).buffer(); - AbstractResponse response = AbstractResponse.parseResponse(key, bytes, version); + ByteBufferAccessor readable = MessageUtil.toByteBufferAccessor(message, version); + AbstractResponse response = AbstractResponse.parseResponse(key, readable, version); try { RequestConvertToJson.response(response, version); } catch (IllegalStateException e) { @@ -106,8 +106,8 @@ public void testAllResponseTypesHandled() { if (key.hasValidVersion()) { short version = key.latestVersion(); ApiMessage message = ApiMessageType.fromApiKey(key.id).newResponse(); - ByteBuffer bytes = MessageUtil.toByteBufferAccessor(message, version).buffer(); - AbstractResponse res = AbstractResponse.parseResponse(key, bytes, version); + ByteBufferAccessor readable = MessageUtil.toByteBufferAccessor(message, version); + AbstractResponse res = AbstractResponse.parseResponse(key, readable, version); try { RequestConvertToJson.response(res, version); } catch (IllegalStateException e) { diff --git a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java index 4de1ffa497582..ca18de5b65c47 100644 --- a/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java +++ b/server/src/test/java/org/apache/kafka/server/share/session/ShareSessionCacheTest.java @@ -33,7 +33,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; public class ShareSessionCacheTest { @@ -44,46 +43,24 @@ public void setUp() { @Test public void testShareSessionCache() throws InterruptedException { - ShareSessionCache cache = new ShareSessionCache(3, 100); + ShareSessionCache cache = new ShareSessionCache(3); assertEquals(0, cache.size()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 0, mockedSharePartitionMap(10)); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 10, mockedSharePartitionMap(20)); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 20, mockedSharePartitionMap(30)); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), 30, mockedSharePartitionMap(40))); - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), 40, mockedSharePartitionMap(5))); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(10)); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(20)); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(30)); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(40))); + assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5))); assertShareCacheContains(cache, List.of(key1, key2, key3)); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 3, - "Share session count should be 3."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 60, - "Share partition count should be 60."); - assertEquals(0, cache.evictionsMeter().count()); - - // Touch the sessions to update the last used time, so that the key-2 can be evicted. - cache.touch(cache.get(key1), 200); - ShareSessionKey key4 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 210, mockedSharePartitionMap(11)); - assertShareCacheContains(cache, List.of(key1, key3, key4)); - - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 3, - "Share session count should be 3."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 51, - "Share partition count should be 51."); - assertEquals(1, cache.evictionsMeter().count()); - assertTrue(cache.evictionsMeter().meanRate() > 0); - - cache.touch(cache.get(key1), 400); - cache.touch(cache.get(key3), 390); - cache.touch(cache.get(key4), 400); - // No key should be evicted as all the sessions are touched to latest time. - assertNull(cache.maybeCreateSession("grp", Uuid.randomUuid(), 410, mockedSharePartitionMap(50))); + assertMetricsValues(3, 60, 0, cache); } @Test public void testResizeCachedSessions() throws InterruptedException { - ShareSessionCache cache = new ShareSessionCache(2, 100); + ShareSessionCache cache = new ShareSessionCache(2); assertEquals(0, cache.size()); assertEquals(0, cache.totalPartitions()); - ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 0, mockedSharePartitionMap(2)); + ShareSessionKey key1 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(2)); assertNotNull(key1); assertShareCacheContains(cache, List.of(key1)); ShareSession session1 = cache.get(key1); @@ -91,28 +68,20 @@ public void testResizeCachedSessions() throws InterruptedException { assertEquals(2, cache.totalPartitions()); assertEquals(1, cache.size()); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 1, - "Share session count should be 1."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 2, - "Share partition count should be 2."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(1, 2, 0, cache); - ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 0, mockedSharePartitionMap(4)); + ShareSessionKey key2 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(4)); assertNotNull(key2); assertShareCacheContains(cache, List.of(key1, key2)); ShareSession session2 = cache.get(key2); assertEquals(6, cache.totalPartitions()); assertEquals(2, cache.size()); - cache.touch(session1, 200); - cache.touch(session2, 200); + cache.updateNumPartitions(session1); + cache.updateNumPartitions(session2); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 2, - "Share session count should be 2."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 6, - "Share partition count should be 6."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(2, 6, 0, cache); - ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), 200, mockedSharePartitionMap(5)); + ShareSessionKey key3 = cache.maybeCreateSession("grp", Uuid.randomUuid(), mockedSharePartitionMap(5)); assertNull(key3); assertShareCacheContains(cache, List.of(key1, key2)); assertEquals(6, cache.totalPartitions()); @@ -122,11 +91,7 @@ public void testResizeCachedSessions() throws InterruptedException { assertEquals(1, cache.size()); assertEquals(4, cache.totalPartitions()); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 1, - "Share session count should be 1."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 4, - "Share partition count should be 4."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(1, 4, 0, cache); Iterator iterator = session2.partitionMap().iterator(); iterator.next(); @@ -137,15 +102,11 @@ public void testResizeCachedSessions() throws InterruptedException { assertEquals(4, session2.cachedSize()); assertEquals(4, cache.totalPartitions()); // Touch the session to update the changes in cache and session's cached size. - cache.touch(session2, session2.lastUsedMs()); + cache.updateNumPartitions(session2); assertEquals(3, session2.cachedSize()); assertEquals(3, cache.totalPartitions()); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == 1, - "Share session count should be 1."); - TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == 3, - "Share partition count should be 3."); - assertEquals(0, cache.evictionsMeter().count()); + assertMetricsValues(1, 3, 0, cache); } private ImplicitLinkedHashCollection mockedSharePartitionMap(int size) { @@ -165,4 +126,17 @@ private void assertShareCacheContains(ShareSessionCache cache, "Missing session " + ++i + " out of " + sessionKeys.size() + " ( " + sessionKey + " )"); } } + + private void assertMetricsValues( + int shareSessionsCount, + int sharePartitionsCount, + int evictionsCount, + ShareSessionCache cache + ) throws InterruptedException { + TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_SESSIONS_COUNT).intValue() == shareSessionsCount, + "Share session count should be " + shareSessionsCount); + TestUtils.waitForCondition(() -> yammerMetricValue(ShareSessionCache.SHARE_PARTITIONS_COUNT).intValue() == sharePartitionsCount, + "Share partition count should be " + sharePartitionsCount); + assertEquals(evictionsCount, cache.evictionsMeter().count()); + } } diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java index f4a11abea8eef..b92947af9cd60 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorConfig.java @@ -76,6 +76,10 @@ public class ShareCoordinatorConfig { public static final int STATE_TOPIC_PRUNE_INTERVAL_MS_DEFAULT = 5 * 60 * 1000; // 5 minutes public static final String STATE_TOPIC_PRUNE_INTERVAL_MS_DOC = "The duration in milliseconds that the share coordinator will wait between pruning eligible records in share-group state topic."; + public static final String COLD_PARTITION_SNAPSHOT_INTERVAL_MS_CONFIG = "share.coordinator.cold.partition.snapshot.interval.ms"; + public static final int COLD_PARTITION_SNAPSHOT_INTERVAL_MS_DEFAULT = 5 * 60 * 1000; // 5 minutes + public static final String COLD_PARTITION_SNAPSHOT_INTERVAL_MS_DOC = "The duration in milliseconds that the share coordinator will wait between force snapshotting share partitions which are not being updated."; + public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(STATE_TOPIC_NUM_PARTITIONS_CONFIG, INT, STATE_TOPIC_NUM_PARTITIONS_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_NUM_PARTITIONS_DOC) .define(STATE_TOPIC_REPLICATION_FACTOR_CONFIG, SHORT, STATE_TOPIC_REPLICATION_FACTOR_DEFAULT, atLeast(1), HIGH, STATE_TOPIC_REPLICATION_FACTOR_DOC) @@ -87,7 +91,8 @@ public class ShareCoordinatorConfig { .define(STATE_TOPIC_COMPRESSION_CODEC_CONFIG, INT, (int) STATE_TOPIC_COMPRESSION_CODEC_DEFAULT.id, HIGH, STATE_TOPIC_COMPRESSION_CODEC_DOC) .define(APPEND_LINGER_MS_CONFIG, INT, APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, APPEND_LINGER_MS_DOC) .define(WRITE_TIMEOUT_MS_CONFIG, INT, WRITE_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, WRITE_TIMEOUT_MS_DOC) - .defineInternal(STATE_TOPIC_PRUNE_INTERVAL_MS_CONFIG, INT, STATE_TOPIC_PRUNE_INTERVAL_MS_DEFAULT, atLeast(1), LOW, STATE_TOPIC_PRUNE_INTERVAL_MS_DOC); + .defineInternal(STATE_TOPIC_PRUNE_INTERVAL_MS_CONFIG, INT, STATE_TOPIC_PRUNE_INTERVAL_MS_DEFAULT, atLeast(1), LOW, STATE_TOPIC_PRUNE_INTERVAL_MS_DOC) + .defineInternal(COLD_PARTITION_SNAPSHOT_INTERVAL_MS_CONFIG, INT, COLD_PARTITION_SNAPSHOT_INTERVAL_MS_DEFAULT, atLeast(1), LOW, COLD_PARTITION_SNAPSHOT_INTERVAL_MS_DOC); private final int stateTopicNumPartitions; private final short stateTopicReplicationFactor; @@ -100,7 +105,7 @@ public class ShareCoordinatorConfig { private final CompressionType compressionType; private final int appendLingerMs; private final int pruneIntervalMs; - + private final int coldPartitionSnapshotIntervalMs; public ShareCoordinatorConfig(AbstractConfig config) { stateTopicNumPartitions = config.getInt(STATE_TOPIC_NUM_PARTITIONS_CONFIG); @@ -116,6 +121,7 @@ public ShareCoordinatorConfig(AbstractConfig config) { .orElse(null); appendLingerMs = config.getInt(APPEND_LINGER_MS_CONFIG); pruneIntervalMs = config.getInt(STATE_TOPIC_PRUNE_INTERVAL_MS_CONFIG); + coldPartitionSnapshotIntervalMs = config.getInt(COLD_PARTITION_SNAPSHOT_INTERVAL_MS_CONFIG); validate(); } @@ -163,6 +169,10 @@ public int shareCoordinatorTopicPruneIntervalMs() { return pruneIntervalMs; } + public int shareCoordinatorColdPartitionSnapshotIntervalMs() { + return coldPartitionSnapshotIntervalMs; + } + private void validate() { Utils.require(snapshotUpdateRecordsPerSnapshot >= 0 && snapshotUpdateRecordsPerSnapshot <= 500, String.format("%s must be between [0, 500]", SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG)); diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java index 8b87aed65ef2f..321a1e92ac09e 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java @@ -261,10 +261,15 @@ public void startup( log.info("Starting up."); numPartitions = shareGroupTopicPartitionCount.getAsInt(); - setupRecordPruning(); + setupPeriodicJobs(); log.info("Startup complete."); } + private void setupPeriodicJobs() { + setupRecordPruning(); + setupSnapshotColdPartitions(); + } + private void setupRecordPruning() { log.info("Scheduling share-group state topic prune job."); timer.add(new TimerTask(config.shareCoordinatorTopicPruneIntervalMs()) { @@ -341,6 +346,28 @@ private CompletableFuture performRecordPruning(TopicPartition tp) { return fut; } + private void setupSnapshotColdPartitions() { + log.info("Scheduling cold share-partition snapshotting."); + timer.add(new TimerTask(config.shareCoordinatorColdPartitionSnapshotIntervalMs()) { + @Override + public void run() { + List> futures = runtime.scheduleWriteAllOperation( + "snapshot-cold-partitions", + Duration.ofMillis(config.shareCoordinatorWriteTimeoutMs()), + ShareCoordinatorShard::snapshotColdPartitions + ); + + CompletableFuture.allOf(futures.toArray(new CompletableFuture[]{})) + .whenComplete((__, exp) -> { + if (exp != null) { + log.error("Received error while snapshotting cold partitions.", exp); + } + setupSnapshotColdPartitions(); + }); + } + }); + } + @Override public void shutdown() { if (!isActive.compareAndSet(true, false)) { diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java index 35891bddf8d94..d38564fd6f849 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java @@ -66,6 +66,7 @@ import org.slf4j.Logger; +import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -574,6 +575,46 @@ public CoordinatorResult(List.of(record), responseData); } + /** + * Iterates over the soft state to determine the share partitions whose last snapshot is + * older than the allowed time interval. The candidate share partitions are force snapshotted. + * + * @return A result containing snapshot records, if any, and a void response. + */ + public CoordinatorResult snapshotColdPartitions() { + long coldSnapshottedPartitionsCount = shareStateMap.values().stream() + .filter(shareGroupOffset -> shareGroupOffset.createTimestamp() - shareGroupOffset.writeTimestamp() != 0) + .count(); + + // If all share partitions are snapshotted, it means that + // system is quiet and cold snapshotting will not help much. + if (coldSnapshottedPartitionsCount == shareStateMap.size()) { + log.debug("All share snapshot records already cold snapshotted, skipping."); + return new CoordinatorResult<>(List.of(), null); + } + + // Some active partitions are there. + List records = new ArrayList<>(); + + shareStateMap.forEach((sharePartitionKey, shareGroupOffset) -> { + long timeSinceLastSnapshot = time.milliseconds() - shareGroupOffset.writeTimestamp(); + if (timeSinceLastSnapshot >= config.shareCoordinatorColdPartitionSnapshotIntervalMs()) { + // We need to force create a snapshot here + log.info("Last snapshot for {} is older than allowed interval.", sharePartitionKey); + records.add(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( + sharePartitionKey.groupId(), + sharePartitionKey.topicId(), + sharePartitionKey.partition(), + shareGroupOffset.builderSupplier() + .setSnapshotEpoch(shareGroupOffset.snapshotEpoch() + 1) // We need to increment by one as this is a new snapshot. + .setWriteTimestamp(time.milliseconds()) + .build() + )); + } + }); + return new CoordinatorResult<>(records, null); + } + /** * Util method to generate a ShareSnapshot or ShareUpdate type record for a key, based on various conditions. *

@@ -589,6 +630,7 @@ private CoordinatorRecord generateShareStateRecord( WriteShareGroupStateRequestData.PartitionData partitionData, SharePartitionKey key ) { + long timestamp = time.milliseconds(); if (!shareStateMap.containsKey(key)) { // Since this is the first time we are getting a write request for key, we should be creating a share snapshot record. // The incoming partition data could have overlapping state batches, we must merge them @@ -600,8 +642,8 @@ private CoordinatorRecord generateShareStateRecord( .setLeaderEpoch(partitionData.leaderEpoch()) .setStateEpoch(partitionData.stateEpoch()) .setStateBatches(mergeBatches(List.of(), partitionData)) - .setCreateTimestamp(time.milliseconds()) - .setWriteTimestamp(time.milliseconds()) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp) .build()); } else if (snapshotUpdateCount.getOrDefault(key, 0) >= config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot()) { ShareGroupOffset currentState = shareStateMap.get(key); // shareStateMap will have the entry as containsKey is true @@ -620,8 +662,8 @@ private CoordinatorRecord generateShareStateRecord( .setLeaderEpoch(newLeaderEpoch) .setStateEpoch(newStateEpoch) .setStateBatches(mergeBatches(currentState.stateBatches(), partitionData, newStartOffset)) - .setCreateTimestamp(time.milliseconds()) - .setWriteTimestamp(time.milliseconds()) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp) .build()); } else { ShareGroupOffset currentState = shareStateMap.get(key); // shareStateMap will have the entry as containsKey is true. @@ -636,8 +678,6 @@ private CoordinatorRecord generateShareStateRecord( .setStartOffset(partitionData.startOffset()) .setLeaderEpoch(partitionData.leaderEpoch()) .setStateBatches(mergeBatches(List.of(), partitionData)) - .setCreateTimestamp(currentState.createTimestamp()) - .setWriteTimestamp(currentState.writeTimestamp()) .build()); } } diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java index d2a91a5bc7a18..c0397a273f4c0 100644 --- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java +++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareGroupOffset.java @@ -24,7 +24,6 @@ import org.apache.kafka.server.share.persister.PersisterStateBatch; import java.util.Collections; -import java.util.LinkedHashSet; import java.util.List; import java.util.Objects; @@ -33,8 +32,8 @@ * This class is immutable (state batches is not modified out of context). */ public class ShareGroupOffset { - public static final int NO_TIMESTAMP = -1; - public static final int UNINITIALIZED_EPOCH = -1; + public static final int NO_TIMESTAMP = 0; + public static final int UNINITIALIZED_EPOCH = 0; public static final int DEFAULT_EPOCH = 0; private final int snapshotEpoch; @@ -161,10 +160,6 @@ public static ShareGroupOffset fromRequest(InitializeShareGroupStateRequestData. ); } - public LinkedHashSet stateBatchAsSet() { - return new LinkedHashSet<>(stateBatches); - } - public static class Builder { private int snapshotEpoch; private int stateEpoch; @@ -195,7 +190,7 @@ public Builder setStartOffset(long startOffset) { } public Builder setStateBatches(List stateBatches) { - this.stateBatches = stateBatches; + this.stateBatches = stateBatches == null ? Collections.emptyList() : stateBatches.stream().toList(); return this; } @@ -245,4 +240,15 @@ public String toString() { ", stateBatches=" + stateBatches + '}'; } + + public Builder builderSupplier() { + return new Builder() + .setSnapshotEpoch(snapshotEpoch()) + .setStateEpoch(stateEpoch()) + .setLeaderEpoch(leaderEpoch()) + .setStartOffset(startOffset()) + .setStateBatches(stateBatches()) + .setCreateTimestamp(createTimestamp()) + .setWriteTimestamp(writeTimestamp()); + } } diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java index 399643e32a9ad..a4b316d9443c8 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java @@ -1458,7 +1458,7 @@ public void testRecordPruningTaskPeriodicityWithAllSuccess() throws Exception { any(), any()); - timer.advanceClock(30005L); // prune should be called + timer.advanceClock(30005L); // Prune should be called. verify(runtime, times(1)) .scheduleWriteOperation( eq("write-state-record-prune"), @@ -1466,7 +1466,7 @@ public void testRecordPruningTaskPeriodicityWithAllSuccess() throws Exception { any(), any()); - timer.advanceClock(30005L); // prune should be called + timer.advanceClock(30005L); // Prune should be called. verify(runtime, times(2)) .scheduleWriteOperation( eq("write-state-record-prune"), @@ -1871,6 +1871,113 @@ public void testRecordPruningTaskRetriesRepeatedSameOffsetForTopic() throws Exce service.shutdown(); } + @Test + public void testColdPartitionSnapshotTaskPeriodicityWithAllSuccess() throws Exception { + CoordinatorRuntime runtime = mockRuntime(); + MockTime time = new MockTime(); + MockTimer timer = new MockTimer(time); + PartitionWriter writer = mock(PartitionWriter.class); + + Metrics metrics = new Metrics(); + + ShareCoordinatorService service = spy(new ShareCoordinatorService( + new LogContext(), + ShareCoordinatorTestConfig.testConfig(), + runtime, + new ShareCoordinatorMetrics(metrics), + time, + timer, + writer + )); + + when(runtime.scheduleWriteAllOperation( + eq("snapshot-cold-partitions"), + any(), + any() + )).thenReturn(List.of(CompletableFuture.completedFuture(null))); + + service.startup(() -> 1); + verify(runtime, times(0)) + .scheduleWriteOperation( + eq("snapshot-cold-partitions"), + any(), + any(), + any()); + + timer.advanceClock(10005L); // Snapshotting should be called. + verify(runtime, times(1)) + .scheduleWriteAllOperation( + eq("snapshot-cold-partitions"), + any(), + any()); + + timer.advanceClock(10005L); // Snapshotting should be called. + verify(runtime, times(2)) + .scheduleWriteAllOperation( + eq("snapshot-cold-partitions"), + any(), + any()); + + checkMetrics(metrics); + + service.shutdown(); + } + + @Test + public void testColdPartitionSnapshotTaskPeriodicityWithSomeFailures() throws Exception { + CoordinatorRuntime runtime = mockRuntime(); + MockTime time = new MockTime(); + MockTimer timer = new MockTimer(time); + PartitionWriter writer = mock(PartitionWriter.class); + + when(runtime.scheduleWriteAllOperation( + eq("snapshot-cold-partitions"), + any(), + any() + )).thenReturn( + List.of(CompletableFuture.completedFuture(null), CompletableFuture.failedFuture(new Exception("bad stuff"))) + ).thenReturn( + List.of(CompletableFuture.completedFuture(null), CompletableFuture.completedFuture(null)) + ); + + Metrics metrics = new Metrics(); + + ShareCoordinatorService service = spy(new ShareCoordinatorService( + new LogContext(), + ShareCoordinatorTestConfig.testConfig(), + runtime, + new ShareCoordinatorMetrics(metrics), + time, + timer, + writer + )); + + service.startup(() -> 2); + verify(runtime, times(0)) + .scheduleWriteAllOperation( + eq("snapshot-cold-partitions"), + any(), + any()); + + timer.advanceClock(10005L); // Snapshotting should be called. + verify(runtime, times(1)) // For 2 topic partitions. + .scheduleWriteAllOperation( + eq("snapshot-cold-partitions"), + any(), + any()); + + timer.advanceClock(10005L); // Snapshotting should be called (despite previous partial failure). + verify(runtime, times(2)) // For 2 topic partitions. + .scheduleWriteAllOperation( + eq("snapshot-cold-partitions"), + any(), + any()); + + checkMetrics(metrics); + + service.shutdown(); + } + @Test public void testShareStateTopicConfigs() { CoordinatorRuntime runtime = mockRuntime(); diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java index 20be20832d823..e4b95d1e43906 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java @@ -44,6 +44,7 @@ import org.apache.kafka.coordinator.common.runtime.CoordinatorResult; import org.apache.kafka.coordinator.share.generated.ShareSnapshotKey; import org.apache.kafka.coordinator.share.generated.ShareSnapshotValue; +import org.apache.kafka.coordinator.share.generated.ShareUpdateKey; import org.apache.kafka.coordinator.share.generated.ShareUpdateValue; import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics; import org.apache.kafka.image.MetadataImage; @@ -1357,6 +1358,351 @@ public void testInitializePartitionIdNonExistentInMetadataImage() { verify(topicsImage, times(1)).getPartition(eq(TOPIC_ID), eq(0)); } + @Test + public void testSnapshotColdPartitionsNoEligiblePartitions() { + ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); + MetadataImage image = mock(MetadataImage.class); + shard.onNewMetadataImage(image, null); + int offset = 0; + int producerId = 0; + short producerEpoch = 0; + int leaderEpoch = 0; + + long timestamp = TIME.milliseconds(); + + CoordinatorRecord record1 = CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(0) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ); + + CoordinatorRecord record2 = CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(1), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(0) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ); + + shard.replay(offset, producerId, producerEpoch, record1); + shard.replay(offset + 1, producerId, producerEpoch, record2); + + assertNotNull(shard.getShareStateMapValue(SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, 0))); + assertNotNull(shard.getShareStateMapValue(SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, 1))); + + TIME.sleep(5000); // Less than config. + + assertEquals(0, shard.snapshotColdPartitions().records().size()); + } + + @Test + public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() { + ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); + MetadataImage image = mock(MetadataImage.class); + shard.onNewMetadataImage(image, null); + int offset = 0; + int producerId = 0; + short producerEpoch = 0; + int leaderEpoch = 0; + + long timestamp = TIME.milliseconds(); + + CoordinatorRecord record1 = CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(0) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ); + + SharePartitionKey key = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, 0); + + shard.replay(offset, producerId, producerEpoch, record1); + assertNotNull(shard.getShareStateMapValue(key)); + + long sleep = 12000; + TIME.sleep(sleep); + + List expectedRecords = List.of( + CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(1) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp + sleep) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ) + ); + + assertEquals(expectedRecords, shard.snapshotColdPartitions().records()); + + shard.replay(offset + 1, producerId, producerEpoch, expectedRecords.get(0)); + assertNotNull(shard.getShareStateMapValue(key)); + + CoordinatorRecord record2 = CoordinatorRecord.record( + new ShareUpdateKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareUpdateValue() + .setSnapshotEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setStateBatches(List.of( + new ShareUpdateValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ); + + shard.replay(offset + 2, producerId, producerEpoch, record2); + + TIME.sleep(sleep); + + assertNotNull(shard.getShareStateMapValue(key)); + assertEquals(timestamp + sleep, shard.getShareStateMapValue(key).writeTimestamp()); // No snapshot since update has no time info. + } + + @Test + public void testSnapshotColdPartitionsDoesNotPerpetuallySnapshot() { + ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); + MetadataImage image = mock(MetadataImage.class); + shard.onNewMetadataImage(image, null); + int offset = 0; + int producerId = 0; + short producerEpoch = 0; + int leaderEpoch = 0; + + long timestamp = TIME.milliseconds(); + + CoordinatorRecord record1 = CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(0) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ); + + shard.replay(offset, producerId, producerEpoch, record1); + assertNotNull(shard.getShareStateMapValue(SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, 0))); + + long sleep = 12000; + TIME.sleep(sleep); + + List expectedRecords = List.of( + CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(1) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp + sleep) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ) + ); + + assertEquals(expectedRecords, shard.snapshotColdPartitions().records()); + + shard.replay(offset + 1, producerId, producerEpoch, expectedRecords.get(0)); + assertNotNull(shard.getShareStateMapValue(SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, 0))); + + // Since all existing snapshots are already snapshotted, no new records will be created. + TIME.sleep(12000); + + assertEquals(0, shard.snapshotColdPartitions().records().size()); + } + + @Test + public void testSnapshotColdPartitionsPartialEligiblePartitions() { + ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); + MetadataImage image = mock(MetadataImage.class); + shard.onNewMetadataImage(image, null); + int offset = 0; + int producerId = 0; + short producerEpoch = 0; + int leaderEpoch = 0; + SharePartitionKey key0 = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, 0); + SharePartitionKey key1 = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, 1); + + long timestamp = TIME.milliseconds(); + int record1SnapshotEpoch = 0; + + CoordinatorRecord record1 = CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(record1SnapshotEpoch) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ); + + long delta = 15000; // 15 seconds + + CoordinatorRecord record2 = CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(1), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(0) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp + delta) + .setWriteTimestamp(timestamp + delta) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ); + + shard.replay(offset, producerId, producerEpoch, record1); + shard.replay(offset + 1, producerId, producerEpoch, record2); + + assertNotNull(shard.getShareStateMapValue(key0)); + assertNotNull(shard.getShareStateMapValue(key1)); + assertEquals(timestamp, shard.getShareStateMapValue(key0).writeTimestamp()); + assertEquals(timestamp + delta, shard.getShareStateMapValue(key1).writeTimestamp()); + + long sleep = 12000; + TIME.sleep(sleep); // Record 1 is eligible now. + + List expectedRecords = List.of( + CoordinatorRecord.record( + new ShareSnapshotKey() + .setGroupId(GROUP_ID) + .setTopicId(TOPIC_ID) + .setPartition(0), + new ApiMessageAndVersion( + new ShareSnapshotValue() + .setSnapshotEpoch(record1SnapshotEpoch + 1) + .setStateEpoch(0) + .setLeaderEpoch(leaderEpoch) + .setCreateTimestamp(timestamp) + .setWriteTimestamp(timestamp + sleep) + .setStateBatches(List.of( + new ShareSnapshotValue.StateBatch() + .setFirstOffset(0) + .setLastOffset(10) + .setDeliveryCount((short) 1) + .setDeliveryState((byte) 0))), + (short) 0 + ) + ) + ); + + List records = shard.snapshotColdPartitions().records(); + assertEquals(expectedRecords, records); + + shard.replay(offset + 2, producerId, producerEpoch, records.get(0)); + + assertEquals(timestamp + delta, shard.getShareStateMapValue(key1).writeTimestamp()); + assertEquals(timestamp + sleep, shard.getShareStateMapValue(key0).writeTimestamp()); + } + private static ShareGroupOffset groupOffset(ApiMessage record) { if (record instanceof ShareSnapshotValue) { return ShareGroupOffset.fromRecord((ShareSnapshotValue) record); diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorTestConfig.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorTestConfig.java index eab6f2966ac6d..853bc11943273 100644 --- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorTestConfig.java +++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorTestConfig.java @@ -50,6 +50,7 @@ private static Map testConfigMapRaw() { configs.put(ShareCoordinatorConfig.APPEND_LINGER_MS_CONFIG, "10"); configs.put(ShareCoordinatorConfig.STATE_TOPIC_COMPRESSION_CODEC_CONFIG, String.valueOf(CompressionType.NONE.id)); configs.put(ShareCoordinatorConfig.STATE_TOPIC_PRUNE_INTERVAL_MS_CONFIG, "30000"); // 30 seconds + configs.put(ShareCoordinatorConfig.COLD_PARTITION_SNAPSHOT_INTERVAL_MS_CONFIG, "10000"); // 10 seconds return configs; } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java index 2c0086c4700a1..939db06c83ea0 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogCleaner.java @@ -556,15 +556,14 @@ private boolean cleanFilthiestLog() throws LogCleaningException { } else { // there's a log, clean it this.lastPreCleanStats = preCleanStats; - LogToClean cleanable = null; + LogToClean cleanable = ltc.get(); try { - cleanable = ltc.get(); cleanLog(cleanable); cleaned = true; } catch (ThreadShutdownException e) { throw e; } catch (Exception e) { - throw new LogCleaningException(cleanable != null ? cleanable.log() : null, e.getMessage(), e); + throw new LogCleaningException(cleanable.log(), e.getMessage(), e); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogManager.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogManager.java new file mode 100644 index 0000000000000..cc46e4fc98436 --- /dev/null +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogManager.java @@ -0,0 +1,96 @@ +/* + * 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.storage.internals.log; + +import org.apache.kafka.common.Uuid; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.metadata.PartitionRegistration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.Future; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +public class LogManager { + + private static final Logger LOG = LoggerFactory.getLogger(LogManager.class); + + public static final String LOCK_FILE_NAME = ".lock"; + public static final String RECOVERY_POINT_CHECKPOINT_FILE = "recovery-point-offset-checkpoint"; + public static final String LOG_START_OFFSET_CHECKPOINT_FILE = "log-start-offset-checkpoint"; + + /** + * Wait for all jobs to complete + * @param jobs The jobs + * @param callback This will be called to handle the exception caused by each Future#get + * @return true if all pass. Otherwise, false + */ + public static boolean waitForAllToComplete(List> jobs, Consumer callback) { + List> failed = new ArrayList<>(); + for (Future job : jobs) { + try { + job.get(); + } catch (Exception e) { + callback.accept(e); + failed.add(job); + } + } + return failed.isEmpty(); + } + + /** + * Returns true if the given log should not be on the current broker + * according to the metadata image. + * + * @param brokerId The ID of the current broker. + * @param newTopicsImage The new topics image after broker has been reloaded + * @param log The log object to check + * @return true if the log should not exist on the broker, false otherwise. + */ + public static boolean isStrayKraftReplica(int brokerId, TopicsImage newTopicsImage, UnifiedLog log) { + if (log.topicId().isEmpty()) { + // Missing topic ID could result from storage failure or unclean shutdown after topic creation but before flushing + // data to the `partition.metadata` file. And before appending data to the log, the `partition.metadata` is always + // flushed to disk. So if the topic ID is missing, it mostly means no data was appended, and we can treat this as + // a stray log. + LOG.info("The topicId does not exist in {}, treat it as a stray log.", log); + return true; + } + + Uuid topicId = log.topicId().get(); + int partitionId = log.topicPartition().partition(); + PartitionRegistration partition = newTopicsImage.getPartition(topicId, partitionId); + if (partition == null) { + LOG.info("Found stray log dir {}: the topicId {} does not exist in the metadata image.", log, topicId); + return true; + } else { + List replicas = Arrays.stream(partition.replicas).boxed().toList(); + if (!replicas.contains(brokerId)) { + LOG.info("Found stray log dir {}: the current replica assignment {} does not contain the local brokerId {}.", + log, replicas.stream().map(String::valueOf).collect(Collectors.joining(", ", "[", "]")), brokerId); + return true; + } else { + return false; + } + } + } +} diff --git a/storage/src/test/java/org/apache/kafka/storage/internals/log/LogManagerTest.java b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogManagerTest.java new file mode 100644 index 0000000000000..a86ec3691cf72 --- /dev/null +++ b/storage/src/test/java/org/apache/kafka/storage/internals/log/LogManagerTest.java @@ -0,0 +1,175 @@ +/* + * 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.storage.internals.log; + +import org.apache.kafka.common.DirectoryId; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.Uuid; +import org.apache.kafka.image.TopicImage; +import org.apache.kafka.image.TopicsImage; +import org.apache.kafka.metadata.LeaderRecoveryState; +import org.apache.kafka.metadata.PartitionRegistration; + +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class LogManagerTest { + + private static final TopicIdPartition FOO_0 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 0)); + private static final TopicIdPartition FOO_1 = new TopicIdPartition(Uuid.fromString("Sl08ZXU2QW6uF5hIoSzc8w"), new TopicPartition("foo", 1)); + private static final TopicIdPartition BAR_0 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 0)); + private static final TopicIdPartition BAR_1 = new TopicIdPartition(Uuid.fromString("69O438ZkTSeqqclTtZO2KA"), new TopicPartition("bar", 1)); + private static final TopicIdPartition QUUX_0 = new TopicIdPartition(Uuid.fromString("YS9owjv5TG2OlsvBM0Qw6g"), new TopicPartition("quux", 0)); + + @SuppressWarnings("unchecked") + @Test + public void testWaitForAllToComplete() throws ExecutionException, InterruptedException { + AtomicInteger invokedCount = new AtomicInteger(0); + Future success = mock(Future.class); + when(success.get()).thenAnswer(a -> { + invokedCount.incrementAndGet(); + return true; + }); + Future failure = mock(Future.class); + when(failure.get()).thenAnswer(a -> { + invokedCount.incrementAndGet(); + throw new RuntimeException(); + }); + + AtomicInteger failureCount = new AtomicInteger(0); + // all futures should be evaluated + assertFalse(LogManager.waitForAllToComplete(List.of(success, failure), t -> failureCount.incrementAndGet())); + assertEquals(2, invokedCount.get()); + assertEquals(1, failureCount.get()); + assertFalse(LogManager.waitForAllToComplete(List.of(failure, success), t -> failureCount.incrementAndGet())); + assertEquals(4, invokedCount.get()); + assertEquals(2, failureCount.get()); + assertTrue(LogManager.waitForAllToComplete(List.of(success, success), t -> failureCount.incrementAndGet())); + assertEquals(6, invokedCount.get()); + assertEquals(2, failureCount.get()); + assertFalse(LogManager.waitForAllToComplete(List.of(failure, failure), t -> failureCount.incrementAndGet())); + assertEquals(8, invokedCount.get()); + assertEquals(4, failureCount.get()); + } + + @Test + public void testIsStrayKraftReplicaWithEmptyImage() { + TopicsImage image = topicsImage(List.of()); + List onDisk = Stream.of(FOO_0, FOO_1, BAR_0, BAR_1, QUUX_0).map(this::mockLog).toList(); + assertTrue(onDisk.stream().allMatch(log -> LogManager.isStrayKraftReplica(0, image, log))); + } + + @Test + public void testIsStrayKraftReplicaInImage() { + TopicsImage image = topicsImage(List.of( + topicImage(Map.of( + FOO_0, List.of(0, 1, 2))), + topicImage(Map.of( + BAR_0, List.of(0, 1, 2), + BAR_1, List.of(0, 1, 2))) + )); + List onDisk = Stream.of(FOO_0, FOO_1, BAR_0, BAR_1, QUUX_0).map(this::mockLog).toList(); + Set expectedStrays = Stream.of(FOO_1, QUUX_0).map(TopicIdPartition::topicPartition).collect(Collectors.toSet()); + + onDisk.forEach(log -> assertEquals(expectedStrays.contains(log.topicPartition()), LogManager.isStrayKraftReplica(0, image, log))); + } + + @Test + public void testIsStrayKraftReplicaInImageWithRemoteReplicas() { + TopicsImage image = topicsImage(List.of( + topicImage(Map.of( + FOO_0, List.of(0, 1, 2))), + topicImage(Map.of( + BAR_0, List.of(1, 2, 3), + BAR_1, List.of(2, 3, 0))) + )); + List onDisk = Stream.of(FOO_0, BAR_0, BAR_1).map(this::mockLog).toList(); + Set expectedStrays = Stream.of(BAR_0).map(TopicIdPartition::topicPartition).collect(Collectors.toSet()); + onDisk.forEach(log -> assertEquals(expectedStrays.contains(log.topicPartition()), LogManager.isStrayKraftReplica(0, image, log))); + } + + @Test + public void testIsStrayKraftMissingTopicId() { + UnifiedLog log = mock(UnifiedLog.class); + when(log.topicId()).thenReturn(Optional.empty()); + assertTrue(LogManager.isStrayKraftReplica(0, topicsImage(List.of()), log)); + } + + private TopicsImage topicsImage(List topics) { + TopicsImage retval = TopicsImage.EMPTY; + for (TopicImage topic : topics) { + retval = retval.including(topic); + } + return retval; + } + + private TopicImage topicImage(Map> partitions) { + String topicName = null; + Uuid topicId = null; + for (TopicIdPartition partition : partitions.keySet()) { + if (topicId == null) { + topicId = partition.topicId(); + } else if (!topicId.equals(partition.topicId())) { + throw new IllegalArgumentException("partition topic IDs did not match"); + } + if (topicName == null) { + topicName = partition.topic(); + } else if (!topicName.equals(partition.topic())) { + throw new IllegalArgumentException("partition topic names did not match"); + } + } + if (topicId == null) { + throw new IllegalArgumentException("Invalid empty partitions map."); + } + Map partitionRegistrations = partitions.entrySet().stream().collect( + Collectors.toMap( + entry -> entry.getKey().partition(), + entry -> new PartitionRegistration.Builder() + .setReplicas(entry.getValue().stream().mapToInt(Integer::intValue).toArray()) + .setDirectories(DirectoryId.unassignedArray(entry.getValue().size())) + .setIsr(entry.getValue().stream().mapToInt(Integer::intValue).toArray()) + .setLeader(entry.getValue().get(0)) + .setLeaderRecoveryState(LeaderRecoveryState.RECOVERED) + .setLeaderEpoch(0) + .setPartitionEpoch(0) + .build())); + return new TopicImage(topicName, topicId, partitionRegistrations); + } + + private UnifiedLog mockLog(TopicIdPartition topicIdPartition) { + UnifiedLog log = mock(UnifiedLog.class); + when(log.topicId()).thenReturn(Optional.of(topicIdPartition.topicId())); + when(log.topicPartition()).thenReturn(topicIdPartition.topicPartition()); + return log; + } +} diff --git a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java index 73699bffac6d3..e82aeff7f8faf 100644 --- a/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java +++ b/storage/src/test/java/org/apache/kafka/tiered/storage/integration/FetchFromLeaderWithCorruptedCheckpointTest.java @@ -16,10 +16,10 @@ */ package org.apache.kafka.tiered.storage.integration; -import kafka.log.LogManager; import kafka.server.ReplicaManager; import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; +import org.apache.kafka.storage.internals.log.LogManager; import org.apache.kafka.tiered.storage.TieredStorageTestBuilder; import org.apache.kafka.tiered.storage.TieredStorageTestHarness; import org.apache.kafka.tiered.storage.specs.KeyValueSpec; @@ -50,7 +50,7 @@ protected void writeTestSpecifications(TieredStorageTestBuilder builder) { final Map> assignment = mkMap(mkEntry(p0, List.of(broker0, broker1))); final List checkpointFiles = List.of( ReplicaManager.HighWatermarkFilename(), - LogManager.RecoveryPointCheckpointFile(), + LogManager.RECOVERY_POINT_CHECKPOINT_FILE, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME); builder.createTopic(topicA, partitionCount, replicationFactor, maxBatchCountPerSegment, assignment, diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java index 2745cc7e02b53..82100a772939e 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.GroupProtocol; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -52,9 +53,10 @@ import org.junit.jupiter.api.BeforeAll; 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.io.IOException; import java.time.Duration; @@ -62,6 +64,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.Properties; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -86,7 +89,7 @@ public class StreamsUncaughtExceptionHandlerIntegrationTest { private static final long NOW = Instant.now().toEpochMilli(); - public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + public static final EmbeddedKafkaCluster CLUSTER = EmbeddedKafkaCluster.withStreamsRebalanceProtocol(1); @BeforeAll public static void startCluster() throws IOException { @@ -111,7 +114,13 @@ public static void closeCluster() { private Properties properties; - private Properties basicProps() { + private Properties basicProps(final boolean streamsRebalanceProtocolEnabled) { + final String protocol; + if (streamsRebalanceProtocolEnabled) { + protocol = GroupProtocol.STREAMS.name().toLowerCase(Locale.getDefault()); + } else { + protocol = GroupProtocol.CLASSIC.name().toLowerCase(Locale.getDefault()); + } return mkObjectProperties( mkMap( mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), @@ -120,7 +129,8 @@ private Properties basicProps() { mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2), mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), - mkEntry(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), 10000) + mkEntry(StreamsConfig.consumerPrefix(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG), 10000), + mkEntry(StreamsConfig.GROUP_PROTOCOL_CONFIG, protocol) ) ); } @@ -136,7 +146,6 @@ public void setup(final TestInfo testInfo) { IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic, inputTopic2, outputTopic, outputTopic2); final KStream stream = builder.stream(inputTopic); stream.process(() -> new ShutdownProcessor<>(processorValueCollector), Named.as("process")); - properties = basicProps(); } @AfterEach @@ -144,8 +153,10 @@ public void teardown() throws IOException { purgeLocalStreamsState(properties); } - @Test - public void shouldShutdownClient() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldShutdownClient(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { kafkaStreams.setUncaughtExceptionHandler(exception -> SHUTDOWN_CLIENT); @@ -159,29 +170,39 @@ public void shouldShutdownClient() throws Exception { } } - @Test - public void shouldReplaceThreads() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldReplaceThreads(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); testReplaceThreads(2); } - @Test - public void shouldReplaceThreadsWithoutJavaHandler() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldReplaceThreadsWithoutJavaHandler(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); Thread.setDefaultUncaughtExceptionHandler((t, e) -> fail("exception thrown")); testReplaceThreads(2); } - @Test - public void shouldReplaceSingleThread() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldReplaceSingleThread(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); testReplaceThreads(1); } - @Test - public void shouldShutdownMultipleThreadApplication() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldShutdownMultipleThreadApplication(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); testShutdownApplication(2); } - @Test - public void shouldShutdownSingleThreadApplication() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldShutdownSingleThreadApplication(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); testShutdownApplication(1); } @@ -212,8 +233,10 @@ public void process(final Record record) { } } - @Test - public void shouldShutDownClientIfGlobalStreamThreadWantsToReplaceThread() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldShutDownClientIfGlobalStreamThreadWantsToReplaceThread(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); builder.addGlobalStore( new KeyValueStoreBuilder<>( Stores.persistentKeyValueStore("globalStore"), @@ -239,8 +262,10 @@ public void shouldShutDownClientIfGlobalStreamThreadWantsToReplaceThread() throw } } - @Test - public void shouldEmitSameRecordAfterFailover() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void shouldEmitSameRecordAfterFailover(final boolean streamsRebalanceProtocolEnabled) throws Exception { + properties = basicProps(streamsRebalanceProtocolEnabled); properties.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); properties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 300000L); properties.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 4ee22f045564e..b51d5733b8275 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -67,7 +67,6 @@ import org.apache.kafka.streams.processor.internals.Task; import org.apache.kafka.streams.processor.internals.ThreadStateTransitionValidator; import org.apache.kafka.streams.processor.internals.TopologyMetadata; -import org.apache.kafka.streams.processor.internals.assignment.AssignorError; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology; import org.apache.kafka.streams.query.FailureReason; @@ -518,7 +517,7 @@ private void handleStreamsUncaughtException(final Throwable throwable, break; case SHUTDOWN_CLIENT: log.error( - "Encountered the following exception during processing and the registered exception handler" + + "Encountered the following exception during processing and the registered exception handler " + "opted to {}. The streams client is going to shut down now.", action, throwable @@ -542,7 +541,7 @@ private void handleStreamsUncaughtException(final Throwable throwable, closeToError(); break; } - processStreamThread(thread -> thread.sendShutdownRequest(AssignorError.SHUTDOWN_REQUESTED)); + processStreamThread(StreamThread::sendShutdownRequest); log.error("Encountered the following exception during processing " + "and sent shutdown request for the entire application.", throwable); break; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ReadOnlyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ReadOnlyTask.java index a895b71e4e952..dd5a2c6e1d79b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ReadOnlyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ReadOnlyTask.java @@ -180,7 +180,7 @@ public boolean maybePunctuateSystemTime() { } @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { throw new UnsupportedOperationException("This task is read-only"); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 417f754ca2c36..4c6e6674bdbcf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -179,7 +179,7 @@ public void resume() { * or flushing state store get IO errors; such error should cause the thread to die */ @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { switch (state()) { case CREATED: log.debug("Skipped preparing created task for commit"); @@ -189,7 +189,11 @@ public Map prepareCommit() { case RUNNING: case SUSPENDED: // do not need to flush state store caches in pre-commit since nothing would be sent for standby tasks - log.debug("Prepared {} task for committing", state()); + if (!clean) { + log.debug("Skipped preparing {} standby task with id {} for commit since the task is getting closed dirty.", state(), id); + } else { + log.debug("Prepared {} task for committing", state()); + } break; @@ -197,7 +201,7 @@ public Map prepareCommit() { throw new IllegalStateException("Illegal state " + state() + " while preparing standby task " + id + " for committing "); } - return Collections.emptyMap(); + return clean ? Collections.emptyMap() : null; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 424d6f7af6148..93737d8228933 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -417,7 +417,6 @@ public void resume() { timeCurrentIdlingStarted = Optional.empty(); } - public void flush() { stateMgr.flushCache(); recordCollector.flush(); @@ -429,7 +428,7 @@ public void flush() { * @return offsets that should be committed for this task */ @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { switch (state()) { case CREATED: case RESTORING: @@ -444,6 +443,10 @@ public Map prepareCommit() { // // TODO: this should be removed after we decouple caching with emitting flush(); + if (!clean) { + log.debug("Skipped preparing {} task with id {} for commit since the task is getting closed dirty.", state(), id); + return null; + } hasPendingTxCommit = eosEnabled; log.debug("Prepared {} task for committing", state()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 5cd747e3cf8ab..d5f67ff4fadd1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -39,7 +39,9 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.internals.KafkaFutureImpl; +import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; @@ -534,7 +536,7 @@ private static MainConsumerSetup setupMainConsumer(final TopologyMetadata topolo final Map consumerConfigs) { if (config.getString(StreamsConfig.GROUP_PROTOCOL_CONFIG).equalsIgnoreCase(GroupProtocol.STREAMS.name)) { if (topologyMetadata.hasNamedTopologies()) { - throw new IllegalStateException("Named topologies and the CONSUMER protocol cannot be used at the same time."); + throw new IllegalStateException("Named topologies and the STREAMS protocol cannot be used at the same time."); } log.info("Streams rebalance protocol enabled"); @@ -1079,8 +1081,9 @@ public void shutdownToError() { shutdownErrorHook.run(); } - public void sendShutdownRequest(final AssignorError assignorError) { - assignmentErrorCode.set(assignorError.code()); + public void sendShutdownRequest() { + assignmentErrorCode.set(AssignorError.SHUTDOWN_REQUESTED.code()); + streamsRebalanceData.ifPresent(StreamsRebalanceData::requestShutdown); } private void handleTaskMigrated(final TaskMigratedException e) { @@ -1486,9 +1489,10 @@ private ConsumerRecords pollRequests(final Duration pollTime) { public void handleStreamsRebalanceData() { if (streamsRebalanceData.isPresent()) { - - if (streamsRebalanceData.get().shutdownRequested()) { - assignmentErrorCode.set(AssignorError.SHUTDOWN_REQUESTED.code()); + for (final StreamsGroupHeartbeatResponseData.Status status : streamsRebalanceData.get().statuses()) { + if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.SHUTDOWN_APPLICATION.code()) { + shutdownErrorHook.run(); + } } } } @@ -2022,4 +2026,8 @@ Consumer restoreConsumer() { Admin adminClient() { return adminClient; } + + Optional streamsRebalanceData() { + return streamsRebalanceData; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java index 484c1ca574b6c..ba09700af8afb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java @@ -201,7 +201,7 @@ default boolean maybePunctuateSystemTime() { /** * @throws StreamsException fatal error, should close the thread */ - Map prepareCommit(); + Map prepareCommit(final boolean clean); void postCommit(boolean enforceCheckpoint); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutor.java index c993787503e19..91deab0dd9dab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskExecutor.java @@ -142,7 +142,7 @@ int commitTasksAndMaybeUpdateCommittableOffsets(final Collection tasksToCo for (final Task task : tasksToCommit) { // we need to call commitNeeded first since we need to update committable offsets if (task.commitNeeded()) { - final Map offsetAndMetadata = task.prepareCommit(); + final Map offsetAndMetadata = task.prepareCommit(true); if (!offsetAndMetadata.isEmpty()) { consumedOffsetsAndMetadata.put(task, offsetAndMetadata); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index eccf0c8f33d86..9376e6887f359 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -278,7 +278,7 @@ private void closeDirtyAndRevive(final Collection taskWithChangelogs, fina // we do not need to take the returned offsets since we are not going to commit anyways; // this call is only used for active tasks to flush the cache before suspending and // closing the topology - task.prepareCommit(); + task.prepareCommit(false); } catch (final RuntimeException swallow) { log.warn("Error flushing cache for corrupted task {}. " + "Since the task is closing dirty, the following exception is swallowed: {}", @@ -812,7 +812,7 @@ private Map closeAndRecycleTasks(final Map offsets = task.prepareCommit(); + final Map offsets = task.prepareCommit(true); if (!offsets.isEmpty()) { log.error("Task {} should have been committed when it was suspended, but it reports non-empty " + "offsets {} to commit; this means it failed during last commit and hence should be closed dirty", @@ -1264,7 +1264,7 @@ private void prepareCommitAndAddOffsetsToMap(final Set tasksToPrepare, final Map> consumedOffsetsPerTask) { for (final Task task : tasksToPrepare) { try { - final Map committableOffsets = task.prepareCommit(); + final Map committableOffsets = task.prepareCommit(true); if (!committableOffsets.isEmpty()) { consumedOffsetsPerTask.put(task, committableOffsets); } @@ -1479,7 +1479,7 @@ private void closeTaskDirty(final Task task, final boolean removeFromTasksRegist try { // we call this function only to flush the case if necessary // before suspending and closing the topology - task.prepareCommit(); + task.prepareCommit(false); } catch (final RuntimeException swallow) { log.warn("Error flushing cache of dirty task {}. " + "Since the task is closing dirty, the following exception is swallowed: {}", @@ -1630,7 +1630,7 @@ private Collection tryCloseCleanActiveTasks(final Collection activeT // first committing all tasks and then suspend and close them clean for (final Task task : activeTasksToClose) { try { - final Map committableOffsets = task.prepareCommit(); + final Map committableOffsets = task.prepareCommit(true); tasksToCommit.add(task); if (!committableOffsets.isEmpty()) { consumedOffsetsAndMetadataPerTask.put(task, committableOffsets); @@ -1719,7 +1719,7 @@ private Collection tryCloseCleanStandbyTasks(final Collection standb // first committing and then suspend / close clean for (final Task task : standbyTasksToClose) { try { - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); task.suspend(); closeTaskClean(task); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java new file mode 100644 index 0000000000000..66cb8e5185b15 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/DefaultStreamsRebalanceListenerTest.java @@ -0,0 +1,208 @@ +/* + * 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.streams.processor.internals; + +import org.apache.kafka.clients.consumer.internals.StreamsRebalanceData; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.processor.TaskId; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.mockito.InOrder; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.inOrder; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +public class DefaultStreamsRebalanceListenerTest { + private final TaskManager taskManager = mock(TaskManager.class); + private final StreamThread streamThread = mock(StreamThread.class); + private DefaultStreamsRebalanceListener defaultStreamsRebalanceListener = new DefaultStreamsRebalanceListener( + LoggerFactory.getLogger(DefaultStreamsRebalanceListener.class), + new MockTime(), + mock(StreamsRebalanceData.class), + streamThread, + taskManager + ); + + private void createRebalanceListenerWithRebalanceData(final StreamsRebalanceData streamsRebalanceData) { + defaultStreamsRebalanceListener = new DefaultStreamsRebalanceListener( + LoggerFactory.getLogger(DefaultStreamsRebalanceListener.class), + new MockTime(), + streamsRebalanceData, + streamThread, + taskManager + ); + } + + @ParameterizedTest + @EnumSource(StreamThread.State.class) + void testOnTasksRevoked(final StreamThread.State state) { + createRebalanceListenerWithRebalanceData(new StreamsRebalanceData( + UUID.randomUUID(), + Optional.empty(), + Map.of( + "1", + new StreamsRebalanceData.Subtopology( + Set.of("source1"), + Set.of(), + Map.of("repartition1", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), + Map.of(), + Set.of() + ) + ), + Map.of() + )); + when(streamThread.state()).thenReturn(state); + + final Optional result = defaultStreamsRebalanceListener.onTasksRevoked( + Set.of(new StreamsRebalanceData.TaskId("1", 0)) + ); + + assertTrue(result.isEmpty()); + + final InOrder inOrder = inOrder(taskManager, streamThread); + inOrder.verify(taskManager).handleRevocation( + Set.of(new TopicPartition("source1", 0), new TopicPartition("repartition1", 0)) + ); + inOrder.verify(streamThread).state(); + if (state != StreamThread.State.PENDING_SHUTDOWN) { + inOrder.verify(streamThread).setState(StreamThread.State.PARTITIONS_REVOKED); + } else { + inOrder.verify(streamThread, never()).setState(StreamThread.State.PARTITIONS_REVOKED); + } + } + + @Test + void testOnTasksRevokedWithException() { + final Exception exception = new RuntimeException("sample exception"); + doThrow(exception).when(taskManager).handleRevocation(any()); + + createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); + + final Optional result = defaultStreamsRebalanceListener.onTasksRevoked(Set.of()); + + assertTrue(result.isPresent()); + verify(taskManager).handleRevocation(any()); + verify(streamThread, never()).setState(any()); + } + + @Test + void testOnTasksAssigned() { + createRebalanceListenerWithRebalanceData(new StreamsRebalanceData( + UUID.randomUUID(), + Optional.empty(), + Map.of( + "1", + new StreamsRebalanceData.Subtopology( + Set.of("source1"), + Set.of(), + Map.of("repartition1", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), + Map.of(), + Set.of() + ), + "2", + new StreamsRebalanceData.Subtopology( + Set.of("source2"), + Set.of(), + Map.of("repartition2", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), + Map.of(), + Set.of() + ), + "3", + new StreamsRebalanceData.Subtopology( + Set.of("source3"), + Set.of(), + Map.of("repartition3", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of())), + Map.of(), + Set.of() + ) + ), + Map.of() + )); + + final Optional result = defaultStreamsRebalanceListener.onTasksAssigned( + new StreamsRebalanceData.Assignment( + Set.of(new StreamsRebalanceData.TaskId("1", 0)), + Set.of(new StreamsRebalanceData.TaskId("2", 0)), + Set.of(new StreamsRebalanceData.TaskId("3", 0)) + ) + ); + + assertTrue(result.isEmpty()); + + final InOrder inOrder = inOrder(taskManager, streamThread); + inOrder.verify(taskManager).handleAssignment( + Map.of(new TaskId(1, 0), Set.of(new TopicPartition("source1", 0), new TopicPartition("repartition1", 0))), + Map.of( + new TaskId(2, 0), Set.of(new TopicPartition("source2", 0), new TopicPartition("repartition2", 0)), + new TaskId(3, 0), Set.of(new TopicPartition("source3", 0), new TopicPartition("repartition3", 0)) + ) + ); + inOrder.verify(streamThread).setState(StreamThread.State.PARTITIONS_ASSIGNED); + inOrder.verify(taskManager).handleRebalanceComplete(); + } + + @Test + void testOnTasksAssignedWithException() { + final Exception exception = new RuntimeException("sample exception"); + doThrow(exception).when(taskManager).handleAssignment(any(), any()); + + createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); + final Optional result = defaultStreamsRebalanceListener.onTasksAssigned(new StreamsRebalanceData.Assignment(Set.of(), Set.of(), Set.of())); + assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty()); + assertTrue(result.isPresent()); + assertEquals(exception, result.get()); + verify(taskManager).handleLostAll(); + verify(streamThread, never()).setState(StreamThread.State.PARTITIONS_ASSIGNED); + verify(taskManager, never()).handleRebalanceComplete(); + } + + @Test + void testOnAllTasksLost() { + createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); + assertTrue(defaultStreamsRebalanceListener.onAllTasksLost().isEmpty()); + verify(taskManager).handleLostAll(); + } + + @Test + void testOnAllTasksLostWithException() { + final Exception exception = new RuntimeException("sample exception"); + doThrow(exception).when(taskManager).handleLostAll(); + + createRebalanceListenerWithRebalanceData(new StreamsRebalanceData(UUID.randomUUID(), Optional.empty(), Map.of(), Map.of())); + final Optional result = defaultStreamsRebalanceListener.onAllTasksLost(); + assertTrue(result.isPresent()); + assertEquals(exception, result.get()); + verify(taskManager).handleLostAll(); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index e953a61fc1f3e..768f3787d0b6d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -213,7 +213,7 @@ public void shouldThrowIfCommittingOnIllegalState() { task.suspend(); task.closeClean(); - assertThrows(IllegalStateException.class, task::prepareCommit); + assertThrows(IllegalStateException.class, () -> task.prepareCommit(true)); } @Test @@ -261,13 +261,13 @@ public void shouldFlushAndCheckpointStateManagerOnCommit() { task = createStandbyTask(); task.initializeIfNeeded(); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); // this should not checkpoint - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); // this should checkpoint - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); // this should not checkpoint verify(stateManager).checkpoint(); @@ -322,7 +322,7 @@ public void shouldSuspendAndCommitBeforeCloseClean() { task = createStandbyTask(); task.initializeIfNeeded(); task.suspend(); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); task.closeClean(); @@ -360,7 +360,7 @@ public void shouldOnlyNeedCommitWhenChangelogOffsetChanged() { // could commit if the offset advanced beyond threshold assertTrue(task.commitNeeded()); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); } @@ -389,7 +389,7 @@ public void shouldThrowOnCloseCleanCheckpointError() { task = createStandbyTask(); task.initializeIfNeeded(); - task.prepareCommit(); + task.prepareCommit(true); assertThrows(RuntimeException.class, () -> task.postCommit(true)); assertEquals(RUNNING, task.state()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index bcf24ee7df888..98807cd63423e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -645,6 +645,22 @@ public void shouldProcessInOrder() { assertEquals(asList(201, 202, 203), source2.values); } + @Test + public void shouldNotGetOffsetsIfPrepareCommitDirty() { + when(stateManager.taskId()).thenReturn(taskId); + when(stateManager.taskType()).thenReturn(TaskType.ACTIVE); + task = createStatefulTask(createConfig("100"), false); + + task.addRecords(partition1, List.of(getConsumerRecordWithOffsetAsTimestamp(partition1, 0))); + task.addRecords(partition2, List.of(getConsumerRecordWithOffsetAsTimestamp(partition2, 0))); + + assertTrue(task.process(0L)); + assertTrue(task.commitNeeded()); + + // committableOffsetsAndMetadata() has not been called, otherwise prepareCommit() would have returned a map + assertNull(task.prepareCommit(false)); + } + @Test public void shouldProcessRecordsAfterPrepareCommitWhenEosDisabled() { when(stateManager.taskId()).thenReturn(taskId); @@ -660,7 +676,7 @@ public void shouldProcessRecordsAfterPrepareCommitWhenEosDisabled() { )); assertTrue(task.process(time.milliseconds())); - task.prepareCommit(); + task.prepareCommit(true); assertTrue(task.process(time.milliseconds())); task.postCommit(false); assertTrue(task.process(time.milliseconds())); @@ -683,7 +699,7 @@ public void shouldNotProcessRecordsAfterPrepareCommitWhenEosV2Enabled() { )); assertTrue(task.process(time.milliseconds())); - task.prepareCommit(); + task.prepareCommit(true); assertFalse(task.process(time.milliseconds())); task.postCommit(false); assertTrue(task.process(time.milliseconds())); @@ -1328,7 +1344,7 @@ public void shouldRespectCommitNeeded() { assertTrue(task.process(0L)); assertTrue(task.commitNeeded()); - task.prepareCommit(); + task.prepareCommit(true); assertTrue(task.commitNeeded()); task.postCommit(true); @@ -1338,7 +1354,7 @@ public void shouldRespectCommitNeeded() { assertTrue(task.maybePunctuateStreamTime()); assertTrue(task.commitNeeded()); - task.prepareCommit(); + task.prepareCommit(true); assertTrue(task.commitNeeded()); task.postCommit(true); @@ -1349,7 +1365,7 @@ public void shouldRespectCommitNeeded() { assertTrue(task.maybePunctuateSystemTime()); assertTrue(task.commitNeeded()); - task.prepareCommit(); + task.prepareCommit(true); assertTrue(task.commitNeeded()); task.postCommit(true); @@ -1374,7 +1390,7 @@ public void shouldCommitNextOffsetAndProcessorMetadataFromQueueIfAvailable() { task.process(0L); processorSystemTime.mockProcessor.addProcessorMetadata("key2", 200L); - final Map offsetsAndMetadata = task.prepareCommit(); + final Map offsetsAndMetadata = task.prepareCommit(true); final TopicPartitionMetadata expected = new TopicPartitionMetadata(3L, new ProcessorMetadata( mkMap( @@ -1413,7 +1429,7 @@ public void shouldCommitFetchedNextOffsetIfRecordQueueIsEmpty() { final TopicPartitionMetadata metadata = new TopicPartitionMetadata(0, new ProcessorMetadata()); assertTrue(task.commitNeeded()); - assertThat(task.prepareCommit(), equalTo( + assertThat(task.prepareCommit(true), equalTo( mkMap( mkEntry(partition1, new OffsetAndMetadata(3L, Optional.of(2), metadata.encode())) ) @@ -1430,7 +1446,7 @@ public void shouldCommitFetchedNextOffsetIfRecordQueueIsEmpty() { task.process(0L); assertTrue(task.commitNeeded()); - assertThat(task.prepareCommit(), equalTo( + assertThat(task.prepareCommit(true), equalTo( mkMap( mkEntry(partition1, new OffsetAndMetadata(3L, Optional.of(2), metadata.encode())), mkEntry(partition2, new OffsetAndMetadata(1L, Optional.of(0), metadata.encode())) @@ -1486,7 +1502,7 @@ public void shouldCommitOldProcessorMetadataWhenNotDirty() { assertTrue(task.commitNeeded()); - assertThat(task.prepareCommit(), equalTo( + assertThat(task.prepareCommit(true), equalTo( mkMap( mkEntry(partition1, new OffsetAndMetadata(1L, Optional.of(1), expectedMetadata1.encode())), mkEntry(partition2, new OffsetAndMetadata(2L, Optional.of(1), expectedMetadata2.encode())) @@ -1509,7 +1525,7 @@ public void shouldCommitOldProcessorMetadataWhenNotDirty() { assertTrue(task.commitNeeded()); // Processor metadata not updated, we just need to commit to partition1 again with new offset - assertThat(task.prepareCommit(), equalTo( + assertThat(task.prepareCommit(true), equalTo( mkMap(mkEntry(partition1, new OffsetAndMetadata(2L, Optional.of(1), expectedMetadata3.encode()))) )); task.postCommit(false); @@ -1526,7 +1542,7 @@ public void shouldFailOnCommitIfTaskIsClosed() { final IllegalStateException thrown = assertThrows( IllegalStateException.class, - task::prepareCommit + () -> task.prepareCommit(true) ); assertThat(thrown.getMessage(), is("Illegal state CLOSED while preparing active task 0_0 for committing")); @@ -1820,10 +1836,10 @@ public void shouldNotCheckpointOffsetsAgainOnCommitIfSnapshotNotChangedMuch() { task.initializeIfNeeded(); task.completeRestoration(noOpResetter -> { }); // should checkpoint - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); // should checkpoint - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); // should not checkpoint assertThat("Map was empty", task.highWaterMark().size() == 2); @@ -1847,10 +1863,10 @@ public void shouldCheckpointOffsetsOnCommitIfSnapshotMuchChanged() { task.initializeIfNeeded(); task.completeRestoration(noOpResetter -> { }); // should checkpoint - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); // should checkpoint - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); // should checkpoint since the offset delta is greater than the threshold assertThat("Map was empty", task.highWaterMark().size() == 2); @@ -1866,7 +1882,7 @@ public void shouldNotCheckpointOffsetsOnCommitIfEosIsEnabled() { task.initializeIfNeeded(); task.completeRestoration(noOpResetter -> { }); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); final File checkpointFile = new File( stateDirectory.getOrCreateDirectoryForTask(taskId), @@ -2011,7 +2027,7 @@ public void shouldMaybeReturnOffsetsForRepartitionTopicsForPurging(final boolean assertTrue(task.process(0L)); assertTrue(task.process(0L)); - task.prepareCommit(); + task.prepareCommit(true); if (doCommit) { task.updateCommittedOffsets(repartition, 10L); } @@ -2050,7 +2066,7 @@ public void shouldThrowIfCommittingOnIllegalState() { task.transitionTo(SUSPENDED); task.transitionTo(Task.State.CLOSED); - assertThrows(IllegalStateException.class, task::prepareCommit); + assertThrows(IllegalStateException.class, () -> task.prepareCommit(true)); } @Test @@ -2101,7 +2117,7 @@ public void shouldNotCheckpointForSuspendedRunningTaskWithSmallProgress() { task.initializeIfNeeded(); task.completeRestoration(noOpResetter -> { }); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); task.suspend(); @@ -2123,7 +2139,7 @@ public void shouldCheckpointForSuspendedRunningTaskWithLargeProgress() { task.initializeIfNeeded(); task.completeRestoration(noOpResetter -> { }); // should checkpoint - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); // should checkpoint since the offset delta is greater than the threshold task.suspend(); @@ -2207,7 +2223,7 @@ public void shouldCheckpointOnCloseRestoringIfNoProgress() { task.initializeIfNeeded(); task.completeRestoration(noOpResetter -> { }); // should flush and checkpoint task.suspend(); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); // should flush and checkpoint task.closeClean(); @@ -2277,7 +2293,7 @@ public void shouldCheckpointOffsetsOnPostCommit() { assertTrue(task.commitNeeded()); task.suspend(); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(false); assertEquals(SUSPENDED, task.state()); @@ -2307,7 +2323,7 @@ public void shouldThrowExceptionOnCloseCleanError() { assertTrue(task.commitNeeded()); task.suspend(); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); // should checkpoint assertThrows(ProcessorStateException.class, () -> task.closeClean()); @@ -2336,7 +2352,7 @@ public void shouldThrowOnCloseCleanFlushError() { task.addRecords(partition1, singletonList(getConsumerRecordWithOffsetAsTimestamp(partition1, offset))); task.process(100L); - assertThrows(ProcessorStateException.class, task::prepareCommit); + assertThrows(ProcessorStateException.class, () -> task.prepareCommit(true)); assertEquals(RUNNING, task.state()); @@ -2369,7 +2385,7 @@ public void shouldThrowOnCloseCleanCheckpointError() { assertTrue(task.commitNeeded()); task.suspend(); - task.prepareCommit(); + task.prepareCommit(true); assertThrows(ProcessorStateException.class, () -> task.postCommit(true)); assertEquals(Task.State.SUSPENDED, task.state()); @@ -2672,7 +2688,7 @@ public void shouldUpdateOffsetIfAllRecordsHaveInvalidTimestamp() { assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(offset + 1, new TopicPartitionMetadata(RecordQueue.UNKNOWN, new ProcessorMetadata()).encode())))) @@ -2704,7 +2720,7 @@ public void shouldUpdateOffsetIfValidRecordFollowsInvalidTimestamp() { assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(offset + 1, new TopicPartitionMetadata(offset, new ProcessorMetadata()).encode())))) ); } @@ -2734,14 +2750,14 @@ public void shouldUpdateOffsetIfInvalidTimestampeRecordFollowsValid() { assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(1, new TopicPartitionMetadata(0, new ProcessorMetadata()).encode())))) ); assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(2, new TopicPartitionMetadata(0, new ProcessorMetadata()).encode())))) ); } @@ -2771,7 +2787,7 @@ public void shouldUpdateOffsetIfAllRecordsAreCorrupted() { assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(offset + 1, new TopicPartitionMetadata(RecordQueue.UNKNOWN, new ProcessorMetadata()).encode())))) @@ -2803,7 +2819,7 @@ public void shouldUpdateOffsetIfValidRecordFollowsCorrupted() { assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(offset + 1, new TopicPartitionMetadata(offset, new ProcessorMetadata()).encode())))) ); } @@ -2834,14 +2850,14 @@ public void shouldUpdateOffsetIfCorruptedRecordFollowsValid() { assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(1, new TopicPartitionMetadata(0, new ProcessorMetadata()).encode())))) ); assertTrue(task.process(offset)); assertTrue(task.commitNeeded()); assertThat( - task.prepareCommit(), + task.prepareCommit(true), equalTo(mkMap(mkEntry(partition1, new OffsetAndMetadata(2, new TopicPartitionMetadata(0, new ProcessorMetadata()).encode())))) ); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index 8ef4e09bb271b..35029519dfb4c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -25,8 +25,10 @@ import org.apache.kafka.clients.consumer.InvalidOffsetException; import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer; import org.apache.kafka.clients.consumer.internals.AutoOffsetResetStrategy; import org.apache.kafka.clients.consumer.internals.MockRebalanceListener; +import org.apache.kafka.clients.consumer.internals.StreamsRebalanceData; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.Cluster; @@ -37,6 +39,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; +import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.InvalidPidMappingException; import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.TimeoutException; @@ -55,9 +58,11 @@ import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.GroupProtocol; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsConfig.InternalConfig; import org.apache.kafka.streams.ThreadMetadata; +import org.apache.kafka.streams.TopologyConfig; import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; @@ -77,9 +82,11 @@ import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.StreamThread.State; +import org.apache.kafka.streams.processor.internals.assignment.AssignorError; import org.apache.kafka.streams.processor.internals.assignment.ReferenceContainer; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.tasks.DefaultTaskManager; +import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.Stores; @@ -96,6 +103,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -154,6 +162,7 @@ import static org.hamcrest.core.IsInstanceOf.instanceOf; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; @@ -3573,6 +3582,329 @@ public void shouldTimeOutOnProducerInstanceId(final boolean stateUpdaterEnabled, ); } + @Test + public void testNamedTopologyWithStreamsProtocol() { + final Properties props = configProps(false, false, false); + props.setProperty(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.toString()); + final StreamsConfig config = new StreamsConfig(props); + final InternalTopologyBuilder topologyBuilder = new InternalTopologyBuilder( + new TopologyConfig( + "my-topology", + config, + new Properties()) + ); + + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( + metrics, + APPLICATION_ID, + PROCESS_ID.toString(), + mockTime + ); + + final TopologyMetadata topologyMetadata = new TopologyMetadata(topologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); + + stateDirectory = new StateDirectory(config, mockTime, true, false); + final StreamsMetadataState streamsMetadataState = new StreamsMetadataState( + new TopologyMetadata(internalTopologyBuilder, config), + StreamsMetadataState.UNKNOWN_HOST, + new LogContext(String.format("stream-client [%s] ", CLIENT_ID)) + ); + final IllegalStateException exception = assertThrows(IllegalStateException.class, () -> + StreamThread.create( + topologyMetadata, + config, + clientSupplier, + clientSupplier.getAdmin(config.getAdminConfigs(CLIENT_ID)), + PROCESS_ID, + CLIENT_ID, + streamsMetrics, + mockTime, + streamsMetadataState, + 0, + stateDirectory, + new MockStateRestoreListener(), + new MockStandbyUpdateListener(), + threadIdx, + null, + HANDLER + ) + ); + assertEquals("Named topologies and the STREAMS protocol cannot be used at the same time.", exception.getMessage()); + } + + @Test + public void testStreamsRebalanceDataWithClassicProtocol() { + final Properties props = configProps(false, false, false); + props.setProperty(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.toString()); + thread = createStreamThread(CLIENT_ID, new StreamsConfig(props)); + assertTrue(thread.streamsRebalanceData().isEmpty()); + } + + @Test + public void testStreamsRebalanceDataWithExtraCopartition() { + final Properties props = configProps(false, false, false); + props.setProperty(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.toString()); + + internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); + + final StreamsConfig config = new StreamsConfig(props); + final InternalTopologyBuilder topologyBuilder = mock(InternalTopologyBuilder.class); + when(topologyBuilder.subtopologyToTopicsInfo()).thenReturn(Map.of()); + when(topologyBuilder.copartitionGroups()).thenReturn(Set.of(Set.of("source1"))); + + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( + metrics, + APPLICATION_ID, + PROCESS_ID.toString(), + mockTime + ); + + final TopologyMetadata topologyMetadata = new TopologyMetadata(topologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); + + stateDirectory = new StateDirectory(config, mockTime, true, false); + final StreamsMetadataState streamsMetadataState = new StreamsMetadataState( + new TopologyMetadata(internalTopologyBuilder, config), + StreamsMetadataState.UNKNOWN_HOST, + new LogContext(String.format("stream-client [%s] ", CLIENT_ID)) + ); + final IllegalStateException exception = assertThrows(IllegalStateException.class, () -> + StreamThread.create( + topologyMetadata, + config, + clientSupplier, + clientSupplier.getAdmin(config.getAdminConfigs(CLIENT_ID)), + PROCESS_ID, + CLIENT_ID, + streamsMetrics, + mockTime, + streamsMetadataState, + 0, + stateDirectory, + new MockStateRestoreListener(), + new MockStandbyUpdateListener(), + threadIdx, + null, + HANDLER + ) + ); + assertEquals("Not all copartition groups were converted to broker topology", exception.getMessage()); + } + + @Test + public void testStreamsRebalanceDataWithStreamsProtocol() { + final Properties props = configProps(false, false, false); + props.setProperty(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.toString()); + props.setProperty(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:1234"); + props.setProperty(StreamsConfig.REPLICATION_FACTOR_CONFIG, "1"); + + internalTopologyBuilder.addSource(null, "source1", null, null, null, topic1); + + final StreamsConfig config = new StreamsConfig(props); + final InternalTopologyBuilder topologyBuilder = mock(InternalTopologyBuilder.class); + when(topologyBuilder.subtopologyToTopicsInfo()).thenReturn(Map.of( + new TopologyMetadata.Subtopology(1, "subTopology1"), + new InternalTopologyBuilder.TopicsInfo( + Set.of("repartitionSource1"), + Set.of("source1"), + Map.of( + "repartitionSource1", + new RepartitionTopicConfig("repartitionSource1", Map.of(), 1, false) + ), + Map.of( + "stateChangeTopic1", + new RepartitionTopicConfig("stateChangeTopic1", Map.of(), 1, false) + ) + ) + )); + when(topologyBuilder.copartitionGroups()).thenReturn(Set.of(Set.of("source1"))); + + final StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl( + metrics, + APPLICATION_ID, + PROCESS_ID.toString(), + mockTime + ); + + final TopologyMetadata topologyMetadata = new TopologyMetadata(topologyBuilder, config); + topologyMetadata.buildAndRewriteTopology(); + + stateDirectory = new StateDirectory(config, mockTime, true, false); + final StreamsMetadataState streamsMetadataState = new StreamsMetadataState( + new TopologyMetadata(internalTopologyBuilder, config), + StreamsMetadataState.UNKNOWN_HOST, + new LogContext(String.format("stream-client [%s] ", CLIENT_ID)) + ); + thread = StreamThread.create( + topologyMetadata, + config, + clientSupplier, + clientSupplier.getAdmin(config.getAdminConfigs(CLIENT_ID)), + PROCESS_ID, + CLIENT_ID, + streamsMetrics, + mockTime, + streamsMetadataState, + 0, + stateDirectory, + new MockStateRestoreListener(), + new MockStandbyUpdateListener(), + threadIdx, + null, + HANDLER + ); + + assertInstanceOf(AsyncKafkaConsumer.class, thread.mainConsumer()); + assertTrue(thread.streamsRebalanceData().isPresent()); + assertEquals(PROCESS_ID, thread.streamsRebalanceData().get().processId()); + assertTrue(thread.streamsRebalanceData().get().endpoint().isPresent()); + assertEquals(new StreamsRebalanceData.HostInfo("localhost", 1234), + thread.streamsRebalanceData().get().endpoint().get()); + + final Map topicConfigs = Map.of( + TopicConfig.SEGMENT_BYTES_CONFIG, "52428800", + TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.CREATE_TIME.name, + TopicConfig.RETENTION_MS_CONFIG, "-1", + TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE + ); + assertEquals(1, thread.streamsRebalanceData().get().subtopologies().size()); + final StreamsRebalanceData.Subtopology subtopology = thread.streamsRebalanceData().get().subtopologies().get("1"); + + assertEquals(Set.of("source1"), subtopology.sourceTopics()); + assertEquals(Set.of("repartitionSource1"), subtopology.repartitionSinkTopics()); + + assertEquals(1, subtopology.repartitionSourceTopics().size()); + assertEquals(Optional.of(1), subtopology.repartitionSourceTopics().get("repartitionSource1").numPartitions()); + assertEquals(Optional.of((short) 1), subtopology.repartitionSourceTopics().get("repartitionSource1").replicationFactor()); + assertEquals(topicConfigs, subtopology.repartitionSourceTopics().get("repartitionSource1").topicConfigs()); + + assertEquals(1, subtopology.stateChangelogTopics().size()); + assertEquals(Optional.of(1), subtopology.stateChangelogTopics().get("stateChangeTopic1").numPartitions()); + assertEquals(Optional.of((short) 1), subtopology.stateChangelogTopics().get("stateChangeTopic1").replicationFactor()); + assertEquals(topicConfigs, subtopology.stateChangelogTopics().get("stateChangeTopic1").topicConfigs()); + + assertEquals(1, subtopology.copartitionGroups().size()); + assertEquals(Set.of("source1"), subtopology.copartitionGroups().stream().findFirst().get()); + } + + @Test + public void testStreamsProtocolRunOnceWithoutProcessingThreads() { + final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class); + when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty()); + when(mainConsumer.poll(Mockito.any(Duration.class))).thenReturn(new ConsumerRecords<>(Map.of(), Map.of())); + when(mainConsumer.groupMetadata()).thenReturn(consumerGroupMetadata); + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData( + UUID.randomUUID(), + Optional.empty(), + Map.of(), + Map.of() + ); + final AtomicInteger assignmentErrorCode = new AtomicInteger(0); + + final Properties props = configProps(false, false, false); + final StreamsConfig config = new StreamsConfig(props); + thread = new StreamThread( + new MockTime(1), + config, + null, + mainConsumer, + consumer, + changelogReader, + null, + mock(TaskManager.class), + null, + new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime), + new TopologyMetadata(internalTopologyBuilder, config), + PROCESS_ID, + CLIENT_ID, + new LogContext(""), + assignmentErrorCode, + new AtomicLong(Long.MAX_VALUE), + new LinkedList<>(), + null, + HANDLER, + null, + Optional.of(streamsRebalanceData), + null + ).updateThreadMetadata(adminClientId(CLIENT_ID)); + + thread.setState(State.STARTING); + thread.runOnceWithoutProcessingThreads(); + assertEquals(0, assignmentErrorCode.get()); + + streamsRebalanceData.requestShutdown(); + thread.runOnceWithoutProcessingThreads(); + assertEquals(AssignorError.SHUTDOWN_REQUESTED.code(), assignmentErrorCode.get()); + } + + @Test + public void testStreamsProtocolRunOnceWithProcessingThreads() { + final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class); + when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty()); + when(mainConsumer.poll(Mockito.any(Duration.class))).thenReturn(new ConsumerRecords<>(Map.of(), Map.of())); + when(mainConsumer.groupMetadata()).thenReturn(consumerGroupMetadata); + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData( + UUID.randomUUID(), + Optional.empty(), + Map.of(), + Map.of() + ); + final AtomicInteger assignmentErrorCode = new AtomicInteger(0); + + final Properties props = configProps(false, false, false); + final StreamsConfig config = new StreamsConfig(props); + thread = new StreamThread( + new MockTime(1), + config, + null, + mainConsumer, + consumer, + changelogReader, + null, + mock(TaskManager.class), + null, + new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime), + new TopologyMetadata(internalTopologyBuilder, config), + PROCESS_ID, + CLIENT_ID, + new LogContext(""), + assignmentErrorCode, + new AtomicLong(Long.MAX_VALUE), + new LinkedList<>(), + null, + HANDLER, + null, + Optional.of(streamsRebalanceData), + null + ).updateThreadMetadata(adminClientId(CLIENT_ID)); + + thread.setState(State.STARTING); + thread.runOnceWithProcessingThreads(); + assertEquals(0, assignmentErrorCode.get()); + + streamsRebalanceData.requestShutdown(); + thread.runOnceWithProcessingThreads(); + assertEquals(AssignorError.SHUTDOWN_REQUESTED.code(), assignmentErrorCode.get()); + } + + @Test + public void testGetTopicPartitionInfo() { + assertEquals( + Map.of( + t1p1, new PartitionInfo(t1p1.topic(), t1p1.partition(), null, new Node[0], new Node[0]), + t1p2, new PartitionInfo(t1p2.topic(), t1p2.partition(), null, new Node[0], new Node[0]), + t2p1, new PartitionInfo(t2p1.topic(), t2p1.partition(), null, new Node[0], new Node[0]) + ), + StreamThread.getTopicPartitionInfo( + Map.of( + new HostInfo("localhost", 9092), Set.of(t1p1, t2p1), + new HostInfo("localhost", 9094), Set.of(t1p2) + ) + ) + ); + } + private StreamThread setUpThread(final Properties streamsConfigProps) { final StreamsConfig config = new StreamsConfig(streamsConfigProps); final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 9d7df53adbe95..d8bb35c000a5f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -459,7 +459,7 @@ public void shouldRemoveUnusedFailedActiveTaskFromStateUpdaterAndCloseDirty() { taskManager.handleAssignment(Collections.emptyMap(), Collections.emptyMap()); - verify(activeTaskToClose).prepareCommit(); + verify(activeTaskToClose).prepareCommit(false); verify(activeTaskToClose).suspend(); verify(activeTaskToClose).closeDirty(); verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap()); @@ -500,7 +500,7 @@ public void shouldRemoveUnusedFailedStandbyTaskFromStateUpdaterAndCloseDirty() { taskManager.handleAssignment(Collections.emptyMap(), Collections.emptyMap()); - verify(standbyTaskToClose).prepareCommit(); + verify(standbyTaskToClose).prepareCommit(false); verify(standbyTaskToClose).suspend(); verify(standbyTaskToClose).closeDirty(); verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap()); @@ -996,7 +996,7 @@ public void shouldAddRecycledStandbyTasksFromActiveToPendingTasksToInitWithState taskManager.handleAssignment(emptyMap(), mkMap(mkEntry(taskId01, taskId01Partitions))); - verify(activeTaskToRecycle).prepareCommit(); + verify(activeTaskToRecycle).prepareCommit(true); verify(tasks).addPendingTasksToInit(Set.of(standbyTask)); verify(tasks).removeTask(activeTaskToRecycle); verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap()); @@ -1019,7 +1019,7 @@ public void shouldAddRecycledStandbyTasksFromActiveToTaskRegistryWithStateUpdate taskManager.handleAssignment(emptyMap(), mkMap(mkEntry(taskId01, taskId01Partitions))); - verify(activeTaskToRecycle).prepareCommit(); + verify(activeTaskToRecycle).prepareCommit(true); verify(tasks).replaceActiveWithStandby(standbyTask); verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap()); verify(standbyTaskCreator).createTasks(Collections.emptyMap()); @@ -1059,7 +1059,7 @@ public void shouldAssignActiveTaskInTasksRegistryToBeClosedCleanlyWithStateUpdat taskManager.handleAssignment(Collections.emptyMap(), Collections.emptyMap()); verify(activeTaskCreator).createTasks(consumer, Collections.emptyMap()); - verify(activeTaskToClose).prepareCommit(); + verify(activeTaskToClose).prepareCommit(true); verify(activeTaskToClose).closeClean(); verify(tasks).removeTask(activeTaskToClose); verify(standbyTaskCreator).createTasks(Collections.emptyMap()); @@ -1536,10 +1536,10 @@ public void shouldCloseDirtyWhenRemoveFailedActiveTasksFromStateUpdaterOnPartiti taskManager.handleLostAll(); - verify(task1).prepareCommit(); + verify(task1).prepareCommit(false); verify(task1).suspend(); verify(task1).closeDirty(); - verify(task2).prepareCommit(); + verify(task2).prepareCommit(false); verify(task2).suspend(); verify(task2).closeDirty(); } @@ -1569,7 +1569,7 @@ public void shouldCloseTasksWhenRemoveFailedActiveTasksFromStateUpdaterOnPartiti verify(task1).suspend(); verify(task1).closeClean(); - verify(task2).prepareCommit(); + verify(task2).prepareCommit(false); verify(task2).suspend(); verify(task2).closeDirty(); verify(task3).suspend(); @@ -2386,10 +2386,10 @@ public void shouldNotCommitNonCorruptedRestoringActiveTasksAndNotCommitRunningSt taskManager.handleCorruption(Set.of(taskId02)); verify(activeRestoringTask, never()).commitNeeded(); - verify(activeRestoringTask, never()).prepareCommit(); + verify(activeRestoringTask, never()).prepareCommit(true); verify(activeRestoringTask, never()).postCommit(anyBoolean()); verify(standbyTask, never()).commitNeeded(); - verify(standbyTask, never()).prepareCommit(); + verify(standbyTask, never()).prepareCommit(true); verify(standbyTask, never()).postCommit(anyBoolean()); } @@ -2418,9 +2418,9 @@ public void shouldNotCommitNonCorruptedRestoringActiveTasksAndCommitRunningStand taskManager.handleCorruption(Set.of(taskId02)); verify(activeRestoringTask, never()).commitNeeded(); - verify(activeRestoringTask, never()).prepareCommit(); + verify(activeRestoringTask, never()).prepareCommit(true); verify(activeRestoringTask, never()).postCommit(anyBoolean()); - verify(standbyTask).prepareCommit(); + verify(standbyTask).prepareCommit(true); verify(standbyTask).postCommit(anyBoolean()); } @@ -2431,7 +2431,7 @@ public void shouldCleanAndReviveCorruptedStandbyTasksBeforeCommittingNonCorrupte final StateMachineTask corruptedStandby = new StateMachineTask(taskId00, taskId00Partitions, false, stateManager); final StateMachineTask runningNonCorruptedActive = new StateMachineTask(taskId01, taskId01Partitions, true, stateManager) { @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { throw new TaskMigratedException("You dropped out of the group!", new RuntimeException()); } }; @@ -3394,7 +3394,7 @@ public void shouldOnlyCommitRevokedStandbyTaskAndPropagatePrepareCommitException final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager) { @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { throw new RuntimeException("task 0_1 prepare commit boom!"); } }; @@ -3560,7 +3560,7 @@ public void shouldShutDownStateUpdaterAndCloseFailedTasksDirty() { verify(activeTaskCreator).close(); verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE)); - verify(failedStatefulTask).prepareCommit(); + verify(failedStatefulTask).prepareCommit(false); verify(failedStatefulTask).suspend(); verify(failedStatefulTask).closeDirty(); } @@ -3634,16 +3634,16 @@ public void shouldShutDownStateUpdaterAndCloseDirtyTasksFailedDuringRemoval() { verify(stateUpdater).shutdown(Duration.ofMillis(Long.MAX_VALUE)); verify(tasks).addTask(removedStatefulTask); verify(tasks).addTask(removedStandbyTask); - verify(removedFailedStatefulTask).prepareCommit(); + verify(removedFailedStatefulTask).prepareCommit(false); verify(removedFailedStatefulTask).suspend(); verify(removedFailedStatefulTask).closeDirty(); - verify(removedFailedStandbyTask).prepareCommit(); + verify(removedFailedStandbyTask).prepareCommit(false); verify(removedFailedStandbyTask).suspend(); verify(removedFailedStandbyTask).closeDirty(); - verify(removedFailedStatefulTaskDuringRemoval).prepareCommit(); + verify(removedFailedStatefulTaskDuringRemoval).prepareCommit(false); verify(removedFailedStatefulTaskDuringRemoval).suspend(); verify(removedFailedStatefulTaskDuringRemoval).closeDirty(); - verify(removedFailedStandbyTaskDuringRemoval).prepareCommit(); + verify(removedFailedStandbyTaskDuringRemoval).prepareCommit(false); verify(removedFailedStandbyTaskDuringRemoval).suspend(); verify(removedFailedStandbyTaskDuringRemoval).closeDirty(); } @@ -3869,7 +3869,7 @@ public void shouldCommitViaProducerIfEosV2Enabled() { public void shouldPropagateExceptionFromActiveCommit() { final StateMachineTask task00 = new StateMachineTask(taskId00, taskId00Partitions, true, stateManager) { @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { throw new RuntimeException("opsh."); } }; @@ -3893,7 +3893,7 @@ public Map prepareCommit() { public void shouldPropagateExceptionFromStandbyCommit() { final StateMachineTask task01 = new StateMachineTask(taskId01, taskId01Partitions, false, stateManager) { @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { throw new RuntimeException("opsh."); } }; @@ -4689,7 +4689,7 @@ public void shouldConvertStandbyTaskToActiveTask() { final StandbyTask standbyTask = mock(StandbyTask.class); when(standbyTask.id()).thenReturn(taskId00); when(standbyTask.isActive()).thenReturn(false); - when(standbyTask.prepareCommit()).thenReturn(Collections.emptyMap()); + when(standbyTask.prepareCommit(true)).thenReturn(Collections.emptyMap()); final StreamTask activeTask = mock(StreamTask.class); when(activeTask.id()).thenReturn(taskId00); @@ -4939,10 +4939,13 @@ public boolean commitRequested() { } @Override - public Map prepareCommit() { + public Map prepareCommit(final boolean clean) { commitPrepared = true; if (commitNeeded) { + if (!clean) { + return null; + } return committableOffsets; } else { return Collections.emptyMap(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskExecutorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskExecutorTest.java index aac2dd36b49da..d43670429b135 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskExecutorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/tasks/DefaultTaskExecutorTest.java @@ -66,7 +66,7 @@ public void setUp() { when(task.isProcessable(anyLong())).thenReturn(true); when(task.id()).thenReturn(new TaskId(0, 0, "A")); when(task.process(anyLong())).thenReturn(true); - when(task.prepareCommit()).thenReturn(Collections.emptyMap()); + when(task.prepareCommit(true)).thenReturn(Collections.emptyMap()); } @AfterEach diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index a4cee67ad5fae..81c90d043cec4 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -591,7 +591,7 @@ private void completeAllProcessableWork() { // Process the record ... task.process(mockWallClockTime.milliseconds()); task.maybePunctuateStreamTime(); - commit(task.prepareCommit()); + commit(task.prepareCommit(true)); task.postCommit(true); captureOutputsAndReEnqueueInternalResults(); } @@ -709,7 +709,7 @@ public void advanceWallClockTime(final Duration advance) { mockWallClockTime.sleep(advance.toMillis()); if (task != null) { task.maybePunctuateSystemTime(); - commit(task.prepareCommit()); + commit(task.prepareCommit(true)); task.postCommit(true); } completeAllProcessableWork(); @@ -1130,7 +1130,7 @@ public SessionStore getSessionStore(final String name) { public void close() { if (task != null) { task.suspend(); - task.prepareCommit(); + task.prepareCommit(true); task.postCommit(true); task.closeClean(); } diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java index b026995ec70bc..21ab67f9bab42 100644 --- a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java @@ -115,6 +115,7 @@ public static class Builder { private final String controllerListenerName; private final String brokerSecurityProtocol; private final String controllerSecurityProtocol; + private boolean deleteOnClose; public Builder(TestKitNodes nodes) { this.nodes = nodes; @@ -122,6 +123,7 @@ public Builder(TestKitNodes nodes) { this.controllerListenerName = nodes.controllerListenerName().value(); this.brokerSecurityProtocol = nodes.brokerListenerProtocol().name; this.controllerSecurityProtocol = nodes.controllerListenerProtocol().name; + this.deleteOnClose = true; } public Builder setConfigProp(String key, Object value) { @@ -229,6 +231,11 @@ private Optional maybeSetupJaasFile() throws Exception { return Optional.empty(); } + public Builder setDeleteOnClose(boolean deleteOnClose) { + this.deleteOnClose = deleteOnClose; + return this; + } + public KafkaClusterTestKit build() throws Exception { Map controllers = new HashMap<>(); Map brokers = new HashMap<>(); @@ -316,7 +323,8 @@ public KafkaClusterTestKit build() throws Exception { baseDirectory, faultHandlerFactory, socketFactoryManager, - jaasFile); + jaasFile, + deleteOnClose); } private String listeners(int node) { @@ -361,6 +369,7 @@ private static void setupNodeDirectories(File baseDirectory, private final PreboundSocketFactoryManager socketFactoryManager; private final String controllerListenerName; private final Optional jaasFile; + private final boolean deleteOnClose; private KafkaClusterTestKit( TestKitNodes nodes, @@ -369,7 +378,8 @@ private KafkaClusterTestKit( File baseDirectory, SimpleFaultHandlerFactory faultHandlerFactory, PreboundSocketFactoryManager socketFactoryManager, - Optional jaasFile + Optional jaasFile, + boolean deleteOnClose ) { /* Number of threads = Total number of brokers + Total number of controllers + Total number of Raft Managers @@ -386,6 +396,7 @@ private KafkaClusterTestKit( this.socketFactoryManager = socketFactoryManager; this.controllerListenerName = nodes.controllerListenerName().value(); this.jaasFile = jaasFile; + this.deleteOnClose = deleteOnClose; } public void format() throws Exception { @@ -645,9 +656,11 @@ public void close() throws Exception { } waitForAllFutures(futureEntries); futureEntries.clear(); - Utils.delete(baseDirectory); - if (jaasFile.isPresent()) { - Utils.delete(jaasFile.get()); + if (deleteOnClose) { + Utils.delete(baseDirectory); + if (jaasFile.isPresent()) { + Utils.delete(jaasFile.get()); + } } } catch (Exception e) { for (Entry> entry : futureEntries) { diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index 049af5a10a9a7..1d0e3a3bd5e00 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -77,7 +77,6 @@ GROUP_COORDINATOR_REBALANCE_PROTOCOLS="group.coordinator.rebalance.protocols" CONSUMER_GROUP_MIGRATION_POLICY = "group.consumer.migration.policy" -UNSTABLE_API_VERSIONS_ENABLE = "unstable.api.versions.enable" SHARE_COORDINATOR_STATE_TOPIC_REPLICATION_FACTOR ="share.coordinator.state.topic.replication.factor" SHARE_COORDINATOR_STATE_TOPIC_MIN_ISR = "share.coordinator.state.topic.min.isr" SHARE_GROUP_ENABLE = "group.share.enable" diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index a55844548ba0a..da62a0da3261d 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -778,7 +778,6 @@ def prop_file(self, node): if self.use_share_groups is not None and self.use_share_groups is True: override_configs[config_property.SHARE_GROUP_ENABLE] = str(self.use_share_groups) - override_configs[config_property.UNSTABLE_API_VERSIONS_ENABLE] = str(self.use_share_groups) override_configs[config_property.GROUP_COORDINATOR_REBALANCE_PROTOCOLS] = 'classic,consumer,share' #update template configs with test override configs diff --git a/tests/kafkatest/tests/client/share_consumer_test.py b/tests/kafkatest/tests/client/share_consumer_test.py index 39dff8c519f2d..f47ac1e771b42 100644 --- a/tests/kafkatest/tests/client/share_consumer_test.py +++ b/tests/kafkatest/tests/client/share_consumer_test.py @@ -168,11 +168,6 @@ def test_share_multiple_partitions(self, metadata_quorum=quorum.isolated_kraft, for event_handler in consumer.event_handlers.values(): assert event_handler.total_consumed > 0 assert event_handler.total_acknowledged_successfully > 0 - for topic_partition in self.get_topic_partitions(self.TOPIC2): - assert topic_partition in event_handler.consumed_per_partition - assert event_handler.consumed_per_partition[topic_partition] > 0 - assert topic_partition in event_handler.acknowledged_per_partition - assert event_handler.acknowledged_per_partition[topic_partition] > 0 producer.stop() consumer.stop_all() diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ShareGroupMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ShareGroupMessageFormatterTest.java index 487e03038fad9..96e8681c00d61 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ShareGroupMessageFormatterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ShareGroupMessageFormatterTest.java @@ -60,7 +60,8 @@ public class ShareGroupMessageFormatterTest extends CoordinatorRecordMessageForm private static final ShareGroupMetadataKey SHARE_GROUP_METADATA_KEY = new ShareGroupMetadataKey() .setGroupId("group-id"); private static final ShareGroupMetadataValue SHARE_GROUP_METADATA_VALUE = new ShareGroupMetadataValue() - .setEpoch(1); + .setEpoch(1) + .setMetadataHash(1); private static final ShareGroupTargetAssignmentMetadataKey SHARE_GROUP_TARGET_ASSIGNMENT_METADATA_KEY = new ShareGroupTargetAssignmentMetadataKey() .setGroupId("group-id"); private static final ShareGroupTargetAssignmentMetadataValue SHARE_GROUP_TARGET_ASSIGNMENT_METADATA_VALUE = new ShareGroupTargetAssignmentMetadataValue() @@ -154,7 +155,8 @@ protected Stream parameters() { """ {"key":{"type":11,"data":{"groupId":"group-id"}}, "value":{"version":0, - "data":{"epoch":1}}} + "data":{"epoch":1, + "metadataHash":1}}} """ ), Arguments.of(