From 38f6cae9e879baa35c5dbc5829bf09ecd59930c2 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 24 May 2017 15:26:46 -0700 Subject: [PATCH] KAFKA-5259; TransactionalId auth implies ProducerId auth Author: Jason Gustafson Reviewers: Apurva Mehta , Jun Rao Closes #3075 from hachikuji/KAFKA-5259-FIXED --- .../apache/kafka/clients/ClientResponse.java | 7 +- .../kafka/clients/admin/AclOperation.java | 7 +- .../clients/consumer/internals/Fetcher.java | 2 +- .../kafka/clients/producer/KafkaProducer.java | 35 +- .../clients/producer/internals/Sender.java | 169 +++---- .../internals/TransactionManager.java | 268 ++++++----- .../ProducerIdAuthorizationException.java | 23 - ...TransactionalIdAuthorizationException.java | 2 +- .../apache/kafka/common/protocol/Errors.java | 13 +- .../kafka/common/protocol/Protocol.java | 3 + .../requests/AddOffsetsToTxnRequest.java | 11 + .../requests/AddOffsetsToTxnResponse.java | 5 +- .../requests/AddPartitionsToTxnRequest.java | 11 + .../requests/AddPartitionsToTxnResponse.java | 3 +- .../kafka/common/requests/EndTxnRequest.java | 11 + .../kafka/common/requests/EndTxnResponse.java | 3 +- .../requests/InitProducerIdResponse.java | 12 +- .../kafka/common/requests/ProduceRequest.java | 5 +- .../common/requests/ProduceResponse.java | 3 + .../requests/TxnOffsetCommitRequest.java | 31 +- .../requests/TxnOffsetCommitResponse.java | 1 + .../requests/WriteTxnMarkersResponse.java | 1 + .../kafka/clients/admin/AclOperationTest.java | 3 +- .../producer/internals/SenderTest.java | 183 +++---- .../internals/TransactionManagerTest.java | 445 ++++++++++++------ .../common/requests/RequestResponseTest.java | 10 +- .../main/scala/kafka/admin/AclCommand.scala | 54 ++- .../coordinator/group/GroupMetadata.scala | 43 +- .../group/GroupMetadataManager.scala | 5 +- .../coordinator/group/MemberMetadata.scala | 14 +- .../scala/kafka/security/auth/Operation.scala | 6 +- .../scala/kafka/security/auth/Resource.scala | 3 +- .../kafka/security/auth/ResourceType.scala | 16 +- .../main/scala/kafka/server/KafkaApis.scala | 316 +++++++------ .../kafka/api/AuthorizerIntegrationTest.scala | 292 +++++++++--- .../kafka/api/TransactionsBounceTest.scala | 31 +- .../kafka/api/TransactionsTest.scala | 7 +- .../unit/kafka/admin/AclCommandTest.scala | 36 +- ...eTest.scala => GroupCoordinatorTest.scala} | 2 +- .../coordinator/group/GroupMetadataTest.scala | 2 +- .../unit/kafka/server/RequestQuotaTest.scala | 9 +- .../scala/unit/kafka/utils/TestUtils.scala | 10 +- 42 files changed, 1280 insertions(+), 833 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java rename core/src/test/scala/unit/kafka/coordinator/group/{GroupCoordinatorResponseTest.scala => GroupCoordinatorTest.scala} (99%) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java index 715eae7f4f4d..0ff30e93515e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.clients; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.requests.AbstractResponse; import org.apache.kafka.common.requests.RequestHeader; @@ -31,7 +32,7 @@ public class ClientResponse { private final long receivedTimeMs; private final long latencyMs; private final boolean disconnected; - private final RuntimeException versionMismatch; + private final UnsupportedVersionException versionMismatch; private final AbstractResponse responseBody; /** @@ -51,7 +52,7 @@ public ClientResponse(RequestHeader requestHeader, long createdTimeMs, long receivedTimeMs, boolean disconnected, - RuntimeException versionMismatch, + UnsupportedVersionException versionMismatch, AbstractResponse responseBody) { this.requestHeader = requestHeader; this.callback = callback; @@ -71,7 +72,7 @@ public boolean wasDisconnected() { return disconnected; } - public RuntimeException versionMismatch() { + public UnsupportedVersionException versionMismatch() { return versionMismatch; } diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java b/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java index 062e5e31f110..0c3ff50d0513 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java @@ -83,7 +83,12 @@ public enum AclOperation { /** * ALTER_CONFIGS operation. */ - ALTER_CONFIGS((byte) 11); + ALTER_CONFIGS((byte) 11), + + /** + * IDEMPOTENT_WRITE operation. + */ + IDEMPOTENT_WRITE((byte) 12); private final static HashMap CODE_TO_VALUE = new HashMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 509993fad2d9..6917a1d437fb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -416,7 +416,7 @@ private void resetOffsets(final Set partitions) { } // we might lose the assignment while fetching the offset, so check it is still active if (subscriptions.isAssigned(partition)) { - log.debug("Resetting offset for partition {} to {} offset.", partition, offsetData.offset); + log.debug("Resetting offset for partition {} to offset {}.", partition, offsetData.offset); this.subscriptions.seek(partition, offsetData.offset); } } 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 ac0169ae1947..c11ecc78054d 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 @@ -51,7 +51,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.network.ChannelBuilder; import org.apache.kafka.common.network.Selector; -import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.RecordBatch; @@ -607,7 +606,9 @@ public Future send(ProducerRecord record, Callback callbac * Implementation of asynchronously send a record to a topic. */ private Future doSend(ProducerRecord record, Callback callback) { - ensureProperTransactionalState(); + if (transactionManager != null) + ensureProperTransactionalState(); + TopicPartition tp = null; try { // first make sure the metadata for the topic is available @@ -642,9 +643,9 @@ private Future doSend(ProducerRecord record, Callback call long timestamp = record.timestamp() == null ? time.milliseconds() : record.timestamp(); log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition); // producer callback will make sure to call both 'callback' and interceptor callback - Callback interceptCallback = new InterceptorCallback<>(callback, this.interceptors, tp, transactionManager); + Callback interceptCallback = this.interceptors == null ? callback : new InterceptorCallback<>(callback, this.interceptors, tp); - if (transactionManager != null) + if (transactionManager != null && transactionManager.isTransactional()) transactionManager.maybeAddPartitionToTransaction(tp); RecordAccumulator.RecordAppendResult result = accumulator.append(tp, timestamp, serializedKey, @@ -690,27 +691,17 @@ private Future doSend(ProducerRecord record, Callback call } private void ensureProperTransactionalState() { - if (transactionManager == null) - return; - if (transactionManager.isTransactional() && !transactionManager.hasProducerId()) - throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions when transactions are enabled."); - - if (transactionManager.isFenced()) - throw Errors.INVALID_PRODUCER_EPOCH.exception(); + throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions " + + "when transactions are enabled."); if (transactionManager.isInErrorState()) { - String errorMessage = - "Cannot perform send because at least one previous transactional or idempotent request has failed with errors."; Exception lastError = transactionManager.lastError(); - if (lastError != null) - throw new KafkaException(errorMessage, lastError); - else - throw new KafkaException(errorMessage); + throw new KafkaException("Cannot perform send because at least one previous transactional or " + + "idempotent request has failed with errors.", lastError); } if (transactionManager.isCompletingTransaction()) throw new IllegalStateException("Cannot call send while a commit or abort is in progress."); - } private void setReadOnly(Headers headers) { @@ -1013,14 +1004,11 @@ private static class InterceptorCallback implements Callback { private final Callback userCallback; private final ProducerInterceptors interceptors; private final TopicPartition tp; - private final TransactionManager transactionManager; - public InterceptorCallback(Callback userCallback, ProducerInterceptors interceptors, - TopicPartition tp, TransactionManager transactionManager) { + private InterceptorCallback(Callback userCallback, ProducerInterceptors interceptors, TopicPartition tp) { this.userCallback = userCallback; this.interceptors = interceptors; this.tp = tp; - this.transactionManager = transactionManager; } public void onCompletion(RecordMetadata metadata, Exception exception) { @@ -1034,9 +1022,6 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { } if (this.userCallback != null) this.userCallback.onCompletion(metadata, exception); - - if (exception != null && transactionManager != null) - transactionManager.setError(exception); } } } 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 4c3b99d15e87..116a1c52b546 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 @@ -24,15 +24,18 @@ import org.apache.kafka.clients.NetworkClientUtils; import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.Cluster; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.errors.OutOfOrderSequenceException; +import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -189,28 +192,34 @@ public void run() { * @param now The current POSIX time in milliseconds */ void run(long now) { - long pollTimeout = retryBackoffMs; - if (!maybeSendTransactionalRequest(now)) { - pollTimeout = sendProducerData(now); + if (transactionManager != null) { + if (!transactionManager.isTransactional()) { + // this is an idempotent producer, so make sure we have a producer id + maybeWaitForProducerId(); + } else if (transactionManager.hasInflightRequest() || maybeSendTransactionalRequest(now)) { + // as long as there are outstanding transactional requests, we simply wait for them to return + client.poll(retryBackoffMs, now); + return; + } + + // do not continue sending if the transaction manager is in a failed state or if there + // is no producer id (for the idempotent case). + if (transactionManager.isInErrorState() || !transactionManager.hasProducerId()) { + RuntimeException lastError = transactionManager.lastError(); + if (lastError != null) + maybeAbortBatches(lastError); + client.poll(retryBackoffMs, now); + return; + } } + long pollTimeout = sendProducerData(now); log.trace("waiting {}ms in poll", pollTimeout); - this.client.poll(pollTimeout, now); + client.poll(pollTimeout, now); } - private long sendProducerData(long now) { Cluster cluster = metadata.fetch(); - maybeWaitForProducerId(); - - if (transactionManager != null && transactionManager.isInErrorState()) { - final KafkaException exception = transactionManager.lastError() instanceof KafkaException - ? (KafkaException) transactionManager.lastError() - : new KafkaException(transactionManager.lastError()); - log.error("aborting producer batches because the transaction manager is in an error state.", exception); - this.accumulator.abortBatches(exception); - return Long.MAX_VALUE; - } // get the list of partitions with data ready to send RecordAccumulator.ReadyCheckResult result = this.accumulator.ready(cluster, now); @@ -286,22 +295,13 @@ private long sendProducerData(long now) { } private boolean maybeSendTransactionalRequest(long now) { - if (transactionManager == null || !transactionManager.isTransactional()) - return false; - - if (transactionManager.hasInflightRequest()) { - log.trace("TransactionalId: {} -- There is already an inflight transactional request. Going to wait for the response.", + TransactionManager.TxnRequestHandler nextRequestHandler = transactionManager.nextRequestHandler(); + if (nextRequestHandler == null) { + log.trace("TransactionalId: {} -- There are no pending transactional requests to send", transactionManager.transactionalId()); - return true; - } - - if (!transactionManager.hasPendingTransactionalRequests()) { - log.trace("TransactionalId: {} -- There are no pending transactional requests to send", transactionManager.transactionalId()); return false; } - TransactionManager.TxnRequestHandler nextRequestHandler = transactionManager.nextRequestHandler(); - if (nextRequestHandler.isEndTxn() && transactionManager.isCompletingTransaction() && accumulator.hasUnflushedBatches()) { if (!accumulator.flushInProgress()) accumulator.beginFlush(); @@ -311,15 +311,11 @@ private boolean maybeSendTransactionalRequest(long now) { return false; } - if (transactionManager.maybeTerminateRequestWithError(nextRequestHandler)) { - log.trace("TransactionalId: {} -- Not sending a transactional request because we are in an error state", - transactionManager.transactionalId()); - return false; - } - - Node targetNode = null; + log.debug("TransactionalId: {} -- Sending transactional request {}", transactionManager.transactionalId(), + nextRequestHandler.requestBuilder()); - while (targetNode == null) { + while (true) { + Node targetNode = null; try { if (nextRequestHandler.needsCoordinator()) { targetNode = transactionManager.coordinator(nextRequestHandler.coordinatorType()); @@ -340,8 +336,8 @@ private boolean maybeSendTransactionalRequest(long now) { transactionManager.transactionalId(), retryBackoffMs, nextRequestHandler.requestBuilder()); time.sleep(retryBackoffMs); } - ClientRequest clientRequest = client.newClientRequest(targetNode.idString(), nextRequestHandler.requestBuilder(), - now, true, nextRequestHandler); + ClientRequest clientRequest = client.newClientRequest(targetNode.idString(), + nextRequestHandler.requestBuilder(), now, true, nextRequestHandler); transactionManager.setInFlightRequestCorrelationId(clientRequest.correlationId()); log.trace("TransactionalId: {} -- Sending transactional request {} to node {}", transactionManager.transactionalId(), nextRequestHandler.requestBuilder(), clientRequest.destination()); @@ -349,9 +345,9 @@ private boolean maybeSendTransactionalRequest(long now) { return true; } } catch (IOException e) { - targetNode = null; - log.warn("TransactionalId: " + transactionManager.transactionalId() + " -- Got an exception when trying " + - "to find a node to send transactional request " + nextRequestHandler.requestBuilder() + ". Going to back off and retry", e); + log.debug("TransactionalId: {} -- Disconnect from {} while trying to send transactional " + + "request {}. Going to back off and retry", transactionManager.transactionalId(), + targetNode, nextRequestHandler.requestBuilder()); } log.trace("TransactionalId: {}. About to wait for {}ms before trying to send another transactional request.", transactionManager.transactionalId(), retryBackoffMs); @@ -364,6 +360,13 @@ private boolean maybeSendTransactionalRequest(long now) { return true; } + private void maybeAbortBatches(RuntimeException exception) { + if (accumulator.hasUnflushedBatches()) { + log.error("Aborting producer batches due to fatal error", exception); + accumulator.abortBatches(exception); + } + } + /** * Start closing the sender (won't actually complete until all data is sent out) */ @@ -383,7 +386,7 @@ public void forceClose() { initiateClose(); } - private ClientResponse sendAndAwaitInitPidRequest(Node node) throws IOException { + private ClientResponse sendAndAwaitInitProducerIdRequest(Node node) throws IOException { String nodeId = node.idString(); InitProducerIdRequest.Builder builder = new InitProducerIdRequest.Builder(null); ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, null); @@ -399,43 +402,37 @@ private Node awaitLeastLoadedNodeReady(long remainingTimeMs) throws IOException } private void maybeWaitForProducerId() { - // If this is a transactional producer, the producer id will be received when recovering transactions in the - // initTransactions() method of the producer. - if (transactionManager == null || transactionManager.isTransactional()) - return; - while (!transactionManager.hasProducerId() && !transactionManager.isInErrorState()) { try { Node node = awaitLeastLoadedNodeReady(requestTimeout); if (node != null) { - ClientResponse response = sendAndAwaitInitPidRequest(node); - - if (response.hasResponse() && (response.responseBody() instanceof InitProducerIdResponse)) { - InitProducerIdResponse initProducerIdResponse = (InitProducerIdResponse) response.responseBody(); - Exception exception = initProducerIdResponse.error().exception(); - if (exception != null && !(exception instanceof RetriableException)) { - transactionManager.setError(exception); - return; - } + ClientResponse response = sendAndAwaitInitProducerIdRequest(node); + InitProducerIdResponse initProducerIdResponse = (InitProducerIdResponse) response.responseBody(); + Errors error = initProducerIdResponse.error(); + if (error == Errors.NONE) { ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch( initProducerIdResponse.producerId(), initProducerIdResponse.epoch()); transactionManager.setProducerIdAndEpoch(producerIdAndEpoch); + } else if (error.exception() instanceof RetriableException) { + log.debug("Retriable error from InitProducerId response", error.message()); } else { - log.error("Received an unexpected response type for an InitProducerIdRequest from {}. " + - "We will back off and try again.", node); + transactionManager.transitionToFatalError(error.exception()); + break; } } else { log.debug("Could not find an available broker to send InitProducerIdRequest to. " + "We will back off and try again."); } - } catch (Exception e) { - log.warn("Received an exception while trying to get a producer id. Will back off and retry.", e); + } catch (UnsupportedVersionException e) { + transactionManager.transitionToFatalError(e); + break; + } catch (IOException e) { + log.debug("Broker {} disconnected while awaiting InitProducerId response", e); } log.trace("Retry InitProducerIdRequest in {}ms.", retryBackoffMs); time.sleep(retryBackoffMs); metadata.requestUpdate(); } - } /** @@ -507,9 +504,9 @@ private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionRespons error); if (transactionManager == null) { reenqueueBatch(batch, now); - } else if (transactionManager.producerIdAndEpoch().producerId == batch.producerId() && - transactionManager.producerIdAndEpoch().epoch == batch.producerEpoch()) { - // If idempotence is enabled only retry the request if the current producer id is the same as the producer id of the batch. + } else if (transactionManager.hasProducerIdAndEpoch(batch.producerId(), batch.producerEpoch())) { + // If idempotence is enabled only retry the request if the current producer id is the same as + // the producer id of the batch. log.debug("Retrying batch to topic-partition {}. Sequence number : {}", batch.topicPartition, transactionManager.sequenceNumber(batch.topicPartition)); reenqueueBatch(batch, now); @@ -523,12 +520,10 @@ private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionRespons final RuntimeException exception; if (error == Errors.TOPIC_AUTHORIZATION_FAILED) exception = new TopicAuthorizationException(batch.topicPartition.topic()); + else if (error == Errors.CLUSTER_AUTHORIZATION_FAILED) + exception = new ClusterAuthorizationException("The producer is not authorized to do idempotent sends"); else exception = error.exception(); - if (error == Errors.OUT_OF_ORDER_SEQUENCE_NUMBER && batch.producerId() == transactionManager.producerIdAndEpoch().producerId) - log.error("The broker received an out of order sequence number for correlation id {}, topic-partition " + - "{} at offset {}. This indicates data loss on the broker, and should be investigated.", - correlationId, batch.topicPartition, response.baseOffset); // tell the user the result of their request failBatch(batch, response, exception); this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount); @@ -543,12 +538,6 @@ private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionRespons } else { completeBatch(batch, response); - if (transactionManager != null && transactionManager.producerIdAndEpoch().producerId == batch.producerId() - && transactionManager.producerIdAndEpoch().epoch == batch.producerEpoch()) { - transactionManager.incrementSequenceNumber(batch.topicPartition, batch.recordCount); - log.debug("Incremented sequence number for topic-partition {} to {}", batch.topicPartition, - transactionManager.sequenceNumber(batch.topicPartition)); - } } // Unmute the completed partition. @@ -562,18 +551,38 @@ private void reenqueueBatch(ProducerBatch batch, long currentTimeMs) { } private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionResponse response) { + if (transactionManager != null && transactionManager.hasProducerIdAndEpoch(batch.producerId(), batch.producerEpoch())) { + transactionManager.incrementSequenceNumber(batch.topicPartition, batch.recordCount); + log.debug("Incremented sequence number for topic-partition {} to {}", batch.topicPartition, + transactionManager.sequenceNumber(batch.topicPartition)); + } + batch.done(response.baseOffset, response.logAppendTime, null); this.accumulator.deallocate(batch); } private void failBatch(ProducerBatch batch, ProduceResponse.PartitionResponse response, RuntimeException exception) { - if (transactionManager != null && !transactionManager.isTransactional() - && batch.producerId() == transactionManager.producerIdAndEpoch().producerId) { - // Reset the transaction state since we have hit an irrecoverable exception and cannot make any guarantees - // about the previously committed message. Note that this will discard the producer id and sequence - // numbers for all existing partitions. - transactionManager.resetProducerId(); + if (transactionManager != null) { + if (exception instanceof OutOfOrderSequenceException + && !transactionManager.isTransactional() + && transactionManager.hasProducerId(batch.producerId())) { + log.error("The broker received an out of order sequence number for topic-partition " + + "{} at offset {}. This indicates data loss on the broker, and should be investigated.", + batch.topicPartition, response.baseOffset); + + // Reset the transaction state since we have hit an irrecoverable exception and cannot make any guarantees + // about the previously committed message. Note that this will discard the producer id and sequence + // numbers for all existing partitions. + transactionManager.resetProducerId(); + } else if (exception instanceof ClusterAuthorizationException + || exception instanceof TransactionalIdAuthorizationException + || exception instanceof ProducerFencedException) { + transactionManager.transitionToFatalError(exception); + } else if (transactionManager.isTransactional()) { + transactionManager.transitionToAbortableError(exception); + } } + batch.done(response.baseOffset, response.logAppendTime, exception); this.accumulator.deallocate(batch); } 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 d84a88e6e1bd..d674697964c2 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 @@ -23,7 +23,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.GroupAuthorizationException; -import org.apache.kafka.common.errors.ProducerFencedException; +import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; @@ -77,7 +77,7 @@ public class TransactionManager { private Node consumerGroupCoordinator; private volatile State currentState = State.UNINITIALIZED; - private volatile Exception lastError = null; + private volatile RuntimeException lastError = null; private volatile ProducerIdAndEpoch producerIdAndEpoch; private enum State { @@ -87,32 +87,34 @@ private enum State { IN_TRANSACTION, COMMITTING_TRANSACTION, ABORTING_TRANSACTION, - FENCED, - ERROR; + ABORTABLE_ERROR, + FATAL_ERROR; private boolean isTransitionValid(State source, State target) { switch (target) { case INITIALIZING: - return source == UNINITIALIZED || source == ERROR; + return source == UNINITIALIZED; case READY: return source == INITIALIZING || source == COMMITTING_TRANSACTION - || source == ABORTING_TRANSACTION || source == ERROR; + || source == ABORTING_TRANSACTION || source == ABORTABLE_ERROR; case IN_TRANSACTION: return source == READY; case COMMITTING_TRANSACTION: return source == IN_TRANSACTION; case ABORTING_TRANSACTION: - return source == IN_TRANSACTION || source == ERROR; + return source == IN_TRANSACTION || source == ABORTABLE_ERROR; + case ABORTABLE_ERROR: + return source == IN_TRANSACTION || source == COMMITTING_TRANSACTION || source == ABORTING_TRANSACTION; + case FATAL_ERROR: default: - // We can transition to FENCED or ERROR unconditionally. - // FENCED is never a valid starting state for any transition. So the only option is to close the + // We can transition to FATAL_ERROR unconditionally. + // FATAL_ERROR is never a valid starting state for any transition. So the only option is to close the // producer or do purely non transactional requests. return true; } } } - // We use the priority to determine the order in which requests need to be sent out. For instance, if we have // a pending FindCoordinator request, that must always go first. Next, If we need a producer id, that must go second. // The endTxn request must always go last. @@ -149,7 +151,7 @@ public int compare(TxnRequestHandler o1, TxnRequestHandler o2) { } TransactionManager() { - this("", 0); + this(null, 0); } public synchronized TransactionalRequestResult initializeTransactions() { @@ -178,8 +180,8 @@ public synchronized TransactionalRequestResult beginCommittingTransaction() { public synchronized TransactionalRequestResult beginAbortingTransaction() { ensureTransactional(); - if (isFenced()) - throw Errors.INVALID_PRODUCER_EPOCH.exception(); + if (currentState != State.ABORTABLE_ERROR) + maybeFailWithError(); transitionTo(State.ABORTING_TRANSACTION); return beginCompletingTransaction(false); } @@ -213,12 +215,16 @@ public synchronized TransactionalRequestResult sendOffsetsToTransaction(Map errors = addPartitionsToTxnResponse.errors(); boolean hasPartitionErrors = false; + Set unauthorizedTopics = new HashSet<>(); + + log.debug("TransactionalId {} -- Received AddPartitionsToTxn response with errors {}", + transactionalId, errors); + for (TopicPartition topicPartition : pendingPartitionsToBeAddedToTransaction) { final Errors error = errors.get(topicPartition); if (error == Errors.NONE || error == null) { @@ -640,23 +657,28 @@ public void handleResponse(AbstractResponse response) { reenqueue(); return; } else if (error == Errors.INVALID_PRODUCER_EPOCH) { - fenced(); + fatalError(error.exception()); return; } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { - fatal(error.exception()); + fatalError(error.exception()); return; } else if (error == Errors.INVALID_PRODUCER_ID_MAPPING || error == Errors.INVALID_TXN_STATE) { - fatal(new KafkaException(error.exception())); + fatalError(new KafkaException(error.exception())); return; + } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { + unauthorizedTopics.add(topicPartition.topic()); } else { - log.error("Could not add partitions to transaction due to partition error. partition={}, error={}", topicPartition, error); + log.error("TransactionalId: {} -- Could not add partition {} due to unexpected error {}", + transactionalId, topicPartition, error); hasPartitionErrors = true; } } - if (hasPartitionErrors) { - fatal(new KafkaException("Could not add partitions to transaction due to partition level errors")); + if (!unauthorizedTopics.isEmpty()) { + abortableError(new TopicAuthorizationException(unauthorizedTopics)); + } else if (hasPartitionErrors) { + abortableError(new KafkaException("Could not add partitions to transaction due to partition level errors")); } else { partitionsInTransaction.addAll(pendingPartitionsToBeAddedToTransaction); pendingPartitionsToBeAddedToTransaction.clear(); @@ -695,7 +717,12 @@ String coordinatorKey() { @Override public void handleResponse(AbstractResponse response) { FindCoordinatorResponse findCoordinatorResponse = (FindCoordinatorResponse) response; - if (findCoordinatorResponse.error() == Errors.NONE) { + Errors error = findCoordinatorResponse.error(); + + log.debug("TransactionalId {} -- Received FindCoordinator response with error {}", + transactionalId, error); + + if (error == Errors.NONE) { Node node = findCoordinatorResponse.node(); switch (builder.coordinatorType()) { case GROUP: @@ -705,12 +732,14 @@ public void handleResponse(AbstractResponse response) { transactionCoordinator = node; } result.done(); - } else if (findCoordinatorResponse.error() == Errors.COORDINATOR_NOT_AVAILABLE) { + } else if (error == Errors.COORDINATOR_NOT_AVAILABLE) { reenqueue(); + } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { + fatalError(error.exception()); } else if (findCoordinatorResponse.error() == Errors.GROUP_AUTHORIZATION_FAILED) { - fatal(new GroupAuthorizationException("Not authorized to commit offsets " + builder.coordinatorKey())); + abortableError(new GroupAuthorizationException(builder.coordinatorKey())); } else { - fatal(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to" + + fatalError(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to" + "unexpected error: %s", builder.coordinatorType(), builder.coordinatorKey(), findCoordinatorResponse.error().message()))); } @@ -743,6 +772,10 @@ boolean isEndTxn() { public void handleResponse(AbstractResponse response) { EndTxnResponse endTxnResponse = (EndTxnResponse) response; Errors error = endTxnResponse.error(); + + log.debug("TransactionalId {} -- Received EndTxn response with error {}", + transactionalId, error); + if (error == Errors.NONE) { completeTransaction(); result.done(); @@ -752,11 +785,13 @@ public void handleResponse(AbstractResponse response) { } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) { reenqueue(); } else if (error == Errors.INVALID_PRODUCER_EPOCH) { - fenced(); + fatalError(error.exception()); } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { - fatal(error.exception()); + fatalError(error.exception()); + } else if (error == Errors.INVALID_TXN_STATE) { + fatalError(error.exception()); } else { - fatal(new KafkaException("Unhandled error in EndTxnResponse: " + error.message())); + fatalError(new KafkaException("Unhandled error in EndTxnResponse: " + error.message())); } } } @@ -785,6 +820,10 @@ Priority priority() { public void handleResponse(AbstractResponse response) { AddOffsetsToTxnResponse addOffsetsToTxnResponse = (AddOffsetsToTxnResponse) response; Errors error = addOffsetsToTxnResponse.error(); + + log.debug("TransactionalId {} -- Received AddOffsetsToTxn response with error {}", + transactionalId, error); + if (error == Errors.NONE) { // note the result is not completed until the TxnOffsetCommit returns pendingRequests.add(txnOffsetCommitHandler(result, offsets, builder.consumerGroupId())); @@ -794,11 +833,13 @@ public void handleResponse(AbstractResponse response) { } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) { reenqueue(); } else if (error == Errors.INVALID_PRODUCER_EPOCH) { - fenced(); + fatalError(error.exception()); } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { - fatal(error.exception()); + fatalError(error.exception()); + } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { + abortableError(new GroupAuthorizationException(builder.consumerGroupId())); } else { - fatal(new KafkaException("Unexpected error in AddOffsetsToTxnResponse: " + error.message())); + fatalError(new KafkaException("Unexpected error in AddOffsetsToTxnResponse: " + error.message())); } } } @@ -837,7 +878,12 @@ public void handleResponse(AbstractResponse response) { TxnOffsetCommitResponse txnOffsetCommitResponse = (TxnOffsetCommitResponse) response; boolean coordinatorReloaded = false; boolean hadFailure = false; - for (Map.Entry entry : txnOffsetCommitResponse.errors().entrySet()) { + Map errors = txnOffsetCommitResponse.errors(); + + log.debug("TransactionalId {} -- Received TxnOffsetCommit response with errors {}", + transactionalId, errors); + + for (Map.Entry entry : errors.entrySet()) { TopicPartition topicPartition = entry.getKey(); Errors error = entry.getValue(); if (error == Errors.NONE) { @@ -850,11 +896,17 @@ public void handleResponse(AbstractResponse response) { } } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) { hadFailure = true; + } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { + abortableError(new GroupAuthorizationException(builder.consumerGroupId())); + return; + } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) { + fatalError(error.exception()); + return; } else if (error == Errors.INVALID_PRODUCER_EPOCH) { - fenced(); + fatalError(error.exception()); return; } else { - fatal(new KafkaException("Unexpected error in TxnOffsetCommitResponse: " + error.message())); + fatalError(new KafkaException("Unexpected error in TxnOffsetCommitResponse: " + error.message())); return; } } diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java deleted file mode 100644 index 2da91585bf28..000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common.errors; - -public class ProducerIdAuthorizationException extends ApiException { - public ProducerIdAuthorizationException(final String message) { - super(message); - } -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java index 9bf1fbb1367b..3f85513ae72e 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.common.errors; -public class TransactionalIdAuthorizationException extends ApiException { +public class TransactionalIdAuthorizationException extends AuthorizationException { public TransactionalIdAuthorizationException(final String message) { super(message); } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index f94fb4de8f05..9444eb574d34 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -57,7 +57,6 @@ import org.apache.kafka.common.errors.OutOfOrderSequenceException; import org.apache.kafka.common.errors.PolicyViolationException; import org.apache.kafka.common.errors.ProducerFencedException; -import org.apache.kafka.common.errors.ProducerIdAuthorizationException; import org.apache.kafka.common.errors.RebalanceInProgressException; import org.apache.kafka.common.errors.RecordBatchTooLargeException; import org.apache.kafka.common.errors.RecordTooLargeException; @@ -483,21 +482,13 @@ public ApiException build(String message) { return new TransactionalIdAuthorizationException(message); } }), - PRODUCER_ID_AUTHORIZATION_FAILED(54, "Producer is not authorized to use producer Ids, " + - "which is required to write idempotent data.", - new ApiExceptionBuilder() { - @Override - public ApiException build(String message) { - return new ProducerIdAuthorizationException(message); - } - }), - SECURITY_DISABLED(55, "Security features are disabled.", new ApiExceptionBuilder() { + SECURITY_DISABLED(54, "Security features are disabled.", new ApiExceptionBuilder() { @Override public ApiException build(String message) { return new SecurityDisabledException(message); } }), - BROKER_AUTHORIZATION_FAILED(56, "Broker authorization failed", new ApiExceptionBuilder() { + BROKER_AUTHORIZATION_FAILED(55, "Broker authorization failed", new ApiExceptionBuilder() { @Override public ApiException build(String message) { return new BrokerAuthorizationException(message); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index d5ce469e16ee..91391e99c4d8 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -1516,6 +1516,9 @@ public class Protocol { ); public static final Schema TXN_OFFSET_COMMIT_REQUEST_V0 = new Schema( + new Field("transactional_id", + STRING, + "The transactional id corresponding to the transaction."), new Field("consumer_group_id", STRING, "Id of the associated consumer group to commit offsets for."), diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java index 4bf8b3eeaf5c..3339470c0b8c 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java @@ -50,6 +50,17 @@ public String consumerGroupId() { public AddOffsetsToTxnRequest build(short version) { return new AddOffsetsToTxnRequest(version, transactionalId, producerId, producerEpoch, consumerGroupId); } + + @Override + public String toString() { + StringBuilder bld = new StringBuilder(); + bld.append("(transactionalId=").append(transactionalId). + append(", producerId=").append(producerId). + append(", producerEpoch=").append(producerEpoch). + append(", consumerGroupId=").append(consumerGroupId). + append(")"); + return bld.toString(); + } } private final String transactionalId; 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 8b3a5890817b..754f2421e18a 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 @@ -30,10 +30,11 @@ public class AddOffsetsToTxnResponse extends AbstractResponse { // NotCoordinator // CoordinatorNotAvailable // CoordinatorLoadInProgress - // InvalidPidMapping + // InvalidProducerIdMapping + // InvalidProducerEpoch // InvalidTxnState // GroupAuthorizationFailed - // InvalidProducerEpoch + // TransactionalIdAuthorizationFailed private final Errors error; private final int throttleTimeMs; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java index 148ebec4e365..e24fa5a08a45 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java @@ -54,6 +54,17 @@ public Builder(String transactionalId, long producerId, short producerEpoch, Lis public AddPartitionsToTxnRequest build(short version) { return new AddPartitionsToTxnRequest(version, transactionalId, producerId, producerEpoch, partitions); } + + @Override + public String toString() { + StringBuilder bld = new StringBuilder(); + bld.append("(transactionalId=").append(transactionalId). + append(", producerId=").append(producerId). + append(", producerEpoch=").append(producerEpoch). + append(", partitions=").append(partitions). + append(")"); + return bld.toString(); + } } private final String transactionalId; 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 697142bb07fa..39172eee51dd 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 @@ -43,11 +43,12 @@ public class AddPartitionsToTxnResponse extends AbstractResponse { // CoordinatorNotAvailable // CoordinatorLoadInProgress // InvalidTxnState - // InvalidPidMapping + // InvalidProducerIdMapping // TopicAuthorizationFailed // InvalidProducerEpoch // UnknownTopicOrPartition // TopicAuthorizationFailed + // TransactionalIdAuthorizationFailed private final Map errors; public AddPartitionsToTxnResponse(int throttleTimeMs, Map errors) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java index 77ec1379579f..b9f052c79cbb 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java @@ -50,6 +50,17 @@ public TransactionResult result() { public EndTxnRequest build(short version) { return new EndTxnRequest(version, transactionalId, producerId, producerEpoch, result); } + + @Override + public String toString() { + StringBuilder bld = new StringBuilder(); + bld.append("(transactionalId=").append(transactionalId). + append(", producerId=").append(producerId). + append(", producerEpoch=").append(producerEpoch). + append(", result=").append(result). + append(")"); + return bld.toString(); + } } private final String transactionalId; 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 99e4e8ce65ea..17cf68dfff5a 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 @@ -31,8 +31,9 @@ public class EndTxnResponse extends AbstractResponse { // CoordinatorNotAvailable // CoordinatorLoadInProgress // InvalidTxnState - // InvalidPidMapping + // InvalidProducerIdMapping // InvalidProducerEpoch + // TransactionalIdAuthorizationFailed private final Errors error; private final int throttleTimeMs; 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 7c8a6e54e6b8..96e1cdf989ae 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 @@ -24,11 +24,13 @@ import java.nio.ByteBuffer; public class InitProducerIdResponse extends AbstractResponse { - /** - * Possible Error codes: - * OK - * - */ + // Possible error codes: + // NotCoordinator + // CoordinatorNotAvailable + // CoordinatorLoadInProgress + // TransactionalIdAuthorizationFailed + // ClusterAuthorizationFailed + private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms"; private static final String PRODUCER_ID_KEY_NAME = "producer_id"; private static final String EPOCH_KEY_NAME = "producer_epoch"; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index 3377f914b3c3..3d696c1bec69 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -228,13 +228,14 @@ public String toString(boolean verbose) { } @Override - public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) { + public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) { /* In case the producer doesn't actually want any response */ if (acks == 0) return null; + Errors error = Errors.forException(e); Map responseMap = new HashMap<>(); - ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(Errors.forException(e)); + ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(error); for (TopicPartition tp : partitions()) responseMap.put(tp, partitionResponse); 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 42ae434fa7af..55332f6b00ec 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 @@ -61,6 +61,9 @@ public class ProduceResponse extends AbstractResponse { * INVALID_REQUIRED_ACKS (21) * TOPIC_AUTHORIZATION_FAILED (29) * UNSUPPORTED_FOR_MESSAGE_FORMAT (43) + * INVALID_PRODUCER_EPOCH (47) + * CLUSTER_AUTHORIZATION_FAILED (31) + * TRANSACTIONAL_ID_AUTHORIZATION_FAILED (53) */ private static final String BASE_OFFSET_KEY_NAME = "base_offset"; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java index f5334f2153fd..68fa3d20bff7 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java @@ -27,6 +27,7 @@ import java.util.Map; public class TxnOffsetCommitRequest extends AbstractRequest { + private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id"; private static final String CONSUMER_GROUP_ID_KEY_NAME = "consumer_group_id"; private static final String PRODUCER_ID_KEY_NAME = "producer_id"; private static final String PRODUCER_EPOCH_KEY_NAME = "producer_epoch"; @@ -38,14 +39,16 @@ public class TxnOffsetCommitRequest extends AbstractRequest { private static final String METADATA_KEY_NAME = "metadata"; public static class Builder extends AbstractRequest.Builder { + private final String transactionalId; private final String consumerGroupId; private final long producerId; private final short producerEpoch; private final Map offsets; - public Builder(String consumerGroupId, long producerId, short producerEpoch, + public Builder(String transactionalId, String consumerGroupId, long producerId, short producerEpoch, Map offsets) { super(ApiKeys.TXN_OFFSET_COMMIT); + this.transactionalId = transactionalId; this.consumerGroupId = consumerGroupId; this.producerId = producerId; this.producerEpoch = producerEpoch; @@ -58,18 +61,32 @@ public String consumerGroupId() { @Override public TxnOffsetCommitRequest build(short version) { - return new TxnOffsetCommitRequest(version, consumerGroupId, producerId, producerEpoch, offsets); + return new TxnOffsetCommitRequest(version, transactionalId, consumerGroupId, producerId, producerEpoch, offsets); + } + + @Override + public String toString() { + StringBuilder bld = new StringBuilder(); + bld.append("(transactionalId=").append(transactionalId). + append(", producerId=").append(producerId). + append(", producerEpoch=").append(producerEpoch). + append(", consumerGroupId=").append(consumerGroupId). + append(", offsets=").append(offsets). + append(")"); + return bld.toString(); } } + private final String transactionalId; private final String consumerGroupId; private final long producerId; private final short producerEpoch; private final Map offsets; - public TxnOffsetCommitRequest(short version, String consumerGroupId, long producerId, short producerEpoch, - Map offsets) { + public TxnOffsetCommitRequest(short version, String transactionalId, String consumerGroupId, long producerId, + short producerEpoch, Map offsets) { super(version); + this.transactionalId = transactionalId; this.consumerGroupId = consumerGroupId; this.producerId = producerId; this.producerEpoch = producerEpoch; @@ -78,6 +95,7 @@ public TxnOffsetCommitRequest(short version, String consumerGroupId, long produc public TxnOffsetCommitRequest(Struct struct, short version) { super(version); + this.transactionalId = struct.getString(TRANSACTIONAL_ID_KEY_NAME); this.consumerGroupId = struct.getString(CONSUMER_GROUP_ID_KEY_NAME); this.producerId = struct.getLong(PRODUCER_ID_KEY_NAME); this.producerEpoch = struct.getShort(PRODUCER_EPOCH_KEY_NAME); @@ -98,6 +116,10 @@ public TxnOffsetCommitRequest(Struct struct, short version) { this.offsets = offsets; } + public String transactionalId() { + return transactionalId; + } + public String consumerGroupId() { return consumerGroupId; } @@ -117,6 +139,7 @@ public Map offsets() { @Override protected Struct toStruct() { Struct struct = new Struct(ApiKeys.TXN_OFFSET_COMMIT.requestSchema(version())); + struct.set(TRANSACTIONAL_ID_KEY_NAME, transactionalId); struct.set(CONSUMER_GROUP_ID_KEY_NAME, consumerGroupId); struct.set(PRODUCER_ID_KEY_NAME, producerId); struct.set(PRODUCER_EPOCH_KEY_NAME, producerEpoch); 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 37b9a50b4af1..a62568f5583e 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 @@ -42,6 +42,7 @@ public class TxnOffsetCommitResponse extends AbstractResponse { // OffsetMetadataTooLarge // GroupAuthorizationFailed // InvalidCommitOffsetSize + // TransactionalIdAuthorizationFailed private final Map errors; private final int throttleTimeMs; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java index 06f6662cecc0..ddddc4269881 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java @@ -48,6 +48,7 @@ public class WriteTxnMarkersResponse extends AbstractResponse { // InvalidRequiredAcks // TransactionCoordinatorFenced // RequestTimeout + // ClusterAuthorizationFailed private final Map> errors; diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java index 06ace639272d..0e3441f04f13 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java @@ -47,7 +47,8 @@ private static class AclOperationTestInfo { new AclOperationTestInfo(AclOperation.DESCRIBE, 8, "describe", false), new AclOperationTestInfo(AclOperation.CLUSTER_ACTION, 9, "cluster_action", false), new AclOperationTestInfo(AclOperation.DESCRIBE_CONFIGS, 10, "describe_configs", false), - new AclOperationTestInfo(AclOperation.ALTER_CONFIGS, 11, "alter_configs", false) + new AclOperationTestInfo(AclOperation.ALTER_CONFIGS, 11, "alter_configs", false), + new AclOperationTestInfo(AclOperation.IDEMPOTENT_WRITE, 12, "idempotent_write", false) }; @Test 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 cc30f4d7d8b2..719efe918a44 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 @@ -25,6 +25,7 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricConfig; @@ -231,20 +232,8 @@ public void testRetries() throws Exception { int maxRetries = 1; Metrics m = new Metrics(); try { - Sender sender = new Sender(client, - metadata, - this.accumulator, - false, - MAX_REQUEST_SIZE, - ACKS_ALL, - maxRetries, - m, - time, - REQUEST_TIMEOUT, - 50, - null, - apiVersions - ); + Sender sender = new Sender(client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL, + maxRetries, m, time, REQUEST_TIMEOUT, 50, null, apiVersions); // do a successful retry Future future = accumulator.append(tp0, 0L, "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); // connect @@ -290,20 +279,8 @@ public void testSendInOrder() throws Exception { int maxRetries = 1; Metrics m = new Metrics(); try { - Sender sender = new Sender(client, - metadata, - this.accumulator, - true, - MAX_REQUEST_SIZE, - ACKS_ALL, - maxRetries, - m, - time, - REQUEST_TIMEOUT, - 50, - null, - apiVersions - ); + Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, + m, time, REQUEST_TIMEOUT, 50, null, apiVersions); // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 Cluster cluster1 = TestUtils.clusterWith(2, "test", 2); metadata.update(cluster1, Collections.emptySet(), time.milliseconds()); @@ -375,21 +352,63 @@ public void testMetadataTopicExpiry() throws Exception { } @Test - public void testInitPidRequest() throws Exception { + public void testInitProducerIdRequest() throws Exception { final long producerId = 343434L; TransactionManager transactionManager = new TransactionManager(); setupWithTransactionState(transactionManager); client.setNode(new Node(1, "localhost", 33343)); + prepareAndReceiveInitProducerId(producerId, Errors.NONE); + assertTrue(transactionManager.hasProducerId()); + assertEquals(producerId, transactionManager.producerIdAndEpoch().producerId); + assertEquals((short) 0, transactionManager.producerIdAndEpoch().epoch); + } + + @Test + public void testClusterAuthorizationExceptionInInitProducerIdRequest() throws Exception { + final long producerId = 343434L; + TransactionManager transactionManager = new TransactionManager(); + setupWithTransactionState(transactionManager); + client.setNode(new Node(1, "localhost", 33343)); + prepareAndReceiveInitProducerId(producerId, Errors.CLUSTER_AUTHORIZATION_FAILED); + assertFalse(transactionManager.hasProducerId()); + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof ClusterAuthorizationException); + + // cluster authorization is a fatal error for the producer + assertSendFailure(ClusterAuthorizationException.class); + } + + @Test + public void testClusterAuthorizationExceptionInProduceRequest() throws Exception { + final long producerId = 343434L; + TransactionManager transactionManager = new TransactionManager(); + setupWithTransactionState(transactionManager); + + client.setNode(new Node(1, "localhost", 33343)); + prepareAndReceiveInitProducerId(producerId, Errors.NONE); + assertTrue(transactionManager.hasProducerId()); + + // cluster authorization is a fatal error for the producer + Future future = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), + null, null, MAX_BLOCK_TIMEOUT).future; client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { - return body instanceof InitProducerIdRequest; + return body instanceof ProduceRequest && ((ProduceRequest) body).isIdempotent(); } - }, new InitProducerIdResponse(0, Errors.NONE, producerId, (short) 0)); + }, produceResponse(tp0, -1, Errors.CLUSTER_AUTHORIZATION_FAILED, 0)); + sender.run(time.milliseconds()); - assertTrue(transactionManager.hasProducerId()); - assertEquals(producerId, transactionManager.producerIdAndEpoch().producerId); - assertEquals((short) 0, transactionManager.producerIdAndEpoch().epoch); + assertTrue(future.isDone()); + try { + future.get(); + fail("Future should have raised ClusterAuthorizationException"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof ClusterAuthorizationException); + } + + // cluster authorization is a fatal error for the producer + assertSendFailure(ClusterAuthorizationException.class); } @Test @@ -402,20 +421,8 @@ public void testSequenceNumberIncrement() throws InterruptedException { int maxRetries = 10; Metrics m = new Metrics(); - Sender sender = new Sender(client, - metadata, - this.accumulator, - true, - MAX_REQUEST_SIZE, - ACKS_ALL, - maxRetries, - m, - time, - REQUEST_TIMEOUT, - 50, - transactionManager, - apiVersions - ); + Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, + m, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; client.prepareResponse(new MockClient.RequestMatcher() { @@ -446,7 +453,7 @@ public boolean matches(AbstractRequest body) { } @Test - public void testAbortRetryWhenPidChanges() throws InterruptedException { + public void testAbortRetryWhenProducerIdChanges() throws InterruptedException { final long producerId = 343434L; TransactionManager transactionManager = new TransactionManager(); transactionManager.setProducerIdAndEpoch(new ProducerIdAndEpoch(producerId, (short) 0)); @@ -455,20 +462,8 @@ public void testAbortRetryWhenPidChanges() throws InterruptedException { int maxRetries = 10; Metrics m = new Metrics(); - Sender sender = new Sender(client, - metadata, - this.accumulator, - true, - MAX_REQUEST_SIZE, - ACKS_ALL, - maxRetries, - m, - time, - REQUEST_TIMEOUT, - 50, - transactionManager, - apiVersions - ); + Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, + m, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); // connect. @@ -504,20 +499,8 @@ public void testResetWhenOutOfOrderSequenceReceived() throws InterruptedExceptio int maxRetries = 10; Metrics m = new Metrics(); - Sender sender = new Sender(client, - metadata, - this.accumulator, - true, - MAX_REQUEST_SIZE, - ACKS_ALL, - maxRetries, - m, - time, - REQUEST_TIMEOUT, - 50, - transactionManager, - apiVersions - ); + Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, + m, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); // connect. @@ -642,20 +625,38 @@ private void setupWithTransactionState(TransactionManager transactionManager) { metricTags.put("client-id", CLIENT_ID); MetricConfig metricConfig = new MetricConfig().tags(metricTags); this.metrics = new Metrics(metricConfig, time); - this.accumulator = new RecordAccumulator(batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, metrics, time, apiVersions, transactionManager); - this.sender = new Sender(this.client, - this.metadata, - this.accumulator, - true, - MAX_REQUEST_SIZE, - ACKS_ALL, - MAX_RETRIES, - this.metrics, - this.time, - REQUEST_TIMEOUT, - 50, - transactionManager, - apiVersions); + this.accumulator = new RecordAccumulator(batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, metrics, time, + apiVersions, transactionManager); + this.sender = new Sender(this.client, this.metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, + MAX_RETRIES, this.metrics, this.time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions); this.metadata.update(this.cluster, Collections.emptySet(), time.milliseconds()); } + + private void assertSendFailure(Class expectedError) throws Exception { + Future future = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), + null, null, MAX_BLOCK_TIMEOUT).future; + sender.run(time.milliseconds()); + assertTrue(future.isDone()); + try { + future.get(); + fail("Future should have raised " + expectedError.getSimpleName()); + } catch (ExecutionException e) { + assertTrue(expectedError.isAssignableFrom(e.getCause().getClass())); + } + } + + private void prepareAndReceiveInitProducerId(long producerId, Errors error) { + short producerEpoch = 0; + if (error != Errors.NONE) + producerEpoch = RecordBatch.NO_PRODUCER_EPOCH; + + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(AbstractRequest body) { + return body instanceof InitProducerIdRequest && ((InitProducerIdRequest) body).transactionalId() == null; + } + }, new InitProducerIdResponse(0, error, producerId, producerEpoch)); + sender.run(time.milliseconds()); + } + } 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 fcf04885498a..e9363d054e54 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 @@ -20,12 +20,14 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; import org.apache.kafka.clients.consumer.OffsetAndMetadata; -import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.errors.TransactionalIdAuthorizationException; import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -43,6 +45,7 @@ import org.apache.kafka.common.requests.EndTxnRequest; import org.apache.kafka.common.requests.EndTxnResponse; import org.apache.kafka.common.requests.FindCoordinatorRequest; +import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType; import org.apache.kafka.common.requests.FindCoordinatorResponse; import org.apache.kafka.common.requests.InitProducerIdRequest; import org.apache.kafka.common.requests.InitProducerIdResponse; @@ -65,6 +68,8 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonMap; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -118,6 +123,7 @@ public void setup() { transactionManager, apiVersions); this.metadata.update(this.cluster, Collections.emptySet(), time.milliseconds()); + client.setNode(brokerNode); } @Test(expected = IllegalStateException.class) @@ -134,7 +140,6 @@ public void testDefaultSequenceNumber() { assertEquals((int) transactionManager.sequenceNumber(tp0), 3); } - @Test public void testProducerIdReset() { TransactionManager transactionManager = new TransactionManager(); @@ -147,23 +152,13 @@ public void testProducerIdReset() { @Test public void testBasicTransaction() throws InterruptedException { - client.setNode(brokerNode); // 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. final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - - prepareInitPidResponse(Errors.NONE, false, pid, epoch); + doInitTransactions(pid, epoch); - sender.run(time.milliseconds()); // get pid. - - assertTrue(transactionManager.hasProducerId()); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); @@ -199,13 +194,13 @@ public void testBasicTransaction() throws InterruptedException { Map txnOffsetCommitResponse = new HashMap<>(); txnOffsetCommitResponse.put(tp1, Errors.NONE); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.GROUP, consumerGroupId); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, txnOffsetCommitResponse); - assertEquals(null, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP)); + assertEquals(null, transactionManager.coordinator(CoordinatorType.GROUP)); sender.run(time.milliseconds()); // try to send TxnOffsetCommitRequest, but find we don't have a group coordinator. sender.run(time.milliseconds()); // send find coordinator for group request - assertNotNull(transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP)); + assertNotNull(transactionManager.coordinator(CoordinatorType.GROUP)); assertTrue(transactionManager.hasPendingOffsetCommits()); sender.run(time.milliseconds()); // send TxnOffsetCommitRequest commit. @@ -224,42 +219,40 @@ public void testBasicTransaction() throws InterruptedException { @Test public void testDisconnectAndRetry() { - client.setNode(brokerNode); // 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(); - prepareFindCoordinatorResponse(Errors.NONE, true, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); + prepareFindCoordinatorResponse(Errors.NONE, true, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // find coordinator, connection lost. - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // find coordinator sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); } @Test public void testCoordinatorLost() { - client.setNode(brokerNode); // 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. final long pid = 13131L; final short epoch = 1; TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); // find coordinator sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); prepareInitPidResponse(Errors.NOT_COORDINATOR, false, pid, epoch); sender.run(time.milliseconds()); // send pid, get not coordinator. Should resend the FindCoordinator and InitPid requests - assertEquals(null, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); + assertEquals(null, transactionManager.coordinator(CoordinatorType.TRANSACTION)); assertFalse(initPidResult.isCompleted()); assertFalse(transactionManager.hasProducerId()); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); assertFalse(initPidResult.isCompleted()); prepareInitPidResponse(Errors.NONE, false, pid, epoch); sender.run(time.milliseconds()); // get pid and epoch @@ -271,24 +264,216 @@ public void testCoordinatorLost() { } @Test - public void testFlushPendingPartitionsOnCommit() throws InterruptedException { - client.setNode(brokerNode); - // 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. + public void testTransactionalIdAuthorizationFailureInFindCoordinator() { + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + prepareFindCoordinatorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, false, + CoordinatorType.TRANSACTION, transactionalId); + sender.run(time.milliseconds()); // find coordinator + sender.run(time.milliseconds()); + + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + + sender.run(time.milliseconds()); // one more run to fail the InitProducerId future + assertTrue(initPidResult.isCompleted()); + assertFalse(initPidResult.isSuccessful()); + assertTrue(initPidResult.error() instanceof TransactionalIdAuthorizationException); + + assertFatalError(TransactionalIdAuthorizationException.class); + } + + @Test + public void testTransactionalIdAuthorizationFailureInInitProducerId() { + final long pid = 13131L; + TransactionalRequestResult initPidResult = transactionManager.initializeTransactions(); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); + sender.run(time.milliseconds()); // find coordinator + sender.run(time.milliseconds()); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); + + prepareInitPidResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, false, pid, RecordBatch.NO_PRODUCER_EPOCH); + sender.run(time.milliseconds()); + + assertTrue(transactionManager.isInErrorState()); + assertTrue(initPidResult.isCompleted()); + assertFalse(initPidResult.isSuccessful()); + assertTrue(initPidResult.error() instanceof TransactionalIdAuthorizationException); + + assertFatalError(TransactionalIdAuthorizationException.class); + } + + @Test + public void testGroupAuthorizationFailureInFindCoordinator() { + final String consumerGroupId = "consumer"; final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - sender.run(time.milliseconds()); // find coordinator + doInitTransactions(pid, epoch); + + transactionManager.beginTransaction(); + TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( + singletonMap(new TopicPartition("foo", 0), new OffsetAndMetadata(39L)), consumerGroupId); + + prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); + sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued + sender.run(time.milliseconds()); // FindCoordinator Enqueued + + prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, false, CoordinatorType.GROUP, consumerGroupId); + sender.run(time.milliseconds()); // FindCoordinator Failed + sender.run(time.milliseconds()); // TxnOffsetCommit Aborted + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); + assertTrue(sendOffsetsResult.isCompleted()); + assertFalse(sendOffsetsResult.isSuccessful()); + assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); + + GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); + assertEquals(consumerGroupId, exception.groupId()); + + assertAbortableError(GroupAuthorizationException.class); + } + + @Test + public void testGroupAuthorizationFailureInTxnOffsetCommit() { + final String consumerGroupId = "consumer"; + final long pid = 13131L; + final short epoch = 1; + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(pid, epoch); + + transactionManager.beginTransaction(); + TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( + singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId); + + prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); + sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued + sender.run(time.milliseconds()); // FindCoordinator Enqueued + + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); + sender.run(time.milliseconds()); // FindCoordinator Returned + + prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp, Errors.GROUP_AUTHORIZATION_FAILED)); + sender.run(time.milliseconds()); // TxnOffsetCommit Handled + + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException); + assertTrue(sendOffsetsResult.isCompleted()); + assertFalse(sendOffsetsResult.isSuccessful()); + assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException); + + GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error(); + assertEquals(consumerGroupId, exception.groupId()); + + assertAbortableError(GroupAuthorizationException.class); + } + + @Test + public void testTransactionalIdAuthorizationFailureInAddOffsetsToTxn() { + final String consumerGroupId = "consumer"; + final long pid = 13131L; + final short epoch = 1; + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(pid, epoch); + + transactionManager.beginTransaction(); + TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( + singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId); + + prepareAddOffsetsToTxnResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, consumerGroupId, pid, epoch); + sender.run(time.milliseconds()); // AddOffsetsToTxn Handled + + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + assertTrue(sendOffsetsResult.isCompleted()); + assertFalse(sendOffsetsResult.isSuccessful()); + assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); + + assertFatalError(TransactionalIdAuthorizationException.class); + } + + @Test + public void testTransactionalIdAuthorizationFailureInTxnOffsetCommit() { + final String consumerGroupId = "consumer"; + final long pid = 13131L; + final short epoch = 1; + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(pid, epoch); + + transactionManager.beginTransaction(); + TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction( + singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId); + + prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch); + sender.run(time.milliseconds()); // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued + sender.run(time.milliseconds()); // FindCoordinator Enqueued + + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); + sender.run(time.milliseconds()); // FindCoordinator Returned + + prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)); + sender.run(time.milliseconds()); // TxnOffsetCommit Handled + + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + assertTrue(sendOffsetsResult.isCompleted()); + assertFalse(sendOffsetsResult.isSuccessful()); + assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException); + + assertFatalError(TransactionalIdAuthorizationException.class); + } + + @Test + public void testTopicAuthorizationFailureInAddPartitions() { + final long pid = 13131L; + final short epoch = 1; + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(pid, epoch); + + transactionManager.beginTransaction(); + transactionManager.maybeAddPartitionToTransaction(tp); + + prepareAddPartitionsToTxn(tp, Errors.TOPIC_AUTHORIZATION_FAILED); sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - prepareInitPidResponse(Errors.NONE, false, pid, epoch); + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof TopicAuthorizationException); - sender.run(time.milliseconds()); // get pid. + TopicAuthorizationException exception = (TopicAuthorizationException) transactionManager.lastError(); + assertEquals(singleton(tp.topic()), exception.unauthorizedTopics()); - assertTrue(transactionManager.hasProducerId()); + assertAbortableError(TopicAuthorizationException.class); + } + + @Test + public void testTransactionalIdAuthorizationFailureInAddPartitions() { + final long pid = 13131L; + final short epoch = 1; + final TopicPartition tp = new TopicPartition("foo", 0); + + doInitTransactions(pid, epoch); + + transactionManager.beginTransaction(); + transactionManager.maybeAddPartitionToTransaction(tp); + + prepareAddPartitionsToTxn(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED); + sender.run(time.milliseconds()); + + assertTrue(transactionManager.isInErrorState()); + assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException); + + assertFatalError(TransactionalIdAuthorizationException.class); + } + + @Test + public void testFlushPendingPartitionsOnCommit() throws InterruptedException { + final long pid = 13131L; + final short epoch = 1; + + doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); @@ -329,23 +514,11 @@ public void testFlushPendingPartitionsOnCommit() throws InterruptedException { @Test public void testMultipleAddPartitionsPerForOneProduce() throws InterruptedException { - client.setNode(brokerNode); - // 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. final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - - prepareInitPidResponse(Errors.NONE, false, pid, epoch); - sender.run(time.milliseconds()); // get pid. + doInitTransactions(pid, epoch); - assertTrue(transactionManager.hasProducerId()); transactionManager.beginTransaction(); // User does one producer.sed transactionManager.maybeAddPartitionToTransaction(tp0); @@ -392,28 +565,16 @@ public void testMultipleAddPartitionsPerForOneProduce() throws InterruptedExcept @Test(expected = ExecutionException.class) public void testProducerFencedException() throws InterruptedException, ExecutionException { - client.setNode(brokerNode); - // 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. final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - - prepareInitPidResponse(Errors.NONE, false, pid, epoch); + doInitTransactions(pid, epoch); - sender.run(time.milliseconds()); // get pid. - - assertTrue(transactionManager.hasProducerId()); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), - "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future; + "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; assertFalse(responseFuture.isDone()); prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, pid); @@ -429,28 +590,16 @@ public void testProducerFencedException() throws InterruptedException, Execution @Test public void testDisallowCommitOnProduceFailure() throws InterruptedException { - client.setNode(brokerNode); - // 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. final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - - prepareInitPidResponse(Errors.NONE, false, pid, epoch); + doInitTransactions(pid, epoch); - sender.run(time.milliseconds()); // get pid. - - assertTrue(transactionManager.hasProducerId()); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), - "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future; + "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; TransactionalRequestResult commitResult = transactionManager.beginCommittingTransaction(); assertFalse(responseFuture.isDone()); @@ -483,28 +632,16 @@ public void testDisallowCommitOnProduceFailure() throws InterruptedException { @Test public void testAllowAbortOnProduceFailure() throws InterruptedException { - client.setNode(brokerNode); - // 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. final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - - prepareInitPidResponse(Errors.NONE, false, pid, epoch); - sender.run(time.milliseconds()); // get pid. - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - assertTrue(transactionManager.hasProducerId()); + doInitTransactions(pid, epoch); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), - "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future; + "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; TransactionalRequestResult abortResult = transactionManager.beginAbortingTransaction(); assertFalse(responseFuture.isDone()); @@ -524,28 +661,16 @@ public void testAllowAbortOnProduceFailure() throws InterruptedException { @Test public void testHandlingOfUnknownTopicPartitionErrorOnAddPartitions() throws InterruptedException { - client.setNode(brokerNode); - // 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. final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - - prepareInitPidResponse(Errors.NONE, false, pid, epoch); - - sender.run(time.milliseconds()); // get pid. + doInitTransactions(pid, epoch); - assertTrue(transactionManager.hasProducerId()); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), - "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future; + "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; assertFalse(responseFuture.isDone()); prepareAddPartitionsToTxnResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION, tp0, epoch, pid); @@ -564,23 +689,11 @@ public void testHandlingOfUnknownTopicPartitionErrorOnAddPartitions() throws Int @Test public void testHandlingOfUnknownTopicPartitionErrorOnTxnOffsetCommit() throws InterruptedException { - client.setNode(brokerNode); - // 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. final long pid = 13131L; final short epoch = 1; - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION)); - - prepareInitPidResponse(Errors.NONE, false, pid, epoch); + doInitTransactions(pid, epoch); - sender.run(time.milliseconds()); // get pid. - - assertTrue(transactionManager.hasProducerId()); transactionManager.beginTransaction(); Map offsets = new HashMap<>(); @@ -597,13 +710,13 @@ public void testHandlingOfUnknownTopicPartitionErrorOnTxnOffsetCommit() throws I Map txnOffsetCommitResponse = new HashMap<>(); txnOffsetCommitResponse.put(tp1, Errors.UNKNOWN_TOPIC_OR_PARTITION); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.GROUP, consumerGroupId); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId); prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, txnOffsetCommitResponse); - assertEquals(null, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP)); + assertEquals(null, transactionManager.coordinator(CoordinatorType.GROUP)); sender.run(time.milliseconds()); // try to send TxnOffsetCommitRequest, but find we don't have a group coordinator. sender.run(time.milliseconds()); // send find coordinator for group request - assertNotNull(transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP)); + assertNotNull(transactionManager.coordinator(CoordinatorType.GROUP)); assertTrue(transactionManager.hasPendingOffsetCommits()); sender.run(time.milliseconds()); // send TxnOffsetCommitRequest request. @@ -625,58 +738,35 @@ public void shouldNotAddPartitionsToTransactionWhenTopicAuthorizationFailed() th } private void verifyAddPartitionsFailsWithPartitionLevelError(final Errors error) throws InterruptedException { - client.setNode(brokerNode); - transactionManager.initializeTransactions(); - prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId); - - sender.run(time.milliseconds()); // find coordinator - sender.run(time.milliseconds()); - final long pid = 1L; final short epoch = 1; - prepareInitPidResponse(Errors.NONE, false, pid, epoch); + doInitTransactions(pid, epoch); - sender.run(time.milliseconds()); // get pid. - - assertTrue(transactionManager.hasProducerId()); transactionManager.beginTransaction(); transactionManager.maybeAddPartitionToTransaction(tp0); Future responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), - "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; + "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future; assertFalse(responseFuture.isDone()); - prepareAddPartitionsToTxnPartitionErrorResponse(tp0, error); + prepareAddPartitionsToTxn(tp0, error); sender.run(time.milliseconds()); // attempt send addPartitions. assertTrue(transactionManager.isInErrorState()); assertFalse(transactionManager.transactionContainsPartition(tp0)); } - private void prepareAddPartitionsToTxnPartitionErrorResponse(final TopicPartition tp0, final Errors error) { + private void prepareAddPartitionsToTxn(final TopicPartition tp, final Errors error) { client.prepareResponse(new MockClient.RequestMatcher() { @Override public boolean matches(AbstractRequest body) { - assertTrue(body instanceof AddPartitionsToTxnRequest); - return true; - } - }, new AddPartitionsToTxnResponse(0, Collections.singletonMap(tp0, error))); - } - - private static class MockCallback implements Callback { - private final TransactionManager transactionManager; - public MockCallback(TransactionManager transactionManager) { - this.transactionManager = transactionManager; - } - @Override - public void onCompletion(RecordMetadata metadata, Exception exception) { - if (exception != null && transactionManager != null) { - transactionManager.setError(exception); + return body instanceof AddPartitionsToTxnRequest && + ((AddPartitionsToTxnRequest) body).partitions().contains(tp); } - } + }, new AddPartitionsToTxnResponse(0, singletonMap(tp, error))); } private void prepareFindCoordinatorResponse(Errors error, boolean shouldDisconnect, - final FindCoordinatorRequest.CoordinatorType coordinatorType, + final CoordinatorType coordinatorType, final String coordinatorKey) { client.prepareResponse(new MockClient.RequestMatcher() { @Override @@ -733,7 +823,7 @@ public boolean matches(AbstractRequest body) { assertEquals(transactionalId, addPartitionsToTxnRequest.transactionalId()); return true; } - }, new AddPartitionsToTxnResponse(0, Collections.singletonMap(topicPartition, error))); + }, new AddPartitionsToTxnResponse(0, singletonMap(topicPartition, error))); } private void prepareEndTxnResponse(Errors error, final TransactionResult result, final long pid, final short epoch) { @@ -782,9 +872,54 @@ public boolean matches(AbstractRequest body) { private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs) { ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP); - Map partResp = Collections.singletonMap(tp, resp); + Map partResp = singletonMap(tp, resp); return new ProduceResponse(partResp, throttleTimeMs); } + private void doInitTransactions(long pid, short epoch) { + transactionManager.initializeTransactions(); + prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId); + sender.run(time.milliseconds()); // find coordinator + sender.run(time.milliseconds()); + assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION)); + + prepareInitPidResponse(Errors.NONE, false, pid, epoch); + sender.run(time.milliseconds()); // get pid. + assertTrue(transactionManager.hasProducerId()); + } + private void assertAbortableError(Class cause) { + try { + transactionManager.beginTransaction(); + fail("Should have raised " + cause.getSimpleName()); + } catch (KafkaException e) { + assertTrue(cause.isAssignableFrom(e.getCause().getClass())); + assertTrue(transactionManager.isInErrorState()); + } + + assertTrue(transactionManager.isInErrorState()); + transactionManager.beginAbortingTransaction(); + assertFalse(transactionManager.isInErrorState()); + } + + private void assertFatalError(Class cause) { + assertTrue(transactionManager.isInErrorState()); + + try { + transactionManager.beginAbortingTransaction(); + fail("Should have raised " + cause.getSimpleName()); + } catch (KafkaException e) { + assertTrue(cause.isAssignableFrom(e.getCause().getClass())); + assertTrue(transactionManager.isInErrorState()); + } + + // Transaction abort cannot clear fatal error state + try { + transactionManager.beginAbortingTransaction(); + fail("Should have raised " + cause.getSimpleName()); + } catch (KafkaException e) { + assertTrue(cause.isAssignableFrom(e.getCause().getClass())); + assertTrue(transactionManager.isInErrorState()); + } + } } 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 9142c90ecc56..2e9a688336bd 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 @@ -115,10 +115,10 @@ public void testSerialization() throws Exception { checkErrorResponse(createListOffsetRequest(2), new UnknownServerException()); checkResponse(createListOffsetResponse(2), 2); checkRequest(MetadataRequest.Builder.allTopics().build((short) 2)); - checkRequest(createMetadataRequest(1, asList("topic1"))); - checkErrorResponse(createMetadataRequest(1, asList("topic1")), new UnknownServerException()); + checkRequest(createMetadataRequest(1, singletonList("topic1"))); + checkErrorResponse(createMetadataRequest(1, singletonList("topic1")), new UnknownServerException()); checkResponse(createMetadataResponse(), 2); - checkErrorResponse(createMetadataRequest(2, asList("topic1")), new UnknownServerException()); + checkErrorResponse(createMetadataRequest(2, singletonList("topic1")), new UnknownServerException()); checkRequest(createOffsetCommitRequest(2)); checkErrorResponse(createOffsetCommitRequest(2), new UnknownServerException()); checkResponse(createOffsetCommitResponse(), 0); @@ -183,7 +183,7 @@ public void testSerialization() throws Exception { checkOlderFetchVersions(); checkResponse(createMetadataResponse(), 0); checkResponse(createMetadataResponse(), 1); - checkErrorResponse(createMetadataRequest(1, asList("topic1")), new UnknownServerException()); + checkErrorResponse(createMetadataRequest(1, singletonList("topic1")), new UnknownServerException()); checkRequest(createOffsetCommitRequest(0)); checkErrorResponse(createOffsetCommitRequest(0), new UnknownServerException()); checkRequest(createOffsetCommitRequest(1)); @@ -984,7 +984,7 @@ private TxnOffsetCommitRequest createTxnOffsetCommitRequest() { final Map offsets = new HashMap<>(); offsets.put(new TopicPartition("topic", 73), new TxnOffsetCommitRequest.CommittedOffset(100, null)); - return new TxnOffsetCommitRequest.Builder("gid", 21L, (short) 42, offsets).build(); + return new TxnOffsetCommitRequest.Builder("transactionalId", "groupId", 21L, (short) 42, offsets).build(); } private TxnOffsetCommitResponse createTxnOffsetCommitResponse() { diff --git a/core/src/main/scala/kafka/admin/AclCommand.scala b/core/src/main/scala/kafka/admin/AclCommand.scala index 925c407c62ff..e02b5dcf2062 100644 --- a/core/src/main/scala/kafka/admin/AclCommand.scala +++ b/core/src/main/scala/kafka/admin/AclCommand.scala @@ -34,7 +34,8 @@ object AclCommand { Broker -> Set(DescribeConfigs), Topic -> Set(Read, Write, Describe, Delete, DescribeConfigs, AlterConfigs, All), Group -> Set(Read, Describe, All), - Cluster -> Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, All) + Cluster -> Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite, All), + TransactionalId -> Set(Describe, Write, All) ) def main(args: Array[String]) { @@ -88,7 +89,7 @@ object AclCommand { CommandLineUtils.printUsageAndDie(opts.parser, "You must specify one of: --allow-principal, --deny-principal when trying to add ACLs.") for ((resource, acls) <- resourceToAcl) { - println(s"Adding ACLs for resource `${resource}`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") + println(s"Adding ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") authorizer.addAcls(acls, resource) } @@ -102,10 +103,10 @@ object AclCommand { for ((resource, acls) <- resourceToAcl) { if (acls.isEmpty) { - if (confirmAction(opts, s"Are you sure you want to delete all ACLs for resource `${resource}`? (y/n)")) + if (confirmAction(opts, s"Are you sure you want to delete all ACLs for resource `$resource`? (y/n)")) authorizer.removeAcls(resource) } else { - if (confirmAction(opts, s"Are you sure you want to remove ACLs: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource `${resource}`? (y/n)")) + if (confirmAction(opts, s"Are you sure you want to remove ACLs: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource `$resource`? (y/n)")) authorizer.removeAcls(acls, resource) } } @@ -123,7 +124,7 @@ object AclCommand { else resources.map(resource => resource -> authorizer.getAcls(resource)) for ((resource, acls) <- resourceToAcls) - println(s"Current ACLs for resource `${resource}`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") + println(s"Current ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline") } } @@ -149,12 +150,16 @@ object AclCommand { private def getProducerResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = { val topics: Set[Resource] = getResource(opts).filter(_.resourceType == Topic) + val transactionalIds: Set[Resource] = getResource(opts).filter(_.resourceType == TransactionalId) + val enableIdempotence = opts.options.has(opts.idempotentOpt) val acls = getAcl(opts, Set(Write, Describe)) - //Write, Describe permission on topics, Create permission on cluster - topics.map(_ -> acls).toMap[Resource, Set[Acl]] + - (Resource.ClusterResource -> getAcl(opts, Set(Create))) + //Write, Describe permission on topics, Create permission on cluster, Write, Describe on transactionalIds + topics.map(_ -> acls).toMap[Resource, Set[Acl]] ++ + transactionalIds.map(_ -> acls).toMap[Resource, Set[Acl]] + + (Resource.ClusterResource -> (getAcl(opts, Set(Create)) ++ + (if (enableIdempotence) getAcl(opts, Set(IdempotentWrite)) else Set.empty[Acl]))) } private def getConsumerResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = { @@ -184,7 +189,7 @@ object AclCommand { val allowedHosts = getHosts(opts, opts.allowHostsOpt, opts.allowPrincipalsOpt) - val deniedHosts = getHosts(opts, opts.denyHostssOpt, opts.denyPrincipalsOpt) + val deniedHosts = getHosts(opts, opts.denyHostsOpt, opts.denyPrincipalsOpt) val acls = new collection.mutable.HashSet[Acl] if (allowedHosts.nonEmpty && allowedPrincipals.nonEmpty) @@ -232,7 +237,7 @@ object AclCommand { if (opts.options.has(opts.topicOpt)) opts.options.valuesOf(opts.topicOpt).asScala.foreach(topic => resources += new Resource(Topic, topic.trim)) - if (opts.options.has(opts.clusterOpt)) + if (opts.options.has(opts.clusterOpt) || opts.options.has(opts.idempotentOpt)) resources += Resource.ClusterResource if (opts.options.has(opts.groupOpt)) @@ -241,6 +246,10 @@ object AclCommand { if (opts.options.has(opts.brokerOpt)) opts.options.valuesOf(opts.brokerOpt).asScala.foreach(broker => resources += new Resource(Broker, broker.toString)) + if (opts.options.has(opts.transactionalIdOpt)) + opts.options.valuesOf(opts.transactionalIdOpt).asScala.foreach(transactionalId => + resources += new Resource(TransactionalId, transactionalId)) + if (resources.isEmpty && dieIfNoResourceFound) CommandLineUtils.printUsageAndDie(opts.parser, "You must provide at least one resource: --topic or --cluster or --group ") @@ -295,6 +304,16 @@ object AclCommand { .describedAs("broker") .ofType(classOf[Int]) + val transactionalIdOpt = parser.accepts("transactional-id", "The transactionalId to which ACLs should " + + "be added or removed. A value of * indicates the ACLs should apply to all transactionalIds.") + .withRequiredArg + .describedAs("transactional-id") + .ofType(classOf[String]) + + val idempotentOpt = parser.accepts("idempotent", "Enable idempotence for the producer. This should be " + + "used in combination with the --producer option. Note that idempotence is enabled automatically if " + + "the producer is authorized to a particular transactional-id.") + val addOpt = parser.accepts("add", "Indicates you are trying to add ACLs.") val removeOpt = parser.accepts("remove", "Indicates you are trying to remove ACLs.") val listOpt = parser.accepts("list", "List ACLs for the specified resource, use --topic or --group or --cluster to specify a resource.") @@ -329,7 +348,7 @@ object AclCommand { .describedAs("allow-host") .ofType(classOf[String]) - val denyHostssOpt = parser.accepts("deny-host", "Host from which principals listed in --deny-principal will be denied access. " + + val denyHostsOpt = parser.accepts("deny-host", "Host from which principals listed in --deny-principal will be denied access. " + "If you have specified --deny-principal then the default for this option will be set to * which denies access from all hosts.") .withRequiredArg .describedAs("deny-host") @@ -354,17 +373,20 @@ object AclCommand { if (actions != 1) CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --add, --remove. ") - CommandLineUtils.checkInvalidArgs(parser, options, listOpt, Set(producerOpt, consumerOpt, allowHostsOpt, allowPrincipalsOpt, denyHostssOpt, denyPrincipalsOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, listOpt, Set(producerOpt, consumerOpt, allowHostsOpt, allowPrincipalsOpt, denyHostsOpt, denyPrincipalsOpt)) //when --producer or --consumer is specified , user should not specify operations as they are inferred and we also disallow --deny-principals and --deny-hosts. - CommandLineUtils.checkInvalidArgs(parser, options, producerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostssOpt)) - CommandLineUtils.checkInvalidArgs(parser, options, consumerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostssOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, producerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostsOpt)) + CommandLineUtils.checkInvalidArgs(parser, options, consumerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostsOpt)) if (options.has(producerOpt) && !options.has(topicOpt)) CommandLineUtils.printUsageAndDie(parser, "With --producer you must specify a --topic") - if (options.has(consumerOpt) && (!options.has(topicOpt) || !options.has(groupOpt) || (!options.has(producerOpt) && options.has(clusterOpt)))) - CommandLineUtils.printUsageAndDie(parser, "With --consumer you must specify a --topic and a --group and no --cluster option should be specified.") + if (options.has(idempotentOpt) && !options.has(producerOpt)) + CommandLineUtils.printUsageAndDie(parser, "The --idempotent option is only available if --producer is set") + + if (options.has(consumerOpt) && (!options.has(topicOpt) || !options.has(groupOpt) || (!options.has(producerOpt) && (options.has(clusterOpt) || options.has(transactionalIdOpt))))) + CommandLineUtils.printUsageAndDie(parser, "With --consumer you must specify a --topic and a --group and no --cluster or --transactional-id option should be specified.") } } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala index 302fcb5784d5..7bde4e2f7936 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala @@ -323,6 +323,7 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState } def prepareTxnOffsetCommit(producerId: Long, offsets: Map[TopicPartition, OffsetAndMetadata]) { + trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offsets $offsets is pending") receivedTransactionalOffsetCommits = true val producerOffsets = pendingTransactionalOffsetCommits.getOrElseUpdate(producerId, mutable.Map.empty[TopicPartition, CommitRecordMetadataAndOffset]) @@ -339,10 +340,12 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState /* Remove a pending transactional offset commit if the actual offset commit record was not written to the log. * We will return an error and the client will retry the request, potentially to a different coordinator. */ - def failPendingTxnOffsetCommit(producerId: Long, topicPartition: TopicPartition, offsetAndMetadata: OffsetAndMetadata): Unit = { + def failPendingTxnOffsetCommit(producerId: Long, topicPartition: TopicPartition): Unit = { pendingTransactionalOffsetCommits.get(producerId) match { case Some(pendingOffsets) => - pendingOffsets.remove(topicPartition) + val pendingOffsetCommit = pendingOffsets.remove(topicPartition) + trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offsets $pendingOffsetCommit failed " + + s"to be appended to the log") if (pendingOffsets.isEmpty) pendingTransactionalOffsetCommits.remove(producerId) case _ => @@ -366,18 +369,28 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState * to the log. */ def completePendingTxnOffsetCommit(producerId: Long, isCommit: Boolean): Unit = { - trace(s"Completing transactional offset commit for producer $producerId and group $groupId. isCommit: $isCommit") + val pendingOffsetsOpt = pendingTransactionalOffsetCommits.remove(producerId) if (isCommit) { - val producerOffsets = pendingTransactionalOffsetCommits.getOrElse(producerId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset]) - producerOffsets.foreach { case (topicPartition, commitRecordMetadataAndOffset) => - if (commitRecordMetadataAndOffset.appendedBatchOffset.isEmpty) - throw new IllegalStateException(s"Trying to complete a transactional offset commit for producerId $producerId " + - s"and groupId $groupId even though the the offset commit record itself hasn't been appended to the log.") - if (!offsets.contains(topicPartition) || offsets(topicPartition).olderThan(commitRecordMetadataAndOffset)) - offsets.put(topicPartition, commitRecordMetadataAndOffset) + pendingOffsetsOpt.foreach { pendingOffsets => + pendingOffsets.foreach { case (topicPartition, commitRecordMetadataAndOffset) => + if (commitRecordMetadataAndOffset.appendedBatchOffset.isEmpty) + throw new IllegalStateException(s"Trying to complete a transactional offset commit for producerId $producerId " + + s"and groupId $groupId even though the the offset commit record itself hasn't been appended to the log.") + + val currentOffsetOpt = offsets.get(topicPartition) + if (currentOffsetOpt.forall(_.olderThan(commitRecordMetadataAndOffset))) { + trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offset $commitRecordMetadataAndOffset " + + "committed and loaded into the cache.") + offsets.put(topicPartition, commitRecordMetadataAndOffset) + } else { + trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offset $commitRecordMetadataAndOffset " + + s"committed, but not loaded since its offset is older than current offset $currentOffsetOpt.") + } + } } + } else { + trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offsets $pendingOffsetsOpt aborted") } - pendingTransactionalOffsetCommits.remove(producerId) } def activeProducers = pendingTransactionalOffsetCommits.keySet @@ -430,7 +443,13 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState } override def toString: String = { - "[%s,%s,%s,%s]".format(groupId, protocolType, currentState.toString, members) + "GroupMetadata(" + + s"groupId=$groupId, " + + s"generation=$generationId, " + + s"protocolType=$protocolType, " + + s"currentState=$currentState, " + + s"members=$members)" } + } diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala index a7eb28b032a1..8e5135d466a7 100644 --- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala @@ -324,7 +324,7 @@ class GroupMetadataManager(brokerId: Int, removeProducerGroup(producerId, group.groupId) filteredOffsetMetadata.foreach { case (topicPartition, offsetAndMetadata) => if (isTxnOffsetCommit) - group.failPendingTxnOffsetCommit(producerId, topicPartition, offsetAndMetadata) + group.failPendingTxnOffsetCommit(producerId, topicPartition) else group.failPendingOffsetWrite(topicPartition, offsetAndMetadata) } @@ -536,7 +536,6 @@ class GroupMetadataManager(brokerId: Int, val groupId = groupMetadataKey.key val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value) if (groupMetadata != null) { - trace(s"Loaded group metadata for group $groupId with generation ${groupMetadata.generationId}") removedGroups.remove(groupId) loadedGroups.put(groupId, groupMetadata) } else { @@ -577,6 +576,7 @@ class GroupMetadataManager(brokerId: Int, loadedGroups.values.foreach { group => val offsets = groupOffsets.getOrElse(group.groupId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset]) val pendingOffsets = pendingGroupOffsets.getOrElse(group.groupId, Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]) + trace(s"Loaded group metadata $group with offsets $offsets and pending offsets $pendingOffsets") loadGroup(group, offsets, pendingOffsets) onGroupLoaded(group) } @@ -587,6 +587,7 @@ class GroupMetadataManager(brokerId: Int, val group = new GroupMetadata(groupId) val offsets = emptyGroupOffsets.getOrElse(groupId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset]) val pendingOffsets = pendingEmptyGroupOffsets.getOrElse(groupId, Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]]) + trace(s"Loaded group metadata $group with offsets $offsets and pending offsets $pendingOffsets") loadGroup(group, offsets, pendingOffsets) onGroupLoaded(group) } diff --git a/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala b/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala index ce542b1fd555..b082b9bad2ef 100644 --- a/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala @@ -85,7 +85,7 @@ private[group] class MemberMetadata(val memberId: String, if (protocols.size != this.supportedProtocols.size) return false - for (i <- 0 until protocols.size) { + for (i <- protocols.indices) { val p1 = protocols(i) val p2 = supportedProtocols(i) if (p1._1 != p2._1 || !util.Arrays.equals(p1._2, p2._2)) @@ -114,7 +114,15 @@ private[group] class MemberMetadata(val memberId: String, } } - override def toString = { - "[%s,%s,%s,%d]".format(memberId, clientId, clientHost, sessionTimeoutMs) + override def toString: String = { + "MemberMetadata(" + + s"memberId=$memberId, " + + s"clientId=$clientId, " + + s"clientHost=$clientHost, " + + s"sessionTimeoutMs=$sessionTimeoutMs, " + + s"rebalanceTimeoutMs=$rebalanceTimeoutMs, " + + s"supportedProtocols=${supportedProtocols.map(_._1)}, " + + ")" } + } diff --git a/core/src/main/scala/kafka/security/auth/Operation.scala b/core/src/main/scala/kafka/security/auth/Operation.scala index f65d9f043592..420c3eb03198 100644 --- a/core/src/main/scala/kafka/security/auth/Operation.scala +++ b/core/src/main/scala/kafka/security/auth/Operation.scala @@ -65,6 +65,10 @@ case object AlterConfigs extends Operation { val name = "AlterConfigs" val toJava = AclOperation.ALTER_CONFIGS } +case object IdempotentWrite extends Operation { + val name = "IdempotentWrite" + val toJava = AclOperation.IDEMPOTENT_WRITE +} case object All extends Operation { val name = "All" val toJava = AclOperation.ALL @@ -86,5 +90,5 @@ object Operation { } def values: Seq[Operation] = List(Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs, - DescribeConfigs, All) + DescribeConfigs, IdempotentWrite, All) } diff --git a/core/src/main/scala/kafka/security/auth/Resource.scala b/core/src/main/scala/kafka/security/auth/Resource.scala index a0ed9f90504a..311f5b5083a6 100644 --- a/core/src/main/scala/kafka/security/auth/Resource.scala +++ b/core/src/main/scala/kafka/security/auth/Resource.scala @@ -21,7 +21,6 @@ object Resource { val ClusterResourceName = "kafka-cluster" val ClusterResource = new Resource(Cluster, Resource.ClusterResourceName) val ProducerIdResourceName = "producer-id" - val ProducerIdResource = new Resource(Cluster, Resource.ProducerIdResourceName) val WildCardResource = "*" def fromString(str: String): Resource = { @@ -38,7 +37,7 @@ object Resource { * @param name name of the resource, for topic this will be topic name , for group it will be group name. For cluster type * it will be a constant string kafka-cluster. */ -case class Resource(val resourceType: ResourceType, val name: String) { +case class Resource(resourceType: ResourceType, name: String) { override def toString: String = { resourceType.name + Resource.Separator + name diff --git a/core/src/main/scala/kafka/security/auth/ResourceType.scala b/core/src/main/scala/kafka/security/auth/ResourceType.scala index ea7ce3c2b907..9cfe1cda879b 100644 --- a/core/src/main/scala/kafka/security/auth/ResourceType.scala +++ b/core/src/main/scala/kafka/security/auth/ResourceType.scala @@ -19,11 +19,6 @@ package kafka.security.auth import kafka.common.{BaseEnum, KafkaException} import org.apache.kafka.common.protocol.Errors -/** - * ResourceTypes. - */ - - sealed trait ResourceType extends BaseEnum { def error: Errors } case object Cluster extends ResourceType { @@ -46,16 +41,11 @@ case object Group extends ResourceType { val error = Errors.GROUP_AUTHORIZATION_FAILED } -case object ProducerTransactionalId extends ResourceType { - val name = "ProducerTransactionalId" +case object TransactionalId extends ResourceType { + val name = "TransactionalId" val error = Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED } -case object ProducerIdResource extends ResourceType { - val name = "ProducerIdResource" - val error = Errors.PRODUCER_ID_AUTHORIZATION_FAILED -} - object ResourceType { def fromString(resourceType: String): ResourceType = { @@ -63,5 +53,5 @@ object ResourceType { rType.getOrElse(throw new KafkaException(resourceType + " not a valid resourceType name. The valid names are " + values.mkString(","))) } - def values: Seq[ResourceType] = List(Cluster, Topic, Group, ProducerTransactionalId, ProducerIdResource, Broker) + def values: Seq[ResourceType] = List(Cluster, Topic, Group, TransactionalId, Broker) } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index b0d354b9e76c..380685f6a44a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -93,7 +93,7 @@ class KafkaApis(val requestChannel: RequestChannel, trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s". format(request.requestDesc(true), request.connectionId, request.securityProtocol, request.session.principal)) ApiKeys.forId(request.requestId) match { - case ApiKeys.PRODUCE => handleProducerRequest(request) + case ApiKeys.PRODUCE => handleProduceRequest(request) case ApiKeys.FETCH => handleFetchRequest(request) case ApiKeys.LIST_OFFSETS => handleListOffsetRequest(request) case ApiKeys.METADATA => handleTopicMetadataRequest(request) @@ -368,97 +368,107 @@ class KafkaApis(val requestChannel: RequestChannel, /** * Handle a produce request */ - def handleProducerRequest(request: RequestChannel.Request) { + def handleProduceRequest(request: RequestChannel.Request) { val produceRequest = request.body[ProduceRequest] val numBytesAppended = request.header.toStruct.sizeOf + request.bodyAndSize.size - if (produceRequest.isTransactional && !authorize(request.session, Write, new Resource(ProducerTransactionalId, produceRequest.transactionalId()))) - sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception)) - else if (produceRequest.isIdempotent && !authorize(request.session, Write, Resource.ProducerIdResource)) - sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.PRODUCER_ID_AUTHORIZATION_FAILED.exception)) - else { - val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = - produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) => - authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic) - } + def sendErrorResponse(error: Errors): Unit = { + sendResponseMaybeThrottle(request, requestThrottleMs => + produceRequest.getErrorResponse(requestThrottleMs, error.exception)) + } - val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { - case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic)) + if (produceRequest.isTransactional) { + if (!authorize(request.session, Write, new Resource(TransactionalId, produceRequest.transactionalId))) { + sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) + return } + // Note that authorization to a transactionalId implies ProducerId authorization - // the callback for sending a produce response - def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { + } else if (produceRequest.isIdempotent && !authorize(request.session, IdempotentWrite, Resource.ClusterResource)) { + sendErrorResponse(Errors.CLUSTER_AUTHORIZATION_FAILED) + return + } + + val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = + produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) => + authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic) + } - val mergedResponseStatus = responseStatus ++ - unauthorizedForWriteRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED)) ++ - nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)) + val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { + case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic)) + } - var errorInResponse = false + // the callback for sending a produce response + def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { - mergedResponseStatus.foreach { case (topicPartition, status) => - if (status.error != Errors.NONE) { - errorInResponse = true - debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( - request.header.correlationId, - request.header.clientId, - topicPartition, - status.error.exceptionName)) - } + val mergedResponseStatus = responseStatus ++ + unauthorizedForWriteRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED)) ++ + nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)) + + var errorInResponse = false + + mergedResponseStatus.foreach { case (topicPartition, status) => + if (status.error != Errors.NONE) { + errorInResponse = true + debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( + request.header.correlationId, + request.header.clientId, + topicPartition, + status.error.exceptionName)) } + } - def produceResponseCallback(bandwidthThrottleTimeMs: Int) { - if (produceRequest.acks == 0) { - // no operation needed if producer request.required.acks = 0; however, if there is any error in handling - // the request, since no response is expected by the producer, the server will close socket server so that - // the producer client will know that some error has happened and will refresh its metadata - val action = - if (errorInResponse) { - val exceptionsSummary = mergedResponseStatus.map { case (topicPartition, status) => - topicPartition -> status.error.exceptionName - }.mkString(", ") - info( - s"Closing connection due to error during produce request with correlation id ${request.header.correlationId} " + - s"from client id ${request.header.clientId} with ack=0\n" + - s"Topic and partition to exceptions: $exceptionsSummary" - ) - RequestChannel.CloseConnectionAction - } else RequestChannel.NoOpAction - sendResponseExemptThrottle(new RequestChannel.Response(request, None, action)) - } else { - sendResponseMaybeThrottle(request, requestThrottleMs => - new ProduceResponse(mergedResponseStatus.asJava, bandwidthThrottleTimeMs + requestThrottleMs) + def produceResponseCallback(bandwidthThrottleTimeMs: Int) { + if (produceRequest.acks == 0) { + // no operation needed if producer request.required.acks = 0; however, if there is any error in handling + // the request, since no response is expected by the producer, the server will close socket server so that + // the producer client will know that some error has happened and will refresh its metadata + val action = + if (errorInResponse) { + val exceptionsSummary = mergedResponseStatus.map { case (topicPartition, status) => + topicPartition -> status.error.exceptionName + }.mkString(", ") + info( + s"Closing connection due to error during produce request with correlation id ${request.header.correlationId} " + + s"from client id ${request.header.clientId} with ack=0\n" + + s"Topic and partition to exceptions: $exceptionsSummary" ) - } + RequestChannel.CloseConnectionAction + } else RequestChannel.NoOpAction + sendResponseExemptThrottle(new RequestChannel.Response(request, None, action)) + } else { + sendResponseMaybeThrottle(request, requestThrottleMs => + new ProduceResponse(mergedResponseStatus.asJava, bandwidthThrottleTimeMs + requestThrottleMs)) } + } - // When this callback is triggered, the remote API call has completed - request.apiRemoteCompleteTimeNanos = time.nanoseconds + // When this callback is triggered, the remote API call has completed + request.apiRemoteCompleteTimeNanos = time.nanoseconds - quotas.produce.recordAndMaybeThrottle( - request.session.sanitizedUser, - request.header.clientId, - numBytesAppended, - produceResponseCallback) - } + quotas.produce.recordAndMaybeThrottle( + request.session.sanitizedUser, + request.header.clientId, + numBytesAppended, + produceResponseCallback) + } - if (authorizedRequestInfo.isEmpty) - sendResponseCallback(Map.empty) - else { - val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId - - // call the replica manager to append messages to the replicas - replicaManager.appendRecords( - timeout = produceRequest.timeout.toLong, - requiredAcks = produceRequest.acks, - internalTopicsAllowed = internalTopicsAllowed, - isFromClient = true, - entriesPerPartition = authorizedRequestInfo, - responseCallback = sendResponseCallback) - - // if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected; - // hence we clear its data here inorder to let GC re-claim its memory since it is already appended to log - produceRequest.clearPartitionRecords() - } + if (authorizedRequestInfo.isEmpty) + sendResponseCallback(Map.empty) + else { + val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId + + // call the replica manager to append messages to the replicas + replicaManager.appendRecords( + timeout = produceRequest.timeout.toLong, + requiredAcks = produceRequest.acks, + internalTopicsAllowed = internalTopicsAllowed, + isFromClient = true, + entriesPerPartition = authorizedRequestInfo, + responseCallback = sendResponseCallback) + + // if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected; + // hence we clear its data here inorder to let GC re-claim its memory since it is already appended to log + produceRequest.clearPartitionRecords() } } @@ -1052,13 +1062,16 @@ class KafkaApis(val requestChannel: RequestChannel, def handleFindCoordinatorRequest(request: RequestChannel.Request) { val findCoordinatorRequest = request.body[FindCoordinatorRequest] - if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.GROUP && - !authorize(request.session, Describe, new Resource(Group, findCoordinatorRequest.coordinatorKey))) { - sendResponseMaybeThrottle(request, requestThrottleMs => - new FindCoordinatorResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode)) - } else { - // TODO: Authorize by transactional id if coordinator type is TRANSACTION + def sendErrorResponse(error: Errors): Unit = + sendResponseMaybeThrottle(request, requestThrottleMs => new FindCoordinatorResponse(error, Node.noNode)) + if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.GROUP && + !authorize(request.session, Describe, new Resource(Group, findCoordinatorRequest.coordinatorKey))) + sendErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED) + else if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.TRANSACTION && + !authorize(request.session, Describe, new Resource(TransactionalId, findCoordinatorRequest.coordinatorKey))) + sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) + else { // get metadata (and create the topic if necessary) val (partition, topicMetadata) = findCoordinatorRequest.coordinatorType match { case FindCoordinatorRequest.CoordinatorType.GROUP => @@ -1102,18 +1115,18 @@ class KafkaApis(val requestChannel: RequestChannel, val describeRequest = request.body[DescribeGroupsRequest] val groups = describeRequest.groupIds.asScala.map { groupId => - if (!authorize(request.session, Describe, new Resource(Group, groupId))) { - groupId -> DescribeGroupsResponse.GroupMetadata.forError(Errors.GROUP_AUTHORIZATION_FAILED) - } else { - val (error, summary) = groupCoordinator.handleDescribeGroup(groupId) - val members = summary.members.map { member => - val metadata = ByteBuffer.wrap(member.metadata) - val assignment = ByteBuffer.wrap(member.assignment) - new DescribeGroupsResponse.GroupMember(member.memberId, member.clientId, member.clientHost, metadata, assignment) - } - groupId -> new DescribeGroupsResponse.GroupMetadata(error, summary.state, summary.protocolType, - summary.protocol, members.asJava) + if (!authorize(request.session, Describe, new Resource(Group, groupId))) { + groupId -> DescribeGroupsResponse.GroupMetadata.forError(Errors.GROUP_AUTHORIZATION_FAILED) + } else { + val (error, summary) = groupCoordinator.handleDescribeGroup(groupId) + val members = summary.members.map { member => + val metadata = ByteBuffer.wrap(member.metadata) + val assignment = ByteBuffer.wrap(member.assignment) + new DescribeGroupsResponse.GroupMember(member.memberId, member.clientId, member.clientHost, metadata, assignment) } + groupId -> new DescribeGroupsResponse.GroupMetadata(error, summary.state, summary.protocolType, + summary.protocol, members.asJava) + } }.toMap sendResponseMaybeThrottle(request, requestThrottleMs => new DescribeGroupsResponse(requestThrottleMs, groups.asJava)) @@ -1414,31 +1427,36 @@ class KafkaApis(val requestChannel: RequestChannel, val initProducerIdRequest = request.body[InitProducerIdRequest] val transactionalId = initProducerIdRequest.transactionalId + def sendErrorResponse(error: Errors): Unit = { + sendResponseMaybeThrottle(request, requestThrottleMs => new InitProducerIdResponse(requestThrottleMs, error)) + } - if (!authorize(request.session, Write, Resource.ProducerIdResource)) { - sendResponseMaybeThrottle(request, requestThrottleMs => - new InitProducerIdResponse(requestThrottleMs, Errors.PRODUCER_ID_AUTHORIZATION_FAILED)) - } else if (transactionalId == null || authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId))) { - // Send response callback - def sendResponseCallback(result: InitProducerIdResult): Unit = { - def createResponse(requestThrottleMs: Int): AbstractResponse = { - val responseBody = new InitProducerIdResponse(requestThrottleMs, result.error, result.producerId, result.producerEpoch) - trace(s"Completed $transactionalId's InitProducerIdRequest with result $result from client ${request.header.clientId}.") - responseBody - } - sendResponseMaybeThrottle(request, createResponse) + if (transactionalId != null) { + if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId))) { + sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) + return } - txnCoordinator.handleInitProducerId(transactionalId, initProducerIdRequest.transactionTimeoutMs, sendResponseCallback) - }else - sendResponseMaybeThrottle(request, requestThrottleMs => - new InitProducerIdResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)) + } else if (!authorize(request.session, IdempotentWrite, Resource.ClusterResource)) { + sendErrorResponse(Errors.CLUSTER_AUTHORIZATION_FAILED) + return + } + + def sendResponseCallback(result: InitProducerIdResult): Unit = { + def createResponse(requestThrottleMs: Int): AbstractResponse = { + val responseBody = new InitProducerIdResponse(requestThrottleMs, result.error, result.producerId, result.producerEpoch) + trace(s"Completed $transactionalId's InitProducerIdRequest with result $result from client ${request.header.clientId}.") + responseBody + } + sendResponseMaybeThrottle(request, createResponse) + } + txnCoordinator.handleInitProducerId(transactionalId, initProducerIdRequest.transactionTimeoutMs, sendResponseCallback) } def handleEndTxnRequest(request: RequestChannel.Request): Unit = { val endTxnRequest = request.body[EndTxnRequest] val transactionalId = endTxnRequest.transactionalId - if(authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId))) { + if (authorize(request.session, Write, new Resource(TransactionalId, transactionalId))) { def sendResponseCallback(error: Errors) { def createResponse(requestThrottleMs: Int): AbstractResponse = { val responseBody = new EndTxnResponse(requestThrottleMs, error) @@ -1471,6 +1489,7 @@ class KafkaApis(val requestChannel: RequestChannel, } def sendResponseCallback(producerId: Long, result: TransactionResult)(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = { + trace(s"End transaction marker append for producer id $producerId completed with status: $responseStatus") errors.put(producerId, responseStatus.mapValues(_.error).asJava) val successfulOffsetsPartitions = responseStatus.filter { case (topicPartition, partitionResponse) => @@ -1524,9 +1543,9 @@ class KafkaApis(val requestChannel: RequestChannel, val transactionalId = addPartitionsToTxnRequest.transactionalId val partitionsToAdd = addPartitionsToTxnRequest.partitions - if(!authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId))) + if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId))) sendResponseMaybeThrottle(request, requestThrottleMs => - addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception())) + addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception)) else { val internalTopics = partitionsToAdd.asScala.filter {tp => org.apache.kafka.common.internals.Topic.isInternal(tp.topic())} @@ -1535,24 +1554,24 @@ class KafkaApis(val requestChannel: RequestChannel, authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic) } - val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { - tp => authorize(request.session, Write, new Resource(Topic, tp.topic)) + val unauthorizedForWriteRequestInfo = existingAndAuthorizedForDescribeTopics.filterNot { tp => + authorize(request.session, Write, new Resource(Topic, tp.topic)) } if (nonExistingOrUnauthorizedForDescribeTopics.nonEmpty || unauthorizedForWriteRequestInfo.nonEmpty || internalTopics.nonEmpty) { - // Only send back error responses for the partitions that failed. If there are any partition failures - // then the entire request fails - val partitionErrors = unauthorizedForWriteRequestInfo.map { tp => (tp, Errors.TOPIC_AUTHORIZATION_FAILED) }.toMap ++ - nonExistingOrUnauthorizedForDescribeTopics.map { tp => (tp, Errors.UNKNOWN_TOPIC_OR_PARTITION) }.toMap ++ - internalTopics.map { tp => (tp, Errors.TOPIC_AUTHORIZATION_FAILED) } + // Any failed partition check causes the entire request to fail. We only send back error responses + // for the partitions that failed to avoid needing to send an ambiguous error code for the partitions + // which succeeded. + val partitionErrors = (unauthorizedForWriteRequestInfo.map(_ -> Errors.TOPIC_AUTHORIZATION_FAILED) ++ + nonExistingOrUnauthorizedForDescribeTopics.map(_ -> Errors.UNKNOWN_TOPIC_OR_PARTITION) ++ + internalTopics.map(_ ->Errors.TOPIC_AUTHORIZATION_FAILED)).toMap sendResponseMaybeThrottle(request, requestThrottleMs => new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava)) } else { - // Send response callback def sendResponseCallback(error: Errors): Unit = { def createResponse(requestThrottleMs: Int): AbstractResponse = { val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs, @@ -1565,14 +1584,12 @@ class KafkaApis(val requestChannel: RequestChannel, } txnCoordinator.handleAddPartitionsToTransaction(transactionalId, - addPartitionsToTxnRequest.producerId(), - addPartitionsToTxnRequest.producerEpoch(), + addPartitionsToTxnRequest.producerId, + addPartitionsToTxnRequest.producerEpoch, partitionsToAdd.asScala.toSet, sendResponseCallback) } } - - } def handleAddOffsetsToTxnRequest(request: RequestChannel.Request): Unit = { @@ -1581,44 +1598,47 @@ class KafkaApis(val requestChannel: RequestChannel, val groupId = addOffsetsToTxnRequest.consumerGroupId val offsetTopicPartition = new TopicPartition(GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)) - if (!authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId))) + if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId))) sendResponseMaybeThrottle(request, requestThrottleMs => new AddOffsetsToTxnResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)) else if (!authorize(request.session, Read, new Resource(Group, groupId))) sendResponseMaybeThrottle(request, requestThrottleMs => new AddOffsetsToTxnResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED)) else { - // Send response callback - def sendResponseCallback(error: Errors): Unit = { - def createResponse(requestThrottleMs: Int): AbstractResponse = { - val responseBody: AddOffsetsToTxnResponse = new AddOffsetsToTxnResponse(requestThrottleMs, error) - trace(s"Completed $transactionalId's AddOffsetsToTxnRequest for group $groupId as on partition $offsetTopicPartition: errors: $error from client ${request.header.clientId}") - responseBody - } - sendResponseMaybeThrottle(request, createResponse) + def sendResponseCallback(error: Errors): Unit = { + def createResponse(requestThrottleMs: Int): AbstractResponse = { + val responseBody: AddOffsetsToTxnResponse = new AddOffsetsToTxnResponse(requestThrottleMs, error) + trace(s"Completed $transactionalId's AddOffsetsToTxnRequest for group $groupId on partition " + + s"$offsetTopicPartition: errors: $error from client ${request.header.clientId}") + responseBody } - - txnCoordinator.handleAddPartitionsToTransaction(transactionalId, - addOffsetsToTxnRequest.producerId, - addOffsetsToTxnRequest.producerEpoch, - Set(offsetTopicPartition), - sendResponseCallback) + sendResponseMaybeThrottle(request, createResponse) } - } + txnCoordinator.handleAddPartitionsToTransaction(transactionalId, + addOffsetsToTxnRequest.producerId, + addOffsetsToTxnRequest.producerEpoch, + Set(offsetTopicPartition), + sendResponseCallback) + } + } def handleTxnOffsetCommitRequest(request: RequestChannel.Request): Unit = { val header = request.header val txnOffsetCommitRequest = request.body[TxnOffsetCommitRequest] - // reject the request if not authorized to the group - if (!authorize(request.session, Read, new Resource(Group, txnOffsetCommitRequest.consumerGroupId))) { - val error = Errors.GROUP_AUTHORIZATION_FAILED - val results = txnOffsetCommitRequest.offsets.keySet.asScala.map { topicPartition => - (topicPartition, error) - }.toMap + + def sendErrorResponse(error: Errors): Unit = { sendResponseMaybeThrottle(request, requestThrottleMs => - new TxnOffsetCommitResponse(requestThrottleMs, results.asJava)) - } else { + txnOffsetCommitRequest.getErrorResponse(requestThrottleMs, error.exception)) + } + + // authorize for the transactionalId and the consumer group. Note that we skip producerId authorization + // since it is implied by transactionalId authorization + if (!authorize(request.session, Write, new Resource(TransactionalId, txnOffsetCommitRequest.transactionalId))) + sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) + else if (!authorize(request.session, Read, new Resource(Group, txnOffsetCommitRequest.consumerGroupId))) + sendErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED) + else { val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = txnOffsetCommitRequest.offsets.asScala.toMap.partition { case (topicPartition, _) => val authorizedForDescribe = authorize(request.session, Describe, new Resource(Topic, topicPartition.topic)) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 0198d38211e1..c46483401eb4 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -42,6 +42,7 @@ import org.apache.kafka.common.KafkaException import kafka.admin.AdminUtils import kafka.log.LogConfig import kafka.network.SocketServer +import org.apache.kafka.clients.consumer.OffsetAndMetadata import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} @@ -55,6 +56,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val createTopic = "topic-new" val deleteTopic = "topic-delete" val transactionalId = "transactional.id" + val producerId = 83392L val part = 0 val correlationId = 0 val clientId = "client-Id" @@ -64,22 +66,24 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val topicResource = new Resource(Topic, topic) val groupResource = new Resource(Group, group) val deleteTopicResource = new Resource(Topic, deleteTopic) - val producerTransactionalIdResource = new Resource(ProducerTransactionalId, transactionalId) + val transactionalIdResource = new Resource(TransactionalId, transactionalId) val groupReadAcl = Map(groupResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read))) + val groupDescribeAcl = Map(groupResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe))) val clusterAcl = Map(Resource.ClusterResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, ClusterAction))) val clusterCreateAcl = Map(Resource.ClusterResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Create))) + val clusterIdempotentWriteAcl = Map(Resource.ClusterResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, IdempotentWrite))) val topicReadAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read))) val topicWriteAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write))) val topicDescribeAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe))) val topicDeleteAcl = Map(deleteTopicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Delete))) val topicDescribeConfigsAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, DescribeConfigs))) val topicAlterConfigsAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, AlterConfigs))) - val producerTransactionalIdWriteAcl = Map(producerTransactionalIdResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write))) + val transactionIdWriteAcl = Map(transactionalIdResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write))) + val transactionalIdDescribeAcl = Map(transactionalIdResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe))) val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() - var transactionalProducer: KafkaProducer[Array[Byte], Array[Byte]] = _ val producerCount = 1 val consumerCount = 2 @@ -115,7 +119,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.DELETE_TOPICS -> classOf[requests.DeleteTopicsResponse], ApiKeys.OFFSET_FOR_LEADER_EPOCH -> classOf[OffsetsForLeaderEpochResponse], ApiKeys.DESCRIBE_CONFIGS -> classOf[DescribeConfigsResponse], - ApiKeys.ALTER_CONFIGS -> classOf[AlterConfigsResponse] + ApiKeys.ALTER_CONFIGS -> classOf[AlterConfigsResponse], + ApiKeys.INIT_PRODUCER_ID -> classOf[InitProducerIdResponse], + ApiKeys.WRITE_TXN_MARKERS -> classOf[WriteTxnMarkersResponse], + ApiKeys.ADD_PARTITIONS_TO_TXN -> classOf[AddPartitionsToTxnResponse], + ApiKeys.ADD_OFFSETS_TO_TXN -> classOf[AddOffsetsToTxnResponse], + ApiKeys.END_TXN -> classOf[EndTxnResponse], + ApiKeys.TXN_OFFSET_COMMIT -> classOf[TxnOffsetCommitResponse] ) val requestKeyToError = Map[ApiKeys, Nothing => Errors]( @@ -140,17 +150,23 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.DESCRIBE_CONFIGS -> ((resp: DescribeConfigsResponse) => resp.configs.get(new RResource(RResourceType.TOPIC, tp.topic)).error.error), ApiKeys.ALTER_CONFIGS -> ((resp: AlterConfigsResponse) => - resp.errors.get(new RResource(RResourceType.TOPIC, tp.topic)).error) + resp.errors.get(new RResource(RResourceType.TOPIC, tp.topic)).error), + ApiKeys.INIT_PRODUCER_ID -> ((resp: InitProducerIdResponse) => resp.error), + ApiKeys.WRITE_TXN_MARKERS -> ((resp: WriteTxnMarkersResponse) => resp.errors(producerId).get(tp)), + ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(tp)), + ApiKeys.ADD_OFFSETS_TO_TXN -> ((resp: AddOffsetsToTxnResponse) => resp.error), + ApiKeys.END_TXN -> ((resp: EndTxnResponse) => resp.error), + ApiKeys.TXN_OFFSET_COMMIT -> ((resp: TxnOffsetCommitResponse) => resp.errors.get(tp)) ) val requestKeysToAcls = Map[ApiKeys, Map[Resource, Set[Acl]]]( ApiKeys.METADATA -> topicDescribeAcl, - ApiKeys.PRODUCE -> topicWriteAcl, + ApiKeys.PRODUCE -> (topicWriteAcl ++ transactionIdWriteAcl ++ clusterIdempotentWriteAcl), ApiKeys.FETCH -> topicReadAcl, ApiKeys.LIST_OFFSETS -> topicDescribeAcl, ApiKeys.OFFSET_COMMIT -> (topicReadAcl ++ groupReadAcl), ApiKeys.OFFSET_FETCH -> (topicReadAcl ++ groupReadAcl), - ApiKeys.FIND_COORDINATOR -> (topicReadAcl ++ groupReadAcl), + ApiKeys.FIND_COORDINATOR -> (topicReadAcl ++ groupDescribeAcl ++ transactionalIdDescribeAcl), ApiKeys.UPDATE_METADATA_KEY -> clusterAcl, ApiKeys.JOIN_GROUP -> groupReadAcl, ApiKeys.SYNC_GROUP -> groupReadAcl, @@ -163,7 +179,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ApiKeys.DELETE_TOPICS -> topicDeleteAcl, ApiKeys.OFFSET_FOR_LEADER_EPOCH -> clusterAcl, ApiKeys.DESCRIBE_CONFIGS -> topicDescribeConfigsAcl, - ApiKeys.ALTER_CONFIGS -> topicAlterConfigsAcl + ApiKeys.ALTER_CONFIGS -> topicAlterConfigsAcl, + ApiKeys.INIT_PRODUCER_ID -> (transactionIdWriteAcl ++ clusterIdempotentWriteAcl), + ApiKeys.WRITE_TXN_MARKERS -> clusterAcl, + ApiKeys.ADD_PARTITIONS_TO_TXN -> (topicWriteAcl ++ transactionIdWriteAcl), + ApiKeys.ADD_OFFSETS_TO_TXN -> (groupReadAcl ++ transactionIdWriteAcl), + ApiKeys.END_TXN -> transactionIdWriteAcl, + ApiKeys.TXN_OFFSET_COMMIT -> (groupReadAcl ++ transactionIdWriteAcl) ) @Before @@ -177,14 +199,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest { maxBlockMs = 3000, acks = 1) - val transactionalProperties = new Properties() - transactionalProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - transactionalProperties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId) - transactionalProducer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), - retries = 3, - props = Some(transactionalProperties) - ) - for (_ <- 0 until consumerCount) consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT) @@ -204,7 +218,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producers.foreach(_.close()) consumers.foreach(_.wakeup()) consumers.foreach(_.close()) - transactionalProducer.close() removeAllAcls() super.tearDown() } @@ -338,17 +351,17 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ) for ((key, request) <- requestKeyToRequest) { - removeAllAcls + removeAllAcls() val resources = requestKeysToAcls(key).map(_._1.resourceType).toSet sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false) val resourceToAcls = requestKeysToAcls(key) - resourceToAcls.get(topicResource).map { acls => + resourceToAcls.get(topicResource).foreach { acls => val describeAcls = topicDescribeAcl(topicResource) val isAuthorized = describeAcls == acls addAndVerifyAcls(describeAcls, topicResource) sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true) - removeAllAcls + removeAllAcls() } for ((resource, acls) <- resourceToAcls) @@ -377,17 +390,17 @@ class AuthorizerIntegrationTest extends BaseRequestTest { ) for ((key, request) <- requestKeyToRequest) { - removeAllAcls + removeAllAcls() val resources = requestKeysToAcls(key).map(_._1.resourceType).toSet sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false, topicExists = false) val resourceToAcls = requestKeysToAcls(key) - resourceToAcls.get(topicResource).map { acls => + resourceToAcls.get(topicResource).foreach { acls => val describeAcls = topicDescribeAcl(topicResource) val isAuthorized = describeAcls == acls addAndVerifyAcls(describeAcls, topicResource) sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true, topicExists = false) - removeAllAcls + removeAllAcls() } for ((resource, acls) <- resourceToAcls) @@ -852,44 +865,182 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } @Test(expected = classOf[TransactionalIdAuthorizationException]) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnInitTransactions(): Unit = { - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource) - transactionalProducer.initTransactions() + def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(): Unit = { + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), transactionalIdResource) + val producer = buildTransactionalProducer() + producer.initTransactions() + } + + @Test(expected = classOf[TransactionalIdAuthorizationException]) + def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(): Unit = { + val producer = buildTransactionalProducer() + producer.initTransactions() + } + + @Test + def testSendOffsetsWithNoConsumerGroupDescribeAccess(): Unit = { + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, ClusterAction)), Resource.ClusterResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) + val producer = buildTransactionalProducer() + producer.initTransactions() + producer.beginTransaction() + try { + producer.sendOffsetsToTransaction(Map(new TopicPartition(topic, 0) -> new OffsetAndMetadata(0L)).asJava, group) + fail("Should have raised GroupAuthorizationException") + } catch { + case e: GroupAuthorizationException => + } + } + + @Test + def testSendOffsetsWithNoConsumerGroupWriteAccess(): Unit = { + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), groupResource) + val producer = buildTransactionalProducer() + producer.initTransactions() + producer.beginTransaction() + try { + producer.sendOffsetsToTransaction(Map(new TopicPartition(topic, 0) -> new OffsetAndMetadata(0L)).asJava, group) + fail("Should have raised GroupAuthorizationException") + } catch { + case e: GroupAuthorizationException => + } + } + + @Test + def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(): Unit = { + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) + val producer = buildIdempotentProducer() + try { + // the InitProducerId is sent asynchronously, so we expect the error either in the callback + // or raised from send itself + producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)).get() + fail("Should have raised ClusterAuthorizationException") + } catch { + case e: ExecutionException => + assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException]) + } + try { + // the second time, the call to send itself should fail (the producer becomes unusable + // if no producerId can be obtained) + producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)) + fail("Should have raised ClusterAuthorizationException") + } catch { + case e: KafkaException => + assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException]) + } + } + + @Test + def testIdempotentProducerNoIdempotentWriteAclInProduce(): Unit = { + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, IdempotentWrite)), Resource.ClusterResource) + + val producer = buildIdempotentProducer() + + // first send should be fine since we have permission to get a ProducerId + producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)).get() + + // revoke the IdempotentWrite permission + removeAllAcls() + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) + + try { + // the send should now fail with a cluster auth error + producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)).get() + fail("Should have raised ClusterAuthorizationException") + } catch { + case e: ExecutionException => + assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException]) + } + try { + // the second time, the call to send itself should fail (the producer becomes unusable + // if no producerId can be obtained) + producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)) + fail("Should have raised ClusterAuthorizationException") + } catch { + case e: KafkaException => + assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException]) + } } @Test def shouldInitTransactionsWhenAclSet(): Unit = { - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource) - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource) - transactionalProducer.initTransactions() + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) + val producer = buildTransactionalProducer() + producer.initTransactions() } + @Test + def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(): Unit = { + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) + // add describe access so that we can fetch metadata + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), topicResource) + val producer = buildTransactionalProducer() + producer.initTransactions() + producer.beginTransaction() + try { + producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get + Assert.fail("expected TopicAuthorizationException") + } catch { + case e: ExecutionException => + e.getCause match { + case cause: TopicAuthorizationException => + assertEquals(Set(topic), cause.unauthorizedTopics().asScala) + case other => + fail("Unexpected failure cause in send callback") + } + } + } + + @Test + def testTransactionalProducerTopicAuthorizationExceptionInCommit(): Unit = { + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) + // add describe access so that we can fetch metadata + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), topicResource) + val producer = buildTransactionalProducer() + producer.initTransactions() + producer.beginTransaction() + try { + producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)) + producer.commitTransaction() + Assert.fail("expected TopicAuthorizationException") + } catch { + case e: TopicAuthorizationException => + assertEquals(Set(topic), e.unauthorizedTopics().asScala) + } + } @Test def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(): Unit = { - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource) - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource) - transactionalProducer.initTransactions() + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) + val producer = buildTransactionalProducer() + producer.initTransactions() removeAllAcls() addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) try { - transactionalProducer.beginTransaction() - transactionalProducer.send(new ProducerRecord(tp.topic(), tp.partition(), "1".getBytes, "1".getBytes)).get + producer.beginTransaction() + producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get Assert.fail("expected TransactionalIdAuthorizationException") } catch { - case e: ExecutionException => assertTrue(s"expected TransactionalIdAuthorizationException, but got ${e.getCause}", e.getCause.isInstanceOf[TransactionalIdAuthorizationException]) + case e: ExecutionException => assertTrue(s"expected TransactionalIdAuthorizationException, but got ${e.getCause}", + e.getCause.isInstanceOf[TransactionalIdAuthorizationException]) } } @Test def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(): Unit = { - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource) - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource) - transactionalProducer.initTransactions() - transactionalProducer.beginTransaction() + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) + val producer = buildTransactionalProducer() + producer.initTransactions() + producer.beginTransaction() + producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get + producer.flush() removeAllAcls() try { - transactionalProducer.commitTransaction() + producer.commitTransaction() Assert.fail("expected TransactionalIdAuthorizationException") } catch { case _: TransactionalIdAuthorizationException => // ok @@ -898,50 +1049,27 @@ class AuthorizerIntegrationTest extends BaseRequestTest { @Test def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(): Unit = { - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource) - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource) addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), groupResource) - transactionalProducer.initTransactions() - transactionalProducer.beginTransaction() + val producer = buildTransactionalProducer() + producer.initTransactions() + producer.beginTransaction() removeAllAcls() try { - val offsets: util.Map[TopicPartition, OffsetAndMetadata] = Map(new TopicPartition(topicAndPartition.topic, topicAndPartition.partition) -> new OffsetAndMetadata(1L)).asJava - transactionalProducer.sendOffsetsToTransaction(offsets, group) + val offsets: util.Map[TopicPartition, OffsetAndMetadata] = Map(new TopicPartition(tp.topic, tp.partition) -> new OffsetAndMetadata(1L)).asJava + producer.sendOffsetsToTransaction(offsets, group) Assert.fail("expected TransactionalIdAuthorizationException") } catch { case _: TransactionalIdAuthorizationException => // ok } } - - @Test - def shouldThrowProducerIdAuthorizationExceptionWhenAclNotSet(): Unit = { - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) - val idempotentProperties = new Properties() - idempotentProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - val idempotentProducer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), - retries = 3, - props = Some(idempotentProperties) - ) - try { - idempotentProducer.send(new ProducerRecord(tp.topic(), tp.partition(), "1".getBytes, "1".getBytes)).get - Assert.fail("expected ProducerIdAuthorizationException") - } catch { - case e: ExecutionException => assertTrue(s"expected ProducerIdAuthorizationException, but got ${e.getCause}", e.getCause.isInstanceOf[ProducerIdAuthorizationException]) - } - } - @Test def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(): Unit = { - addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, IdempotentWrite)), Resource.ClusterResource) addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource) - val idempotentProperties = new Properties() - idempotentProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - val idempotentProducer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), - retries = 3, - props = Some(idempotentProperties) - ) - idempotentProducer.send(new ProducerRecord(tp.topic(), tp.partition(), "1".getBytes, "1".getBytes)).get + val producer = buildIdempotentProducer() + producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get } def removeAllAcls() = { @@ -1032,4 +1160,24 @@ class AuthorizerIntegrationTest extends BaseRequestTest { requests.OffsetFetchResponse.parse(response, request.version) } + private def buildTransactionalProducer(): KafkaProducer[Array[Byte], Array[Byte]] = { + val transactionalProperties = new Properties() + transactionalProperties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId) + val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + retries = 3, + props = Some(transactionalProperties)) + producers += producer + producer + } + + private def buildIdempotentProducer(): KafkaProducer[Array[Byte], Array[Byte]] = { + val idempotentProperties = new Properties() + idempotentProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") + val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + retries = 3, + props = Some(idempotentProperties)) + producers += producer + producer + } + } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index 5d463486ecb3..1b88f40caa68 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -77,17 +77,17 @@ class TransactionsBounceTest extends KafkaServerTestHarness { def testBrokerFailure() { // basic idea is to seed a topic with 10000 records, and copy it transactionally while bouncing brokers // constantly through the period. - val consumerGroup= "myGroup" + val consumerGroup = "myGroup" val numInputRecords = 5000 createTopics() TestUtils.seedTopicWithNumberedRecords(inputTopic, numInputRecords, servers) - val consumer = createConsumerAndSubscribeToTopics(consumerGroup, List(inputTopic)) - val producer = TestUtils.createTransactionalProducer("my-test-producer-t.id", servers) + val producer = TestUtils.createTransactionalProducer("test-txn", servers) - val scheduler = new BounceScheduler producer.initTransactions() + + val scheduler = new BounceScheduler scheduler.start() var numMessagesProcessed = 0 @@ -97,16 +97,17 @@ class TransactionsBounceTest extends KafkaServerTestHarness { val toRead = Math.min(200, numInputRecords - numMessagesProcessed) trace(s"$iteration: About to read $toRead messages, processed $numMessagesProcessed so far..") val records = TestUtils.pollUntilAtLeastNumRecords(consumer, toRead) - trace(s"received ${records.size} messages. sending them transactionally to $outputTopic") + trace(s"Received ${records.size} messages, sending them transactionally to $outputTopic") + producer.beginTransaction() - val shouldAbort = iteration % 2 == 0 - records.zipWithIndex.foreach { case (record, i) => - producer.send( - TestUtils.producerRecordWithExpectedTransactionStatus(outputTopic, record.key, record.value, !shouldAbort), - new ErrorLoggingCallback(outputTopic, record.key, record.value, true)) + val shouldAbort = iteration % 3 == 0 + records.foreach { record => + producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(outputTopic, record.key, record.value, + !shouldAbort), new ErrorLoggingCallback(outputTopic, record.key, record.value, true)) } trace(s"Sent ${records.size} messages. Committing offsets.") producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer), consumerGroup) + if (shouldAbort) { trace(s"Committed offsets. Aborting transaction of ${records.size} messages.") producer.abortTransaction() @@ -125,8 +126,8 @@ class TransactionsBounceTest extends KafkaServerTestHarness { scheduler.shutdown() - val verifyingConsumer = createConsumerAndSubscribeToTopics("randoGroup", List(outputTopic), readCommitted = true) - val outputRecords = TestUtils.pollUntilAtLeastNumRecords(verifyingConsumer, numInputRecords).map { case(record) => + val verifyingConsumer = createConsumerAndSubscribeToTopics("randomGroup", List(outputTopic), readCommitted = true) + val outputRecords = TestUtils.pollUntilAtLeastNumRecords(verifyingConsumer, numInputRecords).map { record => TestUtils.assertCommittedAndGetValue(record).toInt } val recordSet = outputRecords.toSet @@ -142,7 +143,7 @@ class TransactionsBounceTest extends KafkaServerTestHarness { val props = new Properties() if (readCommitted) props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed") - props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "200") + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "2000") props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") props.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "10000") props.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "3000") @@ -157,8 +158,8 @@ class TransactionsBounceTest extends KafkaServerTestHarness { private def createTopics() = { val topicConfig = new Properties() topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 2.toString) - TestUtils.createTopic(zkUtils, inputTopic, numPartitions, numServers, servers, topicConfig) - TestUtils.createTopic(zkUtils, outputTopic, numPartitions, numServers, servers, topicConfig) + TestUtils.createTopic(zkUtils, inputTopic, numPartitions, 3, servers, topicConfig) + TestUtils.createTopic(zkUtils, outputTopic, numPartitions, 3, servers, topicConfig) } private class BounceScheduler extends ShutdownableThread("daemon-broker-bouncer", false) { diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index ec6b3ea85a70..fd9d8846fa2d 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -120,15 +120,15 @@ class TransactionsTest extends KafkaServerTestHarness { consumer.subscribe(List(topic1)) producer.initTransactions() - val random = new Random() var shouldCommit = false var recordsProcessed = 0 try { while (recordsProcessed < numSeedMessages) { + val records = TestUtils.pollUntilAtLeastNumRecords(consumer, Math.min(10, numSeedMessages - recordsProcessed)) + producer.beginTransaction() shouldCommit = !shouldCommit - val records = TestUtils.pollUntilAtLeastNumRecords(consumer, Math.min(10, numSeedMessages - recordsProcessed)) records.zipWithIndex.foreach { case (record, i) => val key = new String(record.key(), "UTF-8") val value = new String(record.value(), "UTF-8") @@ -153,7 +153,7 @@ class TransactionsTest extends KafkaServerTestHarness { consumer.close() } - // Inspite of random aborts, we should still have exactly 1000 messages in topic2. Ie. we should not + // In spite of random aborts, we should still have exactly 1000 messages in topic2. Ie. we should not // re-copy or miss any messages from topic1, since the consumed offsets were committed transactionally. val verifyingConsumer = transactionalConsumer("foobargroup") verifyingConsumer.subscribe(List(topic2)) @@ -334,7 +334,6 @@ class TransactionsTest extends KafkaServerTestHarness { fail("Should not be able to send messages from a fenced producer.") } catch { case e : ProducerFencedException => - producer1.close() case e : ExecutionException => assertTrue(e.getCause.isInstanceOf[ProducerFencedException]) case e : Exception => diff --git a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala index f5b0a0674935..f379585b70e1 100644 --- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala @@ -36,26 +36,33 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { private val TopicResources = Set(new Resource(Topic, "test-1"), new Resource(Topic, "test-2")) private val GroupResources = Set(new Resource(Group, "testGroup-1"), new Resource(Group, "testGroup-2")) private val BrokerResources = Set(new Resource(Broker, "0"), new Resource(Broker, "1")) + private val TransactionalIdResources = Set(new Resource(TransactionalId, "t0"), new Resource(TransactionalId, "t1")) private val ResourceToCommand = Map[Set[Resource], Array[String]]( TopicResources -> Array("--topic", "test-1", "--topic", "test-2"), Set(Resource.ClusterResource) -> Array("--cluster"), GroupResources -> Array("--group", "testGroup-1", "--group", "testGroup-2"), - BrokerResources -> Array("--broker", "0", "--broker", "1") + BrokerResources -> Array("--broker", "0", "--broker", "1"), + TransactionalIdResources -> Array("--transactional-id", "t0", "--transactional-id", "t1") ) private val ResourceToOperations = Map[Set[Resource], (Set[Operation], Array[String])]( TopicResources -> (Set(Read, Write, Describe, Delete, DescribeConfigs, AlterConfigs), Array("--operation", "Read" , "--operation", "Write", "--operation", "Describe", "--operation", "Delete", "--operation", "DescribeConfigs", "--operation", "AlterConfigs")), - Set(Resource.ClusterResource) -> (Set(Create, ClusterAction), Array("--operation", "Create", "--operation", "ClusterAction")), + Set(Resource.ClusterResource) -> (Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite), + Array("--operation", "Create", "--operation", "ClusterAction", "--operation", "DescribeConfigs", + "--operation", "AlterConfigs", "--operation", "IdempotentWrite")), GroupResources -> (Set(Read, Describe), Array("--operation", "Read", "--operation", "Describe")), - BrokerResources -> (Set(DescribeConfigs), Array("--operation", "DescribeConfigs")) + BrokerResources -> (Set(DescribeConfigs), Array("--operation", "DescribeConfigs")), + TransactionalIdResources -> (Set(Describe, Write), Array("--operation", "Describe", "--operation", "Write")) ) - private val ProducerResourceToAcls = Map[Set[Resource], Set[Acl]]( + private def ProducerResourceToAcls(enableIdempotence: Boolean = false) = Map[Set[Resource], Set[Acl]]( TopicResources -> AclCommand.getAcls(Users, Allow, Set(Write, Describe), Hosts), - Set(Resource.ClusterResource) -> AclCommand.getAcls(Users, Allow, Set(Create), Hosts) + TransactionalIdResources -> AclCommand.getAcls(Users, Allow, Set(Write, Describe), Hosts), + Set(Resource.ClusterResource) -> AclCommand.getAcls(Users, Allow, Set(Some(Create), + if (enableIdempotence) Some(IdempotentWrite) else None).flatten, Hosts) ) private val ConsumerResourceToAcls = Map[Set[Resource], Set[Acl]]( @@ -64,10 +71,13 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { ) private val CmdToResourcesToAcl = Map[Array[String], Map[Set[Resource], Set[Acl]]]( - Array[String]("--producer") -> ProducerResourceToAcls, + Array[String]("--producer") -> ProducerResourceToAcls(), + Array[String]("--producer", "--idempotent") -> ProducerResourceToAcls(enableIdempotence = true), Array[String]("--consumer") -> ConsumerResourceToAcls, Array[String]("--producer", "--consumer") -> ConsumerResourceToAcls.map { case (k, v) => k -> (v ++ - ProducerResourceToAcls.getOrElse(k, Set.empty[Acl])) } + ProducerResourceToAcls().getOrElse(k, Set.empty[Acl])) }, + Array[String]("--producer", "--idempotent", "--consumer") -> ConsumerResourceToAcls.map { case (k, v) => k -> (v ++ + ProducerResourceToAcls(enableIdempotence = true).getOrElse(k, Set.empty[Acl])) } ) @Test @@ -108,11 +118,11 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { } } } - testRemove(resourcesToAcls.keys.flatten.toSet, resourceCommand, args, brokerProps) + testRemove(resourcesToAcls.keys.flatten.toSet, resourceCommand ++ cmd, args, brokerProps) } } - @Test (expected = classOf[IllegalArgumentException]) + @Test(expected = classOf[IllegalArgumentException]) def testInvalidAuthorizerProperty() { val args = Array("--authorizer-properties", "zookeeper.connect " + zkConnect) AclCommand.withAuthorizer(new AclCommandOptions(args))(null) @@ -120,10 +130,10 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging { private def testRemove(resources: Set[Resource], resourceCmd: Array[String], args: Array[String], brokerProps: Properties) { for (resource <- resources) { - AclCommand.main(args ++ resourceCmd :+ "--remove" :+ "--force") - withAuthorizer(brokerProps) { authorizer => - TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, resource) - } + AclCommand.main(args ++ resourceCmd :+ "--remove" :+ "--force") + withAuthorizer(brokerProps) { authorizer => + TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, resource) + } } } diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala similarity index 99% rename from core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala rename to core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala index 0a8209e9298c..358e12cfe493 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala @@ -38,7 +38,7 @@ import scala.collection._ import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future, Promise, TimeoutException} -class GroupCoordinatorResponseTest extends JUnitSuite { +class GroupCoordinatorTest extends JUnitSuite { type JoinGroupCallback = JoinGroupResult => Unit type SyncGroupCallbackParams = (Array[Byte], Errors) type SyncGroupCallback = (Array[Byte], Errors) => Unit diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala index 0e13f89ec587..2db6603f1cd8 100644 --- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala @@ -455,7 +455,7 @@ class GroupMetadataTest extends JUnitSuite { assertTrue(group.hasPendingOffsetCommitsFromProducer(producerId)) assertTrue(group.hasOffsets) assertEquals(None, group.offset(partition)) - group.failPendingTxnOffsetCommit(producerId, partition, txnOffsetCommit) + group.failPendingTxnOffsetCommit(producerId, partition) assertFalse(group.hasOffsets) assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId)) diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 7e5004976d54..fa2e55b53486 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -250,19 +250,20 @@ class RequestQuotaTest extends BaseRequestTest { new OffsetsForLeaderEpochRequest.Builder().add(tp, 0) case ApiKeys.ADD_PARTITIONS_TO_TXN => - new AddPartitionsToTxnRequest.Builder("txn1", 1, 0, List(tp).asJava) + new AddPartitionsToTxnRequest.Builder("test-transactional-id", 1, 0, List(tp).asJava) case ApiKeys.ADD_OFFSETS_TO_TXN => - new AddOffsetsToTxnRequest.Builder("txn1", 1, 0, "test-txn-group") + new AddOffsetsToTxnRequest.Builder("test-transactional-id", 1, 0, "test-txn-group") case ApiKeys.END_TXN => - new EndTxnRequest.Builder("txn1", 1, 0, TransactionResult.forId(false)) + new EndTxnRequest.Builder("test-transactional-id", 1, 0, TransactionResult.forId(false)) case ApiKeys.WRITE_TXN_MARKERS => new WriteTxnMarkersRequest.Builder(List.empty.asJava) case ApiKeys.TXN_OFFSET_COMMIT => - new TxnOffsetCommitRequest.Builder("test-txn-group", 2, 0, Map.empty.asJava) + new TxnOffsetCommitRequest.Builder("test-transactional-id", "test-txn-group", 2, 0, + Map.empty[TopicPartition, TxnOffsetCommitRequest.CommittedOffset].asJava) case ApiKeys.DESCRIBE_ACLS => new DescribeAclsRequest.Builder(AclBindingFilter.ANY) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 70c340ba1657..054a4ff4c115 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -1340,21 +1340,19 @@ object TestUtils extends Logging { } // Seeds the given topic with records with keys and values in the range [0..numRecords) - def seedTopicWithNumberedRecords(topic: String, numRecords: Int, servers: Seq[KafkaServer]): Int = { + def seedTopicWithNumberedRecords(topic: String, numRecords: Int, servers: Seq[KafkaServer]): Unit = { val props = new Properties() props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") - var recordsWritten = 0 - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), retries = Integer.MAX_VALUE, acks = -1, props = Some(props)) + val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), + retries = Integer.MAX_VALUE, acks = -1, props = Some(props)) try { for (i <- 0 until numRecords) { producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, asBytes(i.toString), asBytes(i.toString))) - recordsWritten += 1 } producer.flush() } finally { producer.close() } - recordsWritten } private def asString(bytes: Array[Byte]) = new String(bytes, StandardCharsets.UTF_8) @@ -1404,7 +1402,7 @@ object TestUtils extends Logging { offsetsToCommit.toMap } - def pollUntilAtLeastNumRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) : Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = { + def pollUntilAtLeastNumRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int): Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = { val records = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]() TestUtils.waitUntilTrue(() => { records ++= consumer.poll(50)