- * 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:
*
* 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,
* Gets the internal producer id and epoch, used in all future transactional
* messages issued by the producer.
*
+ *
+ *
+ * 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 extends Connector> 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 extends Transformation> 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 extends Connector> connectorClass
+ Class extends Connector> connectorClass,
+ TaskPluginsMetadata pluginsMetadata
);
}
@@ -1884,7 +1905,8 @@ public WorkerTask, SinkRecord> doBuild(
RetryWithToleranceOperator> retryWithToleranceOperator,
TransformationChain, SinkRecord> transformationChain,
ErrorHandlingMetrics errorHandlingMetrics,
- Class extends Connector> connectorClass
+ Class extends Connector> 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