You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "jolshan (via GitHub)" <gi...@apache.org> on 2023/02/10 19:18:22 UTC

[GitHub] [kafka] jolshan opened a new pull request, #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

jolshan opened a new pull request, #13231:
URL: https://github.com/apache/kafka/pull/13231

   I've updated the API spec and related classes.
   
   Clients should only be able to send up to version 3 requests and that is enforced by using the old Builder and forcing the older builder to build version 3 or below. 
   
   Requests > 4 only require cluster permissions as they are initiated from other brokers.
   
   I've added tests for the batched requests and for the verifyOnly mode.
   
   Also -- minor change to the KafkaApis method to properly match the request name.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107618273


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   sorry, i meant the return value `List<TopicPartition>`, it doesn't seem to be used



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107760565


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -49,28 +52,37 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
     private final AddPartitionsToTxnResponseData data;
 
     private Map<TopicPartition, Errors> cachedErrorsMap = null;
+    
+    private Map<String, Map<TopicPartition, Errors>> cachedAllErrorsMap = null;
 
     public AddPartitionsToTxnResponse(AddPartitionsToTxnResponseData data) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN);
         this.data = data;
     }
 
+    // Only used for versions < 4

Review Comment:
   Yeah, the problem is that it makes the usage more confusing. I'd rather have a larger patch I guess and try to abstract some of the version details away.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107759481


##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -352,7 +353,12 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
               // this is an optimization: if the partitions are already in the metadata reply OK immediately
               Left(Errors.NONE)
             } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
+              // If verifyOnly, we should have returned in the step above. If we didn't the partitions are not present in the transaction.
+              if (verifyOnly) {
+                Left(Errors.INVALID_TXN_STATE)

Review Comment:
   You bring up good points and I can give more detailed response on partition level. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1120986902


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   Is the request to make a mapping of every topic partition to the same error? That doesn't seem super efficient, but I guess it makes things simpler in the parameters. We would also need to define a new callback in `handleAddPartitionsToTransaction` that calls the original callback (so we have two still)
   
   I suppose this is doable though. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110442122


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   Yup that was my point here:
   > One benefit is that verifyOnly requests won't be slowed down by the non-verify only ones
   But yeah, I guess we slow things down as well by using separate requests.



##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   Yup that was my point here:
   > One benefit is that verifyOnly requests won't be slowed down by the non-verify only ones
   
   But yeah, I guess we slow things down as well by using separate requests. So maybe unavoidable



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123526676


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   Ok.
   I will remove the errors from the response
   When using the new API it is mandatory I check the top level error before any further handling. 
   I may also need to adjust tests/test this functionality.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110062885


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   I was wondering if having `VerifyOnly` as top level field is the right thing to do. It basically means that we cannot batch transactions with `VerifyOnly=true` and transactions with `VerifyOnly=false` in the same request. This may impact out batching mechanism. Thoughts?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107587497


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {
+        if (cachedPartitionsByTransaction == null) {
+            cachedPartitionsByTransaction = new HashMap<>();
+        }
+        
+        return cachedPartitionsByTransaction.computeIfAbsent(transaction, txn -> {
+            List<TopicPartition> partitions = new ArrayList<>();
+            for (AddPartitionsToTxnTopic topicCollection : data.transactions().find(txn).topics()) {
+                for (Integer partition : topicCollection.partitions()) {
+                    partitions.add(new TopicPartition(topicCollection.name(), partition));
+                }
+            }
+            return partitions;
+        });
+    }
+    
+    public Map<String, List<TopicPartition>> partitionsByTransaction() {
+        if (cachedPartitionsByTransaction != null && cachedPartitionsByTransaction.size() == data.transactions().size()) {

Review Comment:
   I'm not sure how that would happen



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107606857


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4
+        public Builder(String transactionalId,
+                       long producerId,
+                       short producerEpoch,
+                       List<TopicPartition> partitions) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
-            this.data = data;
+            this.isClientRequest = true;
+
+            AddPartitionsToTxnTopicCollection topics = compileTopics(partitions);
+
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactionalId(transactionalId)
+                    .setProducerId(producerId)
+                    .setProducerEpoch(producerEpoch)
+                    .setTopics(topics);
         }
 
-        public Builder(final String transactionalId,
-                       final long producerId,
-                       final short producerEpoch,
-                       final List<TopicPartition> partitions) {
+        public Builder(AddPartitionsToTxnTransactionCollection transactions,
+                       boolean verifyOnly) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
+            this.isClientRequest = false;
 
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactions(transactions)
+                    .setVerifyOnly(verifyOnly);
+        }
+
+        private AddPartitionsToTxnTopicCollection compileTopics(final List<TopicPartition> partitions) {

Review Comment:
   nit: how about `buildTxnTopicCollection`?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4
+        public Builder(String transactionalId,
+                       long producerId,
+                       short producerEpoch,
+                       List<TopicPartition> partitions) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
-            this.data = data;
+            this.isClientRequest = true;
+
+            AddPartitionsToTxnTopicCollection topics = compileTopics(partitions);
+
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactionalId(transactionalId)

Review Comment:
   Does it make sense to set verifyOnly to false explicitly here?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -66,24 +97,22 @@ public Builder(final String transactionalId,
             AddPartitionsToTxnTopicCollection topics = new AddPartitionsToTxnTopicCollection();
             for (Map.Entry<String, List<Integer>> partitionEntry : partitionMap.entrySet()) {
                 topics.add(new AddPartitionsToTxnTopic()
-                               .setName(partitionEntry.getKey())
-                               .setPartitions(partitionEntry.getValue()));
+                        .setName(partitionEntry.getKey())
+                        .setPartitions(partitionEntry.getValue()));
             }
-
-            this.data = new AddPartitionsToTxnRequestData()
-                            .setTransactionalId(transactionalId)
-                            .setProducerId(producerId)
-                            .setProducerEpoch(producerEpoch)
-                            .setTopics(topics);
+            return topics;
         }
 
         @Override
         public AddPartitionsToTxnRequest build(short version) {
-            return new AddPartitionsToTxnRequest(data, version);
+            short clampedVersion = (isClientRequest && version > 3) ? 3 : version;

Review Comment:
   It's a little strange to ignore the version. I think another way to do this is to set the `latestAllowedVersion` to 3 in the client builder. That will ensure that the client does not try to use a higher version even if the broker supports it. Similarly, we can set a min version of 4 for the server.



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +193,41 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        if (version < 4) {
+            final HashMap<TopicPartition, Errors> errors = new HashMap<>();
+            for (TopicPartition partition : partitions()) {
+                errors.put(partition, error);
+            }
+            return new AddPartitionsToTxnResponse(throttleTimeMs, errors);
+        } else {
+            AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();

Review Comment:
   The logic in here makes me wonder if we should add a top-level error code in the response.



##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -352,7 +353,12 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
               // this is an optimization: if the partitions are already in the metadata reply OK immediately
               Left(Errors.NONE)
             } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
+              // If verifyOnly, we should have returned in the step above. If we didn't the partitions are not present in the transaction.
+              if (verifyOnly) {
+                Left(Errors.INVALID_TXN_STATE)

Review Comment:
   Suppose that some of the partitions are added to the transaction and some are not. We return `INVALID_TXN_STATE` which means the broker cannot distinguish which partitions have been added correctly. Does it matter? I think there are two main consequences:
   1. We have to reject the full `Produce` request, which may contain writes to multiple partitions. This means the producer also cannot tell which partition was not added correctly. Maybe this is fine since the producer must abort anyway?
   2. The `AddPartitionsToTxn` cannot batch across multiple `Produce` requests from the same `transactionalId`. Or, if it does batch across requests, then we would have to return INVALID_TXN_STATE in all of the responses. Maybe this is also fine for the same reason?
   
   I do think we'll want to have at least in the logs somewhere which partitions were not present in the transaction.



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4

Review Comment:
   Perhaps we could create factory methods: `Builder.forClient` and `Builder.forServer` (or something like that).



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   nit: could we use `transactionalId` as the argument name?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -99,6 +112,7 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) {
         data.setThrottleTimeMs(throttleTimeMs);
     }
 
+    // Only used for versions < 4

Review Comment:
   Do we need to continue exposing some of these version-specific methods? We sort of intend the request/response objects to abstract a lot of the version-specific details.



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -49,28 +52,37 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
     private final AddPartitionsToTxnResponseData data;
 
     private Map<TopicPartition, Errors> cachedErrorsMap = null;
+    
+    private Map<String, Map<TopicPartition, Errors>> cachedAllErrorsMap = null;
 
     public AddPartitionsToTxnResponse(AddPartitionsToTxnResponseData data) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN);
         this.data = data;
     }
 
+    // Only used for versions < 4

Review Comment:
   I do wonder if we need both constructors. The old version is a subset of the new version where the number of producers happens to be 1.



##########
clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json:
##########
@@ -22,22 +22,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
       "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
-    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
-      "about": "The results for each topic.", "fields": [
+    { "name": "ResultsByTransaction", "type": "[]AddPartitionsToTxnResult", "versions": "4+",
+      "about": "Results categorized by transactional ID.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction."},
+      { "name": "TopicResults", "type": "[]AddPartitionsToTxnTopicResult", "versions": "4+",
+        "about": "The results for each topic." }
+    ]},
+    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0-3",

Review Comment:
   I wonder if it makes sense to use a different naming convention for the fields which are being removed. For example, instead of `Results` we could have `ResultsV3AndBelow`. It is not pretty, but it does make the usage in the code clearer.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110117577


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   One benefit is that verifyOnly requests won't be slowed down by the non-verify only ones



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107748914


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +193,41 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        if (version < 4) {
+            final HashMap<TopicPartition, Errors> errors = new HashMap<>();
+            for (TopicPartition partition : partitions()) {
+                errors.put(partition, error);
+            }
+            return new AddPartitionsToTxnResponse(throttleTimeMs, errors);
+        } else {
+            AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();

Review Comment:
   There is 1 error we can return as top level errors in code (and I suppose any thrown errors will be top level, but I don't think that is common) if that helps. The rest are topic or partition level.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1128156639


##########
core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala:
##########
@@ -231,7 +231,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       resp.errors.get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)).error),
     ApiKeys.INIT_PRODUCER_ID -> ((resp: InitProducerIdResponse) => resp.error),
     ApiKeys.WRITE_TXN_MARKERS -> ((resp: WriteTxnMarkersResponse) => resp.errorsByProducerId.get(producerId).get(tp)),
-    ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(tp)),
+    ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID).get(tp)),

Review Comment:
   I guess it uses `authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107592032


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   I'm not sure I understand the question.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107748914


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +193,41 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        if (version < 4) {
+            final HashMap<TopicPartition, Errors> errors = new HashMap<>();
+            for (TopicPartition partition : partitions()) {
+                errors.put(partition, error);
+            }
+            return new AddPartitionsToTxnResponse(throttleTimeMs, errors);
+        } else {
+            AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();

Review Comment:
   There a few errors we can return as top level errors in code and that's for auth to the transaction (client side only), ensuring ibp, and cluster auth (for server side only)  if that helps. The rest are topic or partition level.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107759662


##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -352,7 +353,12 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
               // this is an optimization: if the partitions are already in the metadata reply OK immediately
               Left(Errors.NONE)
             } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
+              // If verifyOnly, we should have returned in the step above. If we didn't the partitions are not present in the transaction.
+              if (verifyOnly) {
+                Left(Errors.INVALID_TXN_STATE)

Review Comment:
   I think this was mostly an oversight on my part.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1124830314


##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponseTest.java:
##########
@@ -58,42 +65,75 @@ public void setUp() {
         errorsMap.put(tp2, errorTwo);
     }
 
-    @Test
-    public void testConstructorWithErrorResponse() {
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(throttleTimeMs, errorsMap);
-
-        assertEquals(expectedErrorCounts, response.errorCounts());
-        assertEquals(throttleTimeMs, response.throttleTimeMs());
-    }
-
-    @Test
-    public void testParse() {
-
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.ADD_PARTITIONS_TO_TXN)
+    public void testParse(short version) {
         AddPartitionsToTxnTopicResultCollection topicCollection = new AddPartitionsToTxnTopicResultCollection();
 
         AddPartitionsToTxnTopicResult topicResult = new AddPartitionsToTxnTopicResult();
         topicResult.setName(topicOne);
 
-        topicResult.results().add(new AddPartitionsToTxnPartitionResult()
-                                      .setErrorCode(errorOne.code())
+        topicResult.resultsByPartition().add(new AddPartitionsToTxnPartitionResult()
+                                      .setPartitionErrorCode(errorOne.code())
                                       .setPartitionIndex(partitionOne));
 
-        topicResult.results().add(new AddPartitionsToTxnPartitionResult()
-                                      .setErrorCode(errorTwo.code())
+        topicResult.resultsByPartition().add(new AddPartitionsToTxnPartitionResult()
+                                      .setPartitionErrorCode(errorTwo.code())
                                       .setPartitionIndex(partitionTwo));
 
         topicCollection.add(topicResult);
+            
+        if (version < 4) {
+            AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
+                    .setResultsByTopicV3AndBelow(topicCollection)
+                    .setThrottleTimeMs(throttleTimeMs);
+            AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
 
-        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
-                                                  .setResults(topicCollection)
-                                                  .setThrottleTimeMs(throttleTimeMs);
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
-
-        for (short version : ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions()) {
             AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serialize(version), version);
             assertEquals(expectedErrorCounts, parsedResponse.errorCounts());
             assertEquals(throttleTimeMs, parsedResponse.throttleTimeMs());
             assertEquals(version >= 1, parsedResponse.shouldClientThrottle(version));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            results.add(new AddPartitionsToTxnResult().setTransactionalId("txn1").setTopicResults(topicCollection));
+            
+            // Create another transaction with new name and errorOne for a single partition.
+            Map<TopicPartition, Errors> txnTwoExpectedErrors = Collections.singletonMap(tp2, errorOne);
+            results.add(AddPartitionsToTxnResponse.resultForTransaction("txn2", txnTwoExpectedErrors));
+
+            AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
+                    .setResultsByTransaction(results)
+                    .setThrottleTimeMs(throttleTimeMs);
+            AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
+
+            Map<Errors, Integer> newExpectedErrorCounts = new HashMap<>();
+            newExpectedErrorCounts.put(Errors.NONE, 1); // top level error
+            newExpectedErrorCounts.put(errorOne, 2);
+            newExpectedErrorCounts.put(errorTwo, 1);
+            
+            AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serialize(version), version);
+            assertEquals(txnTwoExpectedErrors, errorsForTransaction(response.getTransactionTopicResults("txn2")));
+            assertEquals(newExpectedErrorCounts, parsedResponse.errorCounts());
+            assertEquals(throttleTimeMs, parsedResponse.throttleTimeMs());
+            assertTrue(parsedResponse.shouldClientThrottle(version));
         }
     }
+    
+    @Test
+    public void testBatchedErrors() {
+        Map<TopicPartition, Errors> txn1Errors = Collections.singletonMap(tp1, errorOne);
+        Map<TopicPartition, Errors> txn2Errors = Collections.singletonMap(tp1, errorOne);
+        
+        AddPartitionsToTxnResult transaction1 = AddPartitionsToTxnResponse.resultForTransaction("txn1", txn1Errors);
+        AddPartitionsToTxnResult transaction2 = AddPartitionsToTxnResponse.resultForTransaction("txn2", txn2Errors);
+        
+        AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+        results.add(transaction1);
+        results.add(transaction2);
+        
+        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setResultsByTransaction(results));
+        
+        assertEquals(txn1Errors, errorsForTransaction(response.getTransactionTopicResults("txn1")));
+        assertEquals(txn2Errors, errorsForTransaction(response.getTransactionTopicResults("txn2")));
+    }

Review Comment:
   I tested in other tests, but I can put one here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1120976648


##########
clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json:
##########
@@ -22,22 +22,37 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds support to batch multiple transactions and a top level error code.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
       "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
-    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
-      "about": "The results for each topic.", "fields": [
+    { "name": "ErrorCode", "type": "int16", "versions": "4+",

Review Comment:
   Sounds fine to me.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118946985


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,39 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  // The AddPartitionsToTxnRequest version 4 API is added as part of KIP-890 and is still
+  // under developement. Hence, the API is not exposed by default by brokers
+  // unless explicitely enabled.
+  "latestVersionUnstable": true,
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
-      "about": "The transactional id corresponding to the transaction."},
-    { "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId",
+    { "name": "Transactions", "type": "[]AddPartitionsToTxnTransaction", "versions":  "4+",
+      "about": "List of transactions to add partitions to.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction." },
+      { "name": "ProducerId", "type": "int64", "versions": "4+", "entityType": "producerId",
+        "about": "Current producer id in use by the transactional id." },
+      { "name": "ProducerEpoch", "type": "int16", "versions": "4+",
+        "about": "Current epoch associated with the producer id." },
+      { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,
+        "about": "Boolean to signify if we want to check if the partition is in the transaction rather than add it." },
+      { "name": "Topics", "type": "[]AddPartitionsToTxnTopic", "versions": "4+",
+        "about": "The partitions to add to the transaction." }
+    ]},
+    { "name": "V3AndBelowTransactionalId", "type": "string", "versions": "0-3", "entityType": "transactionalId",

Review Comment:
   I had previous comments from @hachikuji and @guozhangwang that I should do something like this to avoid confusion when setting the fields. I have seen something similiar in other places but we didn't explicitly say the version.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123508496


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1328,7 +1328,7 @@ Priority priority() {
         @Override
         public void handleResponse(AbstractResponse response) {
             AddPartitionsToTxnResponse addPartitionsToTxnResponse = (AddPartitionsToTxnResponse) response;
-            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors();
+            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID);

Review Comment:
   I was told not to have v3 and below specific methods from Jason because the v3 case should generalize to a single version of the v4 case and that should make it easy to use methods for both.
   
   However, if we really think this is an issue. I guess we can change the approach again. I'm just not sure the experience of errors only applying to v4+. Any ideas there besides changing the method name to express it should only be used in v4+?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123514453


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   I was looking at the AlterPartition API as a reference. There, when we have a top level error, we don't set the partitions in the response at all. We basically save space. See [here](https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/requests/AlterPartitionRequest.java#L46).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123765294


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   Yup. This will only be done in v4 responses. So for now, we don't do any checks since the server side usage is upcoming in the next PR.
   
   As a side note, seems that AlterPartition includes the top level error in errorCounts so I did that here as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on PR #13231:
URL: https://github.com/apache/kafka/pull/13231#issuecomment-1441071519

   I've added the changes to the API spec
   -- verify only is now a transaction level config
   -- top level error is added to the response
   
   I've added builders to the request and tried to simplify some of the methods.
   I've also addressed the verifyOnly case where some partitions are in the txn and others are not. 
   
   I will update the KIP to reflect some of these changes (especially with respect to the API spec)
   
   I still need to address the unstable API change, but that will require a pull from master.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107785855


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -99,6 +112,7 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) {
         data.setThrottleTimeMs(throttleTimeMs);
     }
 
+    // Only used for versions < 4

Review Comment:
   I guess if we want to deal with a map or larger data structure that has only one object we can. I think the tricky thing is knowing when we expect just one response how to use it correctly. That may require further refactoring but it is probably doable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107590667


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2383,68 +2384,101 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val lock = new Object

Review Comment:
   i can do that



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107762271


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -99,6 +112,7 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) {
         data.setThrottleTimeMs(throttleTimeMs);
     }
 
+    // Only used for versions < 4

Review Comment:
   As mentioned elsewhere, the old API is a special case of the new API where there just happens to be a single transactionalId. I think if we take that perspective, then we can make the methods general and hide the version details.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1125017580


##########
clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java:
##########
@@ -2598,12 +2604,37 @@ private OffsetsForLeaderEpochResponse createLeaderEpochResponse() {
     }
 
     private AddPartitionsToTxnRequest createAddPartitionsToTxnRequest(short version) {
-        return new AddPartitionsToTxnRequest.Builder("tid", 21L, (short) 42,
-            singletonList(new TopicPartition("topic", 73))).build(version);
+        if (version < 4) {
+            return AddPartitionsToTxnRequest.Builder.forClient("tid", 21L, (short) 42,
+                    singletonList(new TopicPartition("topic", 73))).build(version);
+        } else {
+            AddPartitionsToTxnTransactionCollection transactions = new AddPartitionsToTxnTransactionCollection(
+                    singletonList(new AddPartitionsToTxnTransaction()

Review Comment:
   It's a bit too long for the previous line though. I made it 4 instead of 8 spaces.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118951636


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   It is not the same. In one case we have a single error for all topic partitions, but in the other, we may have different errors per partition. (If we verify and have some that exist and others that don't)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118952253


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   I had to keep the single error because this callback is used in the append path and I didn't want to change it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1122927383


##########
clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java:
##########
@@ -1303,11 +1305,13 @@ public void testCommitWithTopicAuthorizationFailureInAddPartitionsInFlight() thr
         Map<TopicPartition, Errors> errors = new HashMap<>();
         errors.put(tp0, Errors.TOPIC_AUTHORIZATION_FAILED);
         errors.put(tp1, Errors.OPERATION_NOT_ATTEMPTED);
+        AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors);
+        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0);
         client.respond(body -> {
             AddPartitionsToTxnRequest request = (AddPartitionsToTxnRequest) body;
-            assertEquals(new HashSet<>(request.partitions()), new HashSet<>(errors.keySet()));
+            assertEquals(new HashSet<>(AddPartitionsToTxnRequest.getPartitions(request.data().v3AndBelowTopics())), new HashSet<>(errors.keySet()));

Review Comment:
   I tend to agree with this. Are those helpers only used in tests? If so, it may be better to put them in `AddPartitionsToTxnRequestTest`. What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123529173


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   Don't forget to take the version into account here. We can only do this for v4.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123527995


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1328,7 +1328,7 @@ Priority priority() {
         @Override
         public void handleResponse(AbstractResponse response) {
             AddPartitionsToTxnResponse addPartitionsToTxnResponse = (AddPartitionsToTxnResponse) response;
-            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors();
+            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID);

Review Comment:
   Understood. Let's keep it as it is then.
   
   I agree that v3 case should generalized to a single item of the v4 case. It is just unfortunate that we don't have the transaction id in v3 response so we have to use an empty string for it. I suppose that it is the way it is.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1120990845


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()

Review Comment:
   Maybe. But the parameters may be tricky. 😅 The maybeSendResponse was supposed to be the helper. But I guess it's not helpful enough.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110139153


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   Right. At the broker level, I think that it is possible to have old and new producers, right? If we use one connection from the broker to the transaction coordinator, it means that one has to wait on the other. I don't know how you plan to implement this so it is just a thought.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110270761


##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -352,7 +353,12 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
               // this is an optimization: if the partitions are already in the metadata reply OK immediately
               Left(Errors.NONE)
             } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
+              // If verifyOnly, we should have returned in the step above. If we didn't the partitions are not present in the transaction.
+              if (verifyOnly) {
+                Left(Errors.INVALID_TXN_STATE)

Review Comment:
   Ack -- we may make verifyOnly not a global field, but the point still stands.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118949475


##########
clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java:
##########
@@ -1303,11 +1305,13 @@ public void testCommitWithTopicAuthorizationFailureInAddPartitionsInFlight() thr
         Map<TopicPartition, Errors> errors = new HashMap<>();
         errors.put(tp0, Errors.TOPIC_AUTHORIZATION_FAILED);
         errors.put(tp1, Errors.OPERATION_NOT_ATTEMPTED);
+        AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors);
+        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0);
         client.respond(body -> {
             AddPartitionsToTxnRequest request = (AddPartitionsToTxnRequest) body;
-            assertEquals(new HashSet<>(request.partitions()), new HashSet<>(errors.keySet()));
+            assertEquals(new HashSet<>(AddPartitionsToTxnRequest.getPartitions(request.data().v3AndBelowTopics())), new HashSet<>(errors.keySet()));

Review Comment:
   I was told to avoid having version specific methods. In order to use by both the v3 and below, and v4 objects this was the best way to do it. It really only looks bad in the tests.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118980264


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   > It is not the same. In one case we have a single error for all topic partitions, but in the other, we may have different errors per partition. (If we verify and have some that exist and others that don't)
   
   Yeah, I understand this. My point was that on the non-verify case, we also end up having an error per partition in the response (the same one for all of them).
   
   > I had to keep the single error because this callback is used in the append path and I didn't want to change it.
   
   Hum... I think that we could handle this in `handleAddPartitionsToTransaction` without changing the append path. Basically, when an error is return by the append, we could build a map with the error for all partitions.
   
   I think that having two callbacks is a bad design choice here but that's just my humble opinion.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118380508


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -34,22 +43,40 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
 
     private final AddPartitionsToTxnRequestData data;
 
-    private List<TopicPartition> cachedPartitions = null;
+    private final short version;

Review Comment:
   The version is already in the base class. Do we really need it here?



##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1328,7 +1328,7 @@ Priority priority() {
         @Override
         public void handleResponse(AbstractResponse response) {
             AddPartitionsToTxnResponse addPartitionsToTxnResponse = (AddPartitionsToTxnResponse) response;
-            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors();
+            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID);

Review Comment:
   nit: I suppose that `errors` should never be `null` here. I wonder if we should still check it. What do you think?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -34,22 +43,40 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
 
     private final AddPartitionsToTxnRequestData data;
 
-    private List<TopicPartition> cachedPartitions = null;
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        
+        public static Builder forClient(String transactionalId,
+                                        long producerId,
+                                        short producerEpoch,
+                                        List<TopicPartition> partitions) {
+
+            AddPartitionsToTxnTopicCollection topics = buildTxnTopicCollection(partitions);
+            
+            return new Builder(ApiKeys.ADD_PARTITIONS_TO_TXN.oldestVersion(),
+                    (short) 3, 
+                new AddPartitionsToTxnRequestData()
+                    .setV3AndBelowTransactionalId(transactionalId)
+                    .setV3AndBelowProducerId(producerId)
+                    .setV3AndBelowProducerEpoch(producerEpoch)
+                    .setV3AndBelowTopics(topics));

Review Comment:
   nit: The indentation of the arguments looks inconsistent.



##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,39 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.

Review Comment:
   I think that we should explain that v4 is only for other brokers and clients are suppose to use version <= v3.



##########
clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java:
##########
@@ -1303,11 +1305,13 @@ public void testCommitWithTopicAuthorizationFailureInAddPartitionsInFlight() thr
         Map<TopicPartition, Errors> errors = new HashMap<>();
         errors.put(tp0, Errors.TOPIC_AUTHORIZATION_FAILED);
         errors.put(tp1, Errors.OPERATION_NOT_ATTEMPTED);
+        AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors);
+        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0);
         client.respond(body -> {
             AddPartitionsToTxnRequest request = (AddPartitionsToTxnRequest) body;
-            assertEquals(new HashSet<>(request.partitions()), new HashSet<>(errors.keySet()));
+            assertEquals(new HashSet<>(AddPartitionsToTxnRequest.getPartitions(request.data().v3AndBelowTopics())), new HashSet<>(errors.keySet()));

Review Comment:
   `AddPartitionsToTxnRequest.getPartitions(request.data().v3AndBelowTopics()))` looks a bit weird from an encapsulation perspective. Why not just keeping `partitions` as before if you need it?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {

Review Comment:
   nit: `responses.forEach(result => {` -> `responses.forEach { result => `



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   When there is a global error, do we expect to set both the top level error and to return an error for each transaction? If we alway set both, what is the purpose of the top level error?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {

Review Comment:
   nit: `txns.forEach( transaction => {` -> `txns.forEach { transaction =>`.



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet

Review Comment:
   nit: I find this line quite hard to read. Should we put it on multiple lines?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()

Review Comment:
   This pattern is used in a few places. Would having `addResponseAndMaybeSend` helper method make sense?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2493,7 +2542,9 @@ class KafkaApis(val requestChannel: RequestChannel,
         addOffsetsToTxnRequest.data.producerId,
         addOffsetsToTxnRequest.data.producerEpoch,
         Set(offsetTopicPartition),
+        false,
         sendResponseCallback,
+        null,

Review Comment:
   This tend to confirm that having two callbacks is weird.



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()
+      } else {
+        val topics = new AddPartitionsToTxnTopicCollection()
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(createdTopicPartition.topic())
+          .setPartitions(Collections.singletonList(createdTopicPartition.partition())))
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(nonExistentTopic.topic())
+          .setPartitions(Collections.singletonList(nonExistentTopic.partition())))
+
+        val transactions = new AddPartitionsToTxnTransactionCollection()
+        transactions.add(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId)
+          .setProducerId(producerId)
+          .setProducerEpoch(producerEpoch)
+          .setVerifyOnly(false)
+          .setTopics(topics))
+        AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+      }
 
     val leaderId = brokers.head.config.brokerId
     val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId))
+    
+    val errors = 
+      if (version < 4) 
+        response.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID) 
+      else 
+        response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))
+    
+    assertEquals(2, errors.size)
+
+    assertTrue(errors.containsKey(createdTopicPartition))
+    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, errors.get(createdTopicPartition))
+
+    assertTrue(errors.containsKey(nonExistentTopic))
+    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
+  }
+  
+  @Test
+  def testOneSuccessOneErrorInBatchedRequest(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
+    val transactionalId1 = "foobar"
+    val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
+    val producerId2 = 1000L
+    val producerEpoch2: Short = 0
+    
+    val txn2Topics = new AddPartitionsToTxnTopicCollection()
+    txn2Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp0.topic())
+      .setPartitions(Collections.singletonList(tp0.partition())))

Review Comment:
   nit: `()` are not necessary. There are few similar cases in this file.



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()
+      } else {
+        val topics = new AddPartitionsToTxnTopicCollection()
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(createdTopicPartition.topic())
+          .setPartitions(Collections.singletonList(createdTopicPartition.partition())))
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(nonExistentTopic.topic())
+          .setPartitions(Collections.singletonList(nonExistentTopic.partition())))
+
+        val transactions = new AddPartitionsToTxnTransactionCollection()
+        transactions.add(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId)
+          .setProducerId(producerId)
+          .setProducerEpoch(producerEpoch)
+          .setVerifyOnly(false)
+          .setTopics(topics))
+        AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+      }
 
     val leaderId = brokers.head.config.brokerId
     val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId))
+    
+    val errors = 
+      if (version < 4) 
+        response.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID) 
+      else 
+        response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))

Review Comment:
   nit: `response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))` looks really weird from an encapsulation perspective. Can't we use `response.errors.get(transactionalId)`?



##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,39 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  // The AddPartitionsToTxnRequest version 4 API is added as part of KIP-890 and is still
+  // under developement. Hence, the API is not exposed by default by brokers
+  // unless explicitely enabled.
+  "latestVersionUnstable": true,
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
-      "about": "The transactional id corresponding to the transaction."},
-    { "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId",
+    { "name": "Transactions", "type": "[]AddPartitionsToTxnTransaction", "versions":  "4+",
+      "about": "List of transactions to add partitions to.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction." },
+      { "name": "ProducerId", "type": "int64", "versions": "4+", "entityType": "producerId",
+        "about": "Current producer id in use by the transactional id." },
+      { "name": "ProducerEpoch", "type": "int16", "versions": "4+",
+        "about": "Current epoch associated with the producer id." },
+      { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,
+        "about": "Boolean to signify if we want to check if the partition is in the transaction rather than add it." },
+      { "name": "Topics", "type": "[]AddPartitionsToTxnTopic", "versions": "4+",
+        "about": "The partitions to add to the transaction." }
+    ]},
+    { "name": "V3AndBelowTransactionalId", "type": "string", "versions": "0-3", "entityType": "transactionalId",

Review Comment:
   Is it common to prefix old fields by their version? It is the first time I see it.



##########
clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json:
##########
@@ -22,22 +22,37 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds support to batch multiple transactions and a top level error code.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
       "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
-    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
-      "about": "The results for each topic.", "fields": [
+    { "name": "ErrorCode", "type": "int16", "versions": "4+",

Review Comment:
   Should we make it ignorable in case we would set it by mistake?



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()
+      } else {
+        val topics = new AddPartitionsToTxnTopicCollection()
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(createdTopicPartition.topic())
+          .setPartitions(Collections.singletonList(createdTopicPartition.partition())))
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(nonExistentTopic.topic())
+          .setPartitions(Collections.singletonList(nonExistentTopic.partition())))
+
+        val transactions = new AddPartitionsToTxnTransactionCollection()
+        transactions.add(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId)
+          .setProducerId(producerId)
+          .setProducerEpoch(producerEpoch)
+          .setVerifyOnly(false)
+          .setTopics(topics))
+        AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+      }
 
     val leaderId = brokers.head.config.brokerId
     val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId))
+    
+    val errors = 
+      if (version < 4) 
+        response.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID) 
+      else 
+        response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))
+    
+    assertEquals(2, errors.size)
+
+    assertTrue(errors.containsKey(createdTopicPartition))
+    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, errors.get(createdTopicPartition))
+
+    assertTrue(errors.containsKey(nonExistentTopic))
+    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
+  }
+  
+  @Test
+  def testOneSuccessOneErrorInBatchedRequest(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
+    val transactionalId1 = "foobar"
+    val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
+    val producerId2 = 1000L
+    val producerEpoch2: Short = 0
+    
+    val txn2Topics = new AddPartitionsToTxnTopicCollection()
+    txn2Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp0.topic())
+      .setPartitions(Collections.singletonList(tp0.partition())))
+
+    val (coordinatorId, txn1) = setUpTransactions(transactionalId1, false, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn1)
+    transactions.add(new AddPartitionsToTxnTransaction()
+      .setTransactionalId(transactionalId2)
+      .setProducerId(producerId2)
+      .setProducerEpoch(producerEpoch2)
+      .setVerifyOnly(false)
+      .setTopics(txn2Topics))
+
+    val request = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(coordinatorId))
+
+    val errors = response.errors()
+
+    assertTrue(errors.containsKey(transactionalId1))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.NONE, errors.get(transactionalId1).get(tp0))
+
+    assertTrue(errors.containsKey(transactionalId2))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.INVALID_PRODUCER_ID_MAPPING, errors.get(transactionalId2).get(tp0))

Review Comment:
   nit: How about using `assertEquals(Map(... define the expect map ..., errors)`? This is simpler and has the benefit of ensuring that we only have what we expect in `errors`.
   
   Note that assertion as L148 is not correct. `transactionalId2` should be used.



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()
+      } else {
+        val topics = new AddPartitionsToTxnTopicCollection()
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(createdTopicPartition.topic())
+          .setPartitions(Collections.singletonList(createdTopicPartition.partition())))
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(nonExistentTopic.topic())
+          .setPartitions(Collections.singletonList(nonExistentTopic.partition())))
+
+        val transactions = new AddPartitionsToTxnTransactionCollection()
+        transactions.add(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId)
+          .setProducerId(producerId)
+          .setProducerEpoch(producerEpoch)
+          .setVerifyOnly(false)
+          .setTopics(topics))
+        AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+      }
 
     val leaderId = brokers.head.config.brokerId
     val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId))
+    
+    val errors = 
+      if (version < 4) 
+        response.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID) 
+      else 
+        response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))
+    
+    assertEquals(2, errors.size)
+
+    assertTrue(errors.containsKey(createdTopicPartition))
+    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, errors.get(createdTopicPartition))
+
+    assertTrue(errors.containsKey(nonExistentTopic))
+    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
+  }
+  
+  @Test
+  def testOneSuccessOneErrorInBatchedRequest(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
+    val transactionalId1 = "foobar"
+    val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
+    val producerId2 = 1000L
+    val producerEpoch2: Short = 0
+    
+    val txn2Topics = new AddPartitionsToTxnTopicCollection()
+    txn2Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp0.topic())
+      .setPartitions(Collections.singletonList(tp0.partition())))
+
+    val (coordinatorId, txn1) = setUpTransactions(transactionalId1, false, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn1)
+    transactions.add(new AddPartitionsToTxnTransaction()
+      .setTransactionalId(transactionalId2)
+      .setProducerId(producerId2)
+      .setProducerEpoch(producerEpoch2)
+      .setVerifyOnly(false)
+      .setTopics(txn2Topics))
+
+    val request = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(coordinatorId))
+
+    val errors = response.errors()
+
+    assertTrue(errors.containsKey(transactionalId1))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.NONE, errors.get(transactionalId1).get(tp0))
+
+    assertTrue(errors.containsKey(transactionalId2))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.INVALID_PRODUCER_ID_MAPPING, errors.get(transactionalId2).get(tp0))
+  }
 
-    assertEquals(2, response.errors.size)
+  @Test
+  def testVerifyOnly(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
 
-    assertTrue(response.errors.containsKey(createdTopicPartition))
-    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, response.errors.get(createdTopicPartition))
+    val transactionalId = "foobar"
+    val (coordinatorId, txn) = setUpTransactions(transactionalId, true, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn)
+    
+    val verifyRequest = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val verifyResponse = connectAndReceive[AddPartitionsToTxnResponse](verifyRequest, brokerSocketServer(coordinatorId))
+
+    val verifyErrors = verifyResponse.errors()
+
+    assertTrue(verifyErrors.containsKey(transactionalId))
+    assertTrue(verifyErrors.get(transactionalId).containsKey(tp0))
+    assertEquals(Errors.INVALID_TXN_STATE, verifyErrors.get(transactionalId).get(tp0))
+  }
+  
+  private def setUpTransactions(transactionalId: String, verifyOnly: Boolean, partitions: Set[TopicPartition]): (Int, AddPartitionsToTxnTransaction) = {
+    val findCoordinatorRequest = new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData().setKey(transactionalId).setKeyType(CoordinatorType.TRANSACTION.id)).build()
+    // First find coordinator request creates the state topic, then wait for transactional topics to be created.
+    connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId))
+    TestUtils.waitForAllPartitionsMetadata(brokers, "__transaction_state", 50)
+    val findCoordinatorResponse = connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId))
+    val coordinatorId = findCoordinatorResponse.data().coordinators().get(0).nodeId()
+
+    val initPidRequest = new InitProducerIdRequest.Builder(new InitProducerIdRequestData().setTransactionalId(transactionalId).setTransactionTimeoutMs(10000)).build()
+    val initPidResponse = connectAndReceive[InitProducerIdResponse](initPidRequest, brokerSocketServer(coordinatorId))
+
+    val producerId1 = initPidResponse.data().producerId()
+    val producerEpoch1 = initPidResponse.data().producerEpoch()
+
+    val txn1Topics = new AddPartitionsToTxnTopicCollection()
+    partitions.foreach(tp => 
+    txn1Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp.topic())
+      .setPartitions(Collections.singletonList(tp.partition())))
+    )
+
+    (coordinatorId, new AddPartitionsToTxnTransaction()
+      .setTransactionalId(transactionalId)
+      .setProducerId(producerId1)
+      .setProducerEpoch(producerEpoch1)
+      .setVerifyOnly(verifyOnly)
+      .setTopics(txn1Topics))
+  }
+}
 
-    assertTrue(response.errors.containsKey(nonExistentTopic))
-    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response.errors.get(nonExistentTopic))
+object AddPartitionsToTxnRequestServerTest {
+   def parameters: JStream[Arguments] = {
+    val arguments = mutable.ListBuffer[Arguments]()
+    ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions().forEach( version =>
+      Array("kraft", "zk").foreach( quorum =>

Review Comment:
   nit: ditto.



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()
+      } else {
+        val topics = new AddPartitionsToTxnTopicCollection()
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(createdTopicPartition.topic())
+          .setPartitions(Collections.singletonList(createdTopicPartition.partition())))
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(nonExistentTopic.topic())
+          .setPartitions(Collections.singletonList(nonExistentTopic.partition())))
+
+        val transactions = new AddPartitionsToTxnTransactionCollection()
+        transactions.add(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId)
+          .setProducerId(producerId)
+          .setProducerEpoch(producerEpoch)
+          .setVerifyOnly(false)
+          .setTopics(topics))
+        AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+      }
 
     val leaderId = brokers.head.config.brokerId
     val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId))
+    
+    val errors = 
+      if (version < 4) 
+        response.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID) 
+      else 
+        response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))
+    
+    assertEquals(2, errors.size)
+
+    assertTrue(errors.containsKey(createdTopicPartition))
+    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, errors.get(createdTopicPartition))
+
+    assertTrue(errors.containsKey(nonExistentTopic))
+    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
+  }
+  
+  @Test
+  def testOneSuccessOneErrorInBatchedRequest(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
+    val transactionalId1 = "foobar"
+    val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
+    val producerId2 = 1000L
+    val producerEpoch2: Short = 0
+    
+    val txn2Topics = new AddPartitionsToTxnTopicCollection()
+    txn2Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp0.topic())
+      .setPartitions(Collections.singletonList(tp0.partition())))
+
+    val (coordinatorId, txn1) = setUpTransactions(transactionalId1, false, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn1)
+    transactions.add(new AddPartitionsToTxnTransaction()
+      .setTransactionalId(transactionalId2)
+      .setProducerId(producerId2)
+      .setProducerEpoch(producerEpoch2)
+      .setVerifyOnly(false)
+      .setTopics(txn2Topics))
+
+    val request = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(coordinatorId))
+
+    val errors = response.errors()
+
+    assertTrue(errors.containsKey(transactionalId1))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.NONE, errors.get(transactionalId1).get(tp0))
+
+    assertTrue(errors.containsKey(transactionalId2))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.INVALID_PRODUCER_ID_MAPPING, errors.get(transactionalId2).get(tp0))
+  }
 
-    assertEquals(2, response.errors.size)
+  @Test
+  def testVerifyOnly(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
 
-    assertTrue(response.errors.containsKey(createdTopicPartition))
-    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, response.errors.get(createdTopicPartition))
+    val transactionalId = "foobar"
+    val (coordinatorId, txn) = setUpTransactions(transactionalId, true, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn)
+    
+    val verifyRequest = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val verifyResponse = connectAndReceive[AddPartitionsToTxnResponse](verifyRequest, brokerSocketServer(coordinatorId))
+
+    val verifyErrors = verifyResponse.errors()
+
+    assertTrue(verifyErrors.containsKey(transactionalId))
+    assertTrue(verifyErrors.get(transactionalId).containsKey(tp0))
+    assertEquals(Errors.INVALID_TXN_STATE, verifyErrors.get(transactionalId).get(tp0))

Review Comment:
   ditto.



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   I am not sure to understand why we need two callbacks. I find it weird in the first place. My understanding is that in both cases, we end up with an error per partition in the response so it seems to me that we could unify them, no?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }

Review Comment:
   nit: A more scala-ish version of this:
   
   ```
   val canSend = responses.synchronized {
     responses.size == txns.size
   }
   ```



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()

Review Comment:
   nit: `build()` looks weird here. Should we put the closing parenthesis of `forClient` on a new line: `).build()`?



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()
+      } else {
+        val topics = new AddPartitionsToTxnTopicCollection()
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(createdTopicPartition.topic())
+          .setPartitions(Collections.singletonList(createdTopicPartition.partition())))
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(nonExistentTopic.topic())
+          .setPartitions(Collections.singletonList(nonExistentTopic.partition())))
+
+        val transactions = new AddPartitionsToTxnTransactionCollection()
+        transactions.add(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId)
+          .setProducerId(producerId)
+          .setProducerEpoch(producerEpoch)
+          .setVerifyOnly(false)
+          .setTopics(topics))
+        AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+      }
 
     val leaderId = brokers.head.config.brokerId
     val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId))
+    
+    val errors = 
+      if (version < 4) 
+        response.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID) 
+      else 
+        response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))
+    
+    assertEquals(2, errors.size)
+
+    assertTrue(errors.containsKey(createdTopicPartition))
+    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, errors.get(createdTopicPartition))
+
+    assertTrue(errors.containsKey(nonExistentTopic))
+    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
+  }
+  
+  @Test
+  def testOneSuccessOneErrorInBatchedRequest(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
+    val transactionalId1 = "foobar"
+    val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
+    val producerId2 = 1000L
+    val producerEpoch2: Short = 0
+    
+    val txn2Topics = new AddPartitionsToTxnTopicCollection()
+    txn2Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp0.topic())
+      .setPartitions(Collections.singletonList(tp0.partition())))
+
+    val (coordinatorId, txn1) = setUpTransactions(transactionalId1, false, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn1)
+    transactions.add(new AddPartitionsToTxnTransaction()
+      .setTransactionalId(transactionalId2)
+      .setProducerId(producerId2)
+      .setProducerEpoch(producerEpoch2)
+      .setVerifyOnly(false)
+      .setTopics(txn2Topics))
+
+    val request = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(coordinatorId))
+
+    val errors = response.errors()
+
+    assertTrue(errors.containsKey(transactionalId1))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.NONE, errors.get(transactionalId1).get(tp0))
+
+    assertTrue(errors.containsKey(transactionalId2))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.INVALID_PRODUCER_ID_MAPPING, errors.get(transactionalId2).get(tp0))
+  }
 
-    assertEquals(2, response.errors.size)
+  @Test
+  def testVerifyOnly(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
 
-    assertTrue(response.errors.containsKey(createdTopicPartition))
-    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, response.errors.get(createdTopicPartition))
+    val transactionalId = "foobar"
+    val (coordinatorId, txn) = setUpTransactions(transactionalId, true, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn)
+    
+    val verifyRequest = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val verifyResponse = connectAndReceive[AddPartitionsToTxnResponse](verifyRequest, brokerSocketServer(coordinatorId))
+
+    val verifyErrors = verifyResponse.errors()
+
+    assertTrue(verifyErrors.containsKey(transactionalId))
+    assertTrue(verifyErrors.get(transactionalId).containsKey(tp0))
+    assertEquals(Errors.INVALID_TXN_STATE, verifyErrors.get(transactionalId).get(tp0))
+  }
+  
+  private def setUpTransactions(transactionalId: String, verifyOnly: Boolean, partitions: Set[TopicPartition]): (Int, AddPartitionsToTxnTransaction) = {
+    val findCoordinatorRequest = new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData().setKey(transactionalId).setKeyType(CoordinatorType.TRANSACTION.id)).build()
+    // First find coordinator request creates the state topic, then wait for transactional topics to be created.
+    connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId))
+    TestUtils.waitForAllPartitionsMetadata(brokers, "__transaction_state", 50)
+    val findCoordinatorResponse = connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId))
+    val coordinatorId = findCoordinatorResponse.data().coordinators().get(0).nodeId()
+
+    val initPidRequest = new InitProducerIdRequest.Builder(new InitProducerIdRequestData().setTransactionalId(transactionalId).setTransactionTimeoutMs(10000)).build()
+    val initPidResponse = connectAndReceive[InitProducerIdResponse](initPidRequest, brokerSocketServer(coordinatorId))
+
+    val producerId1 = initPidResponse.data().producerId()
+    val producerEpoch1 = initPidResponse.data().producerEpoch()
+
+    val txn1Topics = new AddPartitionsToTxnTopicCollection()
+    partitions.foreach(tp => 
+    txn1Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp.topic())
+      .setPartitions(Collections.singletonList(tp.partition())))
+    )
+
+    (coordinatorId, new AddPartitionsToTxnTransaction()
+      .setTransactionalId(transactionalId)
+      .setProducerId(producerId1)
+      .setProducerEpoch(producerEpoch1)
+      .setVerifyOnly(verifyOnly)
+      .setTopics(txn1Topics))
+  }
+}
 
-    assertTrue(response.errors.containsKey(nonExistentTopic))
-    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response.errors.get(nonExistentTopic))
+object AddPartitionsToTxnRequestServerTest {
+   def parameters: JStream[Arguments] = {
+    val arguments = mutable.ListBuffer[Arguments]()
+    ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions().forEach( version =>

Review Comment:
   nit: `forEach { `.



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,149 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava)
+          .build()
+      } else {
+        val topics = new AddPartitionsToTxnTopicCollection()
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(createdTopicPartition.topic())
+          .setPartitions(Collections.singletonList(createdTopicPartition.partition())))
+        topics.add(new AddPartitionsToTxnTopic()
+          .setName(nonExistentTopic.topic())
+          .setPartitions(Collections.singletonList(nonExistentTopic.partition())))
+
+        val transactions = new AddPartitionsToTxnTransactionCollection()
+        transactions.add(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId)
+          .setProducerId(producerId)
+          .setProducerEpoch(producerEpoch)
+          .setVerifyOnly(false)
+          .setTopics(topics))
+        AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+      }
 
     val leaderId = brokers.head.config.brokerId
     val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(leaderId))
+    
+    val errors = 
+      if (version < 4) 
+        response.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID) 
+      else 
+        response.errorsForTransaction(response.getTransactionTopicResults(transactionalId))
+    
+    assertEquals(2, errors.size)
+
+    assertTrue(errors.containsKey(createdTopicPartition))
+    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, errors.get(createdTopicPartition))
+
+    assertTrue(errors.containsKey(nonExistentTopic))
+    assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
+  }
+  
+  @Test
+  def testOneSuccessOneErrorInBatchedRequest(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
+    val transactionalId1 = "foobar"
+    val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
+    val producerId2 = 1000L
+    val producerEpoch2: Short = 0
+    
+    val txn2Topics = new AddPartitionsToTxnTopicCollection()
+    txn2Topics.add(new AddPartitionsToTxnTopic()
+      .setName(tp0.topic())
+      .setPartitions(Collections.singletonList(tp0.partition())))
+
+    val (coordinatorId, txn1) = setUpTransactions(transactionalId1, false, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn1)
+    transactions.add(new AddPartitionsToTxnTransaction()
+      .setTransactionalId(transactionalId2)
+      .setProducerId(producerId2)
+      .setProducerEpoch(producerEpoch2)
+      .setVerifyOnly(false)
+      .setTopics(txn2Topics))
+
+    val request = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val response = connectAndReceive[AddPartitionsToTxnResponse](request, brokerSocketServer(coordinatorId))
+
+    val errors = response.errors()
+
+    assertTrue(errors.containsKey(transactionalId1))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.NONE, errors.get(transactionalId1).get(tp0))
+
+    assertTrue(errors.containsKey(transactionalId2))
+    assertTrue(errors.get(transactionalId1).containsKey(tp0))
+    assertEquals(Errors.INVALID_PRODUCER_ID_MAPPING, errors.get(transactionalId2).get(tp0))
+  }
 
-    assertEquals(2, response.errors.size)
+  @Test
+  def testVerifyOnly(): Unit = {
+    val tp0 = new TopicPartition(topic1, 0)
 
-    assertTrue(response.errors.containsKey(createdTopicPartition))
-    assertEquals(Errors.OPERATION_NOT_ATTEMPTED, response.errors.get(createdTopicPartition))
+    val transactionalId = "foobar"
+    val (coordinatorId, txn) = setUpTransactions(transactionalId, true, Set(tp0))
+
+    val transactions = new AddPartitionsToTxnTransactionCollection()
+    transactions.add(txn)
+    
+    val verifyRequest = AddPartitionsToTxnRequest.Builder.forBroker(transactions).build()
+
+    val verifyResponse = connectAndReceive[AddPartitionsToTxnResponse](verifyRequest, brokerSocketServer(coordinatorId))
+
+    val verifyErrors = verifyResponse.errors()
+
+    assertTrue(verifyErrors.containsKey(transactionalId))
+    assertTrue(verifyErrors.get(transactionalId).containsKey(tp0))
+    assertEquals(Errors.INVALID_TXN_STATE, verifyErrors.get(transactionalId).get(tp0))
+  }
+  
+  private def setUpTransactions(transactionalId: String, verifyOnly: Boolean, partitions: Set[TopicPartition]): (Int, AddPartitionsToTxnTransaction) = {
+    val findCoordinatorRequest = new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData().setKey(transactionalId).setKeyType(CoordinatorType.TRANSACTION.id)).build()
+    // First find coordinator request creates the state topic, then wait for transactional topics to be created.
+    connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId))
+    TestUtils.waitForAllPartitionsMetadata(brokers, "__transaction_state", 50)
+    val findCoordinatorResponse = connectAndReceive[FindCoordinatorResponse](findCoordinatorRequest, brokerSocketServer(brokers.head.config.brokerId))
+    val coordinatorId = findCoordinatorResponse.data().coordinators().get(0).nodeId()
+
+    val initPidRequest = new InitProducerIdRequest.Builder(new InitProducerIdRequestData().setTransactionalId(transactionalId).setTransactionTimeoutMs(10000)).build()
+    val initPidResponse = connectAndReceive[InitProducerIdResponse](initPidRequest, brokerSocketServer(coordinatorId))
+
+    val producerId1 = initPidResponse.data().producerId()
+    val producerEpoch1 = initPidResponse.data().producerEpoch()
+
+    val txn1Topics = new AddPartitionsToTxnTopicCollection()
+    partitions.foreach(tp => 

Review Comment:
   nit: `partitions.foreach {` 



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -48,29 +51,50 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
 
     private final AddPartitionsToTxnResponseData data;
 
-    private Map<TopicPartition, Errors> cachedErrorsMap = null;
+    public static final String V3_AND_BELOW_TXN_ID = "";
 
     public AddPartitionsToTxnResponse(AddPartitionsToTxnResponseData data) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN);
         this.data = data;
     }
 
-    public AddPartitionsToTxnResponse(int throttleTimeMs, Map<TopicPartition, Errors> errors) {
-        super(ApiKeys.ADD_PARTITIONS_TO_TXN);
+    @Override
+    public int throttleTimeMs() {
+        return data.throttleTimeMs();
+    }
 
+    @Override
+    public void maybeSetThrottleTimeMs(int throttleTimeMs) {
+        data.setThrottleTimeMs(throttleTimeMs);
+    }
+
+    public Map<String, Map<TopicPartition, Errors>> errors() {
+        Map<String, Map<TopicPartition, Errors>> errorsMap = new HashMap<>();
+
+        errorsMap.put(V3_AND_BELOW_TXN_ID, errorsForTransaction(this.data.resultsByTopicV3AndBelow()));
+
+        for (AddPartitionsToTxnResult result : this.data.resultsByTransaction()) {
+            String transactionalId = result.transactionalId();
+            errorsMap.put(transactionalId, errorsForTransaction(data().resultsByTransaction().find(transactionalId).topicResults()));

Review Comment:
   Can't you reuse `result` instead of calling `data().resultsByTransaction().find(transactionalId)`?



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1962,7 +1962,9 @@ class KafkaApisTest {
         ArgumentMatchers.eq(producerId),
         ArgumentMatchers.eq(epoch),
         ArgumentMatchers.eq(Set(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partition))),
+        ArgumentMatchers.eq(false),

Review Comment:
   It would be great if we could also add new unit tests to cover the batch mode in `KafkaApisTest`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123767340


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1328,7 +1328,7 @@ Priority priority() {
         @Override
         public void handleResponse(AbstractResponse response) {
             AddPartitionsToTxnResponse addPartitionsToTxnResponse = (AddPartitionsToTxnResponse) response;
-            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors();
+            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID);

Review Comment:
   Yeah. It really is unfortunate. 😞 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107764891


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +193,41 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        if (version < 4) {
+            final HashMap<TopicPartition, Errors> errors = new HashMap<>();
+            for (TopicPartition partition : partitions()) {
+                errors.put(partition, error);
+            }
+            return new AddPartitionsToTxnResponse(throttleTimeMs, errors);
+        } else {
+            AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();

Review Comment:
   Many new APIs do include a top-level error code, so there is definitely precedent. I kind of like giving the broker an easy way to indicate a failure without requiring it to do a bunch of wasteful work. It's not a common case though, so I'm not sure how much it matters.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107590295


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2383,68 +2384,101 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val lock = new Object
+    val addPartitionsToTxnRequest = if (request.context.apiVersion() < 4) request.body[AddPartitionsToTxnRequest].normalizeRequest() else request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResults(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      lock synchronized {
+        if (responses.size() == txns.size()) {
+          requestHelper.sendResponseMaybeThrottle(request, createResponse)

Review Comment:
   no



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107589437


##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -352,7 +353,12 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
               // this is an optimization: if the partitions are already in the metadata reply OK immediately
               Left(Errors.NONE)
             } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
+              // If verifyOnly, we should have returned in the step above. If we didn't the partitions are not present in the transaction.

Review Comment:
   yes



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107838433


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4
+        public Builder(String transactionalId,
+                       long producerId,
+                       short producerEpoch,
+                       List<TopicPartition> partitions) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
-            this.data = data;
+            this.isClientRequest = true;
+
+            AddPartitionsToTxnTopicCollection topics = compileTopics(partitions);
+
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactionalId(transactionalId)
+                    .setProducerId(producerId)
+                    .setProducerEpoch(producerEpoch)
+                    .setTopics(topics);

Review Comment:
   i don't know either 😅 i only bring this up because i had feedback on this before. it seems like other places use both double indentation and single so maybe we should have a formalized decision



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1124792127


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -34,22 +43,38 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
 
     private final AddPartitionsToTxnRequestData data;
 
-    private List<TopicPartition> cachedPartitions = null;
-
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        
+        public static Builder forClient(String transactionalId,
+                                        long producerId,
+                                        short producerEpoch,
+                                        List<TopicPartition> partitions) {
+
+            AddPartitionsToTxnTopicCollection topics = buildTxnTopicCollection(partitions);
+            
+            return new Builder(ApiKeys.ADD_PARTITIONS_TO_TXN.oldestVersion(),
+                (short) 3, 

Review Comment:
   nit: Should we put `(short) 3` on the previous line to be consistent with how you did it at L66?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -34,22 +43,38 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
 
     private final AddPartitionsToTxnRequestData data;
 
-    private List<TopicPartition> cachedPartitions = null;
-
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        
+        public static Builder forClient(String transactionalId,
+                                        long producerId,
+                                        short producerEpoch,
+                                        List<TopicPartition> partitions) {
+
+            AddPartitionsToTxnTopicCollection topics = buildTxnTopicCollection(partitions);
+            
+            return new Builder(ApiKeys.ADD_PARTITIONS_TO_TXN.oldestVersion(),
+                (short) 3, 
+                new AddPartitionsToTxnRequestData()
+                    .setV3AndBelowTransactionalId(transactionalId)
+                    .setV3AndBelowProducerId(producerId)
+                    .setV3AndBelowProducerEpoch(producerEpoch)
+                    .setV3AndBelowTopics(topics));
+        }
+        
+        public static Builder forBroker(AddPartitionsToTxnTransactionCollection transactions) {
+            return new Builder((short) 4, ApiKeys.ADD_PARTITIONS_TO_TXN.latestVersion(),
+                new AddPartitionsToTxnRequestData()
+                    .setTransactions(transactions));
+        }
+        
+        public Builder(short minVersion, short maxVersion, AddPartitionsToTxnRequestData data) {

Review Comment:
   nit: Do we still use this constructor anywhere? It may be good to make it private or package private to ensure that `forClient` or `forBroker` is used.



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -48,29 +51,51 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
 
     private final AddPartitionsToTxnResponseData data;
 
-    private Map<TopicPartition, Errors> cachedErrorsMap = null;
+    public static final String V3_AND_BELOW_TXN_ID = "";
 
     public AddPartitionsToTxnResponse(AddPartitionsToTxnResponseData data) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN);
         this.data = data;
     }
 
-    public AddPartitionsToTxnResponse(int throttleTimeMs, Map<TopicPartition, Errors> errors) {
-        super(ApiKeys.ADD_PARTITIONS_TO_TXN);
+    @Override
+    public int throttleTimeMs() {
+        return data.throttleTimeMs();
+    }
+
+    @Override
+    public void maybeSetThrottleTimeMs(int throttleTimeMs) {
+        data.setThrottleTimeMs(throttleTimeMs);
+    }
+
+    public Map<String, Map<TopicPartition, Errors>> errors() {
+        Map<String, Map<TopicPartition, Errors>> errorsMap = new HashMap<>();
+
+        if (this.data.resultsByTopicV3AndBelow().size() != 0) {

Review Comment:
   nit: I think that we usually prefer using `isEmpty()`.



##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponseTest.java:
##########
@@ -58,42 +65,75 @@ public void setUp() {
         errorsMap.put(tp2, errorTwo);
     }
 
-    @Test
-    public void testConstructorWithErrorResponse() {
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(throttleTimeMs, errorsMap);
-
-        assertEquals(expectedErrorCounts, response.errorCounts());
-        assertEquals(throttleTimeMs, response.throttleTimeMs());
-    }
-
-    @Test
-    public void testParse() {
-
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.ADD_PARTITIONS_TO_TXN)
+    public void testParse(short version) {
         AddPartitionsToTxnTopicResultCollection topicCollection = new AddPartitionsToTxnTopicResultCollection();
 
         AddPartitionsToTxnTopicResult topicResult = new AddPartitionsToTxnTopicResult();
         topicResult.setName(topicOne);
 
-        topicResult.results().add(new AddPartitionsToTxnPartitionResult()
-                                      .setErrorCode(errorOne.code())
+        topicResult.resultsByPartition().add(new AddPartitionsToTxnPartitionResult()
+                                      .setPartitionErrorCode(errorOne.code())
                                       .setPartitionIndex(partitionOne));
 
-        topicResult.results().add(new AddPartitionsToTxnPartitionResult()
-                                      .setErrorCode(errorTwo.code())
+        topicResult.resultsByPartition().add(new AddPartitionsToTxnPartitionResult()
+                                      .setPartitionErrorCode(errorTwo.code())
                                       .setPartitionIndex(partitionTwo));
 
         topicCollection.add(topicResult);
+            
+        if (version < 4) {
+            AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
+                    .setResultsByTopicV3AndBelow(topicCollection)
+                    .setThrottleTimeMs(throttleTimeMs);
+            AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
 
-        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
-                                                  .setResults(topicCollection)
-                                                  .setThrottleTimeMs(throttleTimeMs);
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
-
-        for (short version : ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions()) {
             AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serialize(version), version);
             assertEquals(expectedErrorCounts, parsedResponse.errorCounts());
             assertEquals(throttleTimeMs, parsedResponse.throttleTimeMs());
             assertEquals(version >= 1, parsedResponse.shouldClientThrottle(version));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            results.add(new AddPartitionsToTxnResult().setTransactionalId("txn1").setTopicResults(topicCollection));
+            
+            // Create another transaction with new name and errorOne for a single partition.
+            Map<TopicPartition, Errors> txnTwoExpectedErrors = Collections.singletonMap(tp2, errorOne);
+            results.add(AddPartitionsToTxnResponse.resultForTransaction("txn2", txnTwoExpectedErrors));
+
+            AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
+                    .setResultsByTransaction(results)
+                    .setThrottleTimeMs(throttleTimeMs);
+            AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
+
+            Map<Errors, Integer> newExpectedErrorCounts = new HashMap<>();
+            newExpectedErrorCounts.put(Errors.NONE, 1); // top level error
+            newExpectedErrorCounts.put(errorOne, 2);
+            newExpectedErrorCounts.put(errorTwo, 1);
+            
+            AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serialize(version), version);
+            assertEquals(txnTwoExpectedErrors, errorsForTransaction(response.getTransactionTopicResults("txn2")));
+            assertEquals(newExpectedErrorCounts, parsedResponse.errorCounts());
+            assertEquals(throttleTimeMs, parsedResponse.throttleTimeMs());
+            assertTrue(parsedResponse.shouldClientThrottle(version));
         }
     }
+    
+    @Test
+    public void testBatchedErrors() {
+        Map<TopicPartition, Errors> txn1Errors = Collections.singletonMap(tp1, errorOne);
+        Map<TopicPartition, Errors> txn2Errors = Collections.singletonMap(tp1, errorOne);
+        
+        AddPartitionsToTxnResult transaction1 = AddPartitionsToTxnResponse.resultForTransaction("txn1", txn1Errors);
+        AddPartitionsToTxnResult transaction2 = AddPartitionsToTxnResponse.resultForTransaction("txn2", txn2Errors);
+        
+        AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+        results.add(transaction1);
+        results.add(transaction2);
+        
+        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setResultsByTransaction(results));
+        
+        assertEquals(txn1Errors, errorsForTransaction(response.getTransactionTopicResults("txn1")));
+        assertEquals(txn2Errors, errorsForTransaction(response.getTransactionTopicResults("txn2")));
+    }

Review Comment:
   nit: Should we add a test for `errors()`?



##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -317,6 +322,34 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
       }
     }
   }
+  
+  def handleVerifyPartitionsInTransaction(transactionalId: String,
+                                          producerId: Long,
+                                          producerEpoch: Short,
+                                          partitions: collection.Set[TopicPartition],
+                                          responseCallback: VerifyPartitionsCallback): Unit = {
+    if (transactionalId == null || transactionalId.isEmpty) {
+      debug(s"Returning ${Errors.INVALID_REQUEST} error code to client for $transactionalId's AddPartitions request")
+      responseCallback(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitions.map(_ -> Errors.INVALID_REQUEST).toMap.asJava))
+    } else {
+      val result: ApiResult[(Int, TransactionMetadata)] = getTransactionMetadata(transactionalId, producerId, producerEpoch, partitions)
+      
+      result match {
+        case Left(err) =>
+          debug(s"Returning $err error code to client for $transactionalId's AddPartitions request")

Review Comment:
   nit: Should update this line to mention that we are validating only here?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -80,45 +105,44 @@ topicName, new AddPartitionsToTxnPartitionResultCollection()
         AddPartitionsToTxnTopicResultCollection topicCollection = new AddPartitionsToTxnTopicResultCollection();
         for (Map.Entry<String, AddPartitionsToTxnPartitionResultCollection> entry : resultMap.entrySet()) {
             topicCollection.add(new AddPartitionsToTxnTopicResult()
-                                    .setName(entry.getKey())
-                                    .setResults(entry.getValue()));
+                .setName(entry.getKey())
+                .setResultsByPartition(entry.getValue()));
         }
-
-        this.data = new AddPartitionsToTxnResponseData()
-                        .setThrottleTimeMs(throttleTimeMs)
-                        .setResults(topicCollection);
+        return topicCollection;
     }
 
-    @Override
-    public int throttleTimeMs() {
-        return data.throttleTimeMs();
+    public static AddPartitionsToTxnResult resultForTransaction(String transactionalId, Map<TopicPartition, Errors> errors) {
+        return new AddPartitionsToTxnResult().setTransactionalId(transactionalId).setTopicResults(topicCollectionForErrors(errors));
     }
 
-    @Override
-    public void maybeSetThrottleTimeMs(int throttleTimeMs) {
-        data.setThrottleTimeMs(throttleTimeMs);
+    public AddPartitionsToTxnTopicResultCollection getTransactionTopicResults(String transactionalId) {
+        return data.resultsByTransaction().find(transactionalId).topicResults();
     }
 
-    public Map<TopicPartition, Errors> errors() {
-        if (cachedErrorsMap != null) {
-            return cachedErrorsMap;
-        }
-
-        cachedErrorsMap = new HashMap<>();
-
-        for (AddPartitionsToTxnTopicResult topicResult : this.data.results()) {
-            for (AddPartitionsToTxnPartitionResult partitionResult : topicResult.results()) {
-                cachedErrorsMap.put(new TopicPartition(
-                        topicResult.name(), partitionResult.partitionIndex()),
-                    Errors.forCode(partitionResult.errorCode()));
+    public static Map<TopicPartition, Errors> errorsForTransaction(AddPartitionsToTxnTopicResultCollection topicCollection) {
+        Map<TopicPartition, Errors> topicResults = new HashMap<>();
+        for (AddPartitionsToTxnTopicResult topicResult : topicCollection) {
+            for (AddPartitionsToTxnPartitionResult partitionResult : topicResult.resultsByPartition()) {
+                topicResults.put(
+                    new TopicPartition(topicResult.name(), partitionResult.partitionIndex()), Errors.forCode(partitionResult.partitionErrorCode()));
             }
         }
-        return cachedErrorsMap;
+        return topicResults;
     }
 
     @Override
     public Map<Errors, Integer> errorCounts() {
-        return errorCounts(errors().values());
+        List<Errors> allErrors = new ArrayList<>();
+
+        // If we are not using this field, we have request 4 or later
+        if (this.data.resultsByTopicV3AndBelow().size() == 0) {
+            allErrors.add(Errors.forCode(data.errorCode()));

Review Comment:
   nit: Should we use `updateErrorCounts` from `AbstractResponse` instead of creating `allErrors`?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,66 +2386,111 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach { result => 
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        }
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def addResultAndMaybeSendResponse(result: AddPartitionsToTxnResult): Unit = {
+      val canSend = responses.synchronized {
+        responses.add(result)
+        responses.size() == txns.size()
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach { transaction => 
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        addResultAndMaybeSendResponse(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        // Only request versions less than 4 need write authorization since they come from clients.
+        val authorizedTopics = 
+          if (version < 4) 
+            authHelper.filterByAuthorized(request.context, WRITE, TOPIC, partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) 
+          else 
+            partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          addResultAndMaybeSendResponse(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            addResultAndMaybeSendResponse(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
-        }
 
-        txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
-          addPartitionsToTxnRequest.data.producerId,
-          addPartitionsToTxnRequest.data.producerEpoch,
-          authorizedPartitions,
-          sendResponseCallback,
-          requestLocal)
+          if (!transaction.verifyOnly) {
+            txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+              transaction.producerId,
+              transaction.producerEpoch,
+              authorizedPartitions,
+              sendResponseCallback,
+              requestLocal)
+          } else {
+            txnCoordinator.handleVerifyPartitionsInTransaction(transactionalId,
+              transaction.producerId,
+              transaction.producerEpoch,
+              authorizedPartitions,
+              addResultAndMaybeSendResponse)
+          }
+        }

Review Comment:
   👍🏻 



##########
core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala:
##########
@@ -314,6 +316,32 @@ class TransactionCoordinatorTest {
     verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
   }
 
+  @Test
+  def shouldRespondWithErrorsNoneOnAddPartitionWhenOngoingVerifyOnlyAndPartitionsTheSame(): Unit = {
+    when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
+      .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch,
+        new TransactionMetadata(transactionalId, 0, 0, 0, RecordBatch.NO_PRODUCER_EPOCH, 0, Ongoing, partitions, 0, 0)))))
+
+    coordinator.handleVerifyPartitionsInTransaction(transactionalId, 0L, 0, partitions, verifyPartitionsInTxnCallback)
+    assertEquals(Errors.NONE, error)
+    verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
+  }
+  
+  @Test
+  def shouldRespondWithInvalidTxnStateWhenVerifyOnlyAndPartitionNotPresent(): Unit = {
+    when(transactionManager.getTransactionState(ArgumentMatchers.eq(transactionalId)))
+      .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch,
+        new TransactionMetadata(transactionalId, 0, 0, 0, RecordBatch.NO_PRODUCER_EPOCH, 0, Empty, partitions, 0, 0)))))
+

Review Comment:
   nit: Extra empty line.



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -103,26 +113,82 @@ public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short
         this.data = data;
     }
 
-    public List<TopicPartition> partitions() {
-        if (cachedPartitions != null) {
-            return cachedPartitions;
-        }
-        cachedPartitions = Builder.getPartitions(data);
-        return cachedPartitions;
-    }
-
     @Override
     public AddPartitionsToTxnRequestData data() {
         return data;
     }
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version() < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            response.setResultsByTransaction(results);

Review Comment:
   nit: I think that you can remove those two lines. The collection should be initialized by default.



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -2030,13 +2032,93 @@ class KafkaApisTest {
       val response = capturedResponse.getValue
 
       if (version < 2) {
-        assertEquals(Collections.singletonMap(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors())
+        assertEquals(Collections.singletonMap(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID))
       } else {
-        assertEquals(Collections.singletonMap(topicPartition, Errors.PRODUCER_FENCED), response.errors())
+        assertEquals(Collections.singletonMap(topicPartition, Errors.PRODUCER_FENCED), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID))
       }
     }
   }
 
+  @Test
+  def testBatchedRequest(): Unit = {
+    val topic = "topic"
+    addTopicToMetadataCache(topic, numPartitions = 2)
+
+    val capturedResponse: ArgumentCaptor[AddPartitionsToTxnResponse] = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnResponse])
+    val responseCallback: ArgumentCaptor[Errors => Unit] = ArgumentCaptor.forClass(classOf[Errors => Unit])
+    val verifyPartitionsCallback: ArgumentCaptor[AddPartitionsToTxnResult => Unit] = ArgumentCaptor.forClass(classOf[AddPartitionsToTxnResult => Unit])
+
+    val transactionalId1 = "txnId1"
+    val transactionalId2 = "txnId2"
+    val producerId = 15L
+    val epoch = 0.toShort
+    
+    val tp0 = new TopicPartition(topic, 0)
+    val tp1 = new TopicPartition(topic, 1)
+
+    val addPartitionsToTxnRequest = AddPartitionsToTxnRequest.Builder.forBroker(
+      new AddPartitionsToTxnTransactionCollection(
+        List(new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId1)
+          .setProducerId(producerId)
+          .setProducerEpoch(epoch)
+          .setVerifyOnly(false)
+          .setTopics(new AddPartitionsToTxnTopicCollection(
+            Collections.singletonList(new AddPartitionsToTxnTopic()
+              .setName(tp0.topic)
+              .setPartitions(Collections.singletonList(tp0.partition))
+            ).iterator())
+          ), new AddPartitionsToTxnTransaction()
+          .setTransactionalId(transactionalId2)
+          .setProducerId(producerId)
+          .setProducerEpoch(epoch)
+          .setVerifyOnly(true)
+          .setTopics(new AddPartitionsToTxnTopicCollection(
+            Collections.singletonList(new AddPartitionsToTxnTopic()
+              .setName(tp1.topic)
+              .setPartitions(Collections.singletonList(tp1.partition))
+            ).iterator())
+          )
+        ).asJava.iterator()
+      )
+    ).build(4.toShort)
+    val request = buildRequest(addPartitionsToTxnRequest)
+
+    val requestLocal = RequestLocal.withThreadConfinedCaching
+    when(txnCoordinator.handleAddPartitionsToTransaction(
+      ArgumentMatchers.eq(transactionalId1),
+      ArgumentMatchers.eq(producerId),
+      ArgumentMatchers.eq(epoch),
+      ArgumentMatchers.eq(Set(tp0)),
+      responseCallback.capture(),
+      ArgumentMatchers.eq(requestLocal)
+    )).thenAnswer(_ => responseCallback.getValue.apply(Errors.NONE))
+
+    when(txnCoordinator.handleVerifyPartitionsInTransaction(
+      ArgumentMatchers.eq(transactionalId2),
+      ArgumentMatchers.eq(producerId),
+      ArgumentMatchers.eq(epoch),
+      ArgumentMatchers.eq(Set(tp1)),
+      verifyPartitionsCallback.capture(),
+    )).thenAnswer(_ => verifyPartitionsCallback.getValue.apply(AddPartitionsToTxnResponse.resultForTransaction(transactionalId2, Map(tp1 -> Errors.PRODUCER_FENCED).asJava)))
+
+    createKafkaApis().handleAddPartitionsToTxnRequest(request, requestLocal)
+
+    verify(requestChannel).sendResponse(
+      ArgumentMatchers.eq(request),
+      capturedResponse.capture(),
+      ArgumentMatchers.eq(None)
+    )
+    val response = capturedResponse.getValue

Review Comment:
   nit: You can use `verifyNoThrottling`.



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -80,45 +105,44 @@ topicName, new AddPartitionsToTxnPartitionResultCollection()
         AddPartitionsToTxnTopicResultCollection topicCollection = new AddPartitionsToTxnTopicResultCollection();
         for (Map.Entry<String, AddPartitionsToTxnPartitionResultCollection> entry : resultMap.entrySet()) {
             topicCollection.add(new AddPartitionsToTxnTopicResult()
-                                    .setName(entry.getKey())
-                                    .setResults(entry.getValue()));
+                .setName(entry.getKey())
+                .setResultsByPartition(entry.getValue()));
         }
-
-        this.data = new AddPartitionsToTxnResponseData()
-                        .setThrottleTimeMs(throttleTimeMs)
-                        .setResults(topicCollection);
+        return topicCollection;
     }
 
-    @Override
-    public int throttleTimeMs() {
-        return data.throttleTimeMs();
+    public static AddPartitionsToTxnResult resultForTransaction(String transactionalId, Map<TopicPartition, Errors> errors) {
+        return new AddPartitionsToTxnResult().setTransactionalId(transactionalId).setTopicResults(topicCollectionForErrors(errors));
     }
 
-    @Override
-    public void maybeSetThrottleTimeMs(int throttleTimeMs) {
-        data.setThrottleTimeMs(throttleTimeMs);
+    public AddPartitionsToTxnTopicResultCollection getTransactionTopicResults(String transactionalId) {
+        return data.resultsByTransaction().find(transactionalId).topicResults();
     }
 
-    public Map<TopicPartition, Errors> errors() {
-        if (cachedErrorsMap != null) {
-            return cachedErrorsMap;
-        }
-
-        cachedErrorsMap = new HashMap<>();
-
-        for (AddPartitionsToTxnTopicResult topicResult : this.data.results()) {
-            for (AddPartitionsToTxnPartitionResult partitionResult : topicResult.results()) {
-                cachedErrorsMap.put(new TopicPartition(
-                        topicResult.name(), partitionResult.partitionIndex()),
-                    Errors.forCode(partitionResult.errorCode()));
+    public static Map<TopicPartition, Errors> errorsForTransaction(AddPartitionsToTxnTopicResultCollection topicCollection) {
+        Map<TopicPartition, Errors> topicResults = new HashMap<>();
+        for (AddPartitionsToTxnTopicResult topicResult : topicCollection) {
+            for (AddPartitionsToTxnPartitionResult partitionResult : topicResult.resultsByPartition()) {
+                topicResults.put(
+                    new TopicPartition(topicResult.name(), partitionResult.partitionIndex()), Errors.forCode(partitionResult.partitionErrorCode()));
             }
         }
-        return cachedErrorsMap;
+        return topicResults;
     }
 
     @Override
     public Map<Errors, Integer> errorCounts() {
-        return errorCounts(errors().values());
+        List<Errors> allErrors = new ArrayList<>();
+
+        // If we are not using this field, we have request 4 or later
+        if (this.data.resultsByTopicV3AndBelow().size() == 0) {

Review Comment:
   nit: `isEmpty()`?



##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -2030,13 +2032,93 @@ class KafkaApisTest {
       val response = capturedResponse.getValue
 
       if (version < 2) {
-        assertEquals(Collections.singletonMap(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors())
+        assertEquals(Collections.singletonMap(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID))
       } else {
-        assertEquals(Collections.singletonMap(topicPartition, Errors.PRODUCER_FENCED), response.errors())
+        assertEquals(Collections.singletonMap(topicPartition, Errors.PRODUCER_FENCED), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID))
       }
     }
   }
 
+  @Test
+  def testBatchedRequest(): Unit = {

Review Comment:
   nit: Could we include the related api in the name?



##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -317,6 +322,34 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
       }
     }
   }
+  
+  def handleVerifyPartitionsInTransaction(transactionalId: String,
+                                          producerId: Long,
+                                          producerEpoch: Short,
+                                          partitions: collection.Set[TopicPartition],
+                                          responseCallback: VerifyPartitionsCallback): Unit = {
+    if (transactionalId == null || transactionalId.isEmpty) {
+      debug(s"Returning ${Errors.INVALID_REQUEST} error code to client for $transactionalId's AddPartitions request")
+      responseCallback(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitions.map(_ -> Errors.INVALID_REQUEST).toMap.asJava))
+    } else {
+      val result: ApiResult[(Int, TransactionMetadata)] = getTransactionMetadata(transactionalId, producerId, producerEpoch, partitions)
+      
+      result match {
+        case Left(err) =>
+          debug(s"Returning $err error code to client for $transactionalId's AddPartitions request")
+          responseCallback(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitions.map(_ -> err).toMap.asJava))
+
+        case Right((_, txnMetadata)) =>
+          val txnMetadataPartitions = txnMetadata.topicPartitions
+          val addedPartitions = partitions.intersect(txnMetadataPartitions)
+          val nonAddedPartitions = partitions.diff(txnMetadataPartitions)
+          val errors = mutable.Map[TopicPartition, Errors]()
+          addedPartitions.foreach(errors.put(_, Errors.NONE))
+          nonAddedPartitions.foreach(errors.put(_, Errors.INVALID_TXN_STATE))

Review Comment:
   nit: I am not sure if it makes a real difference but did you consider doing something like this:
   
   ```
   partitions.foreach { tp =>
     if (txnMetadata.topicPartitions.contains(tp))
       ...
     else
      ...
   }
   ```
   
   If works, it would avoid allocating the intermediate collections. I leave this up to you.



##########
clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java:
##########
@@ -2598,12 +2604,37 @@ private OffsetsForLeaderEpochResponse createLeaderEpochResponse() {
     }
 
     private AddPartitionsToTxnRequest createAddPartitionsToTxnRequest(short version) {
-        return new AddPartitionsToTxnRequest.Builder("tid", 21L, (short) 42,
-            singletonList(new TopicPartition("topic", 73))).build(version);
+        if (version < 4) {
+            return AddPartitionsToTxnRequest.Builder.forClient("tid", 21L, (short) 42,
+                    singletonList(new TopicPartition("topic", 73))).build(version);
+        } else {
+            AddPartitionsToTxnTransactionCollection transactions = new AddPartitionsToTxnTransactionCollection(
+                    singletonList(new AddPartitionsToTxnTransaction()

Review Comment:
   nit: Indentation of this line looks weird. Should it be on the previous line?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,66 +2386,111 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach { result => 
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        }
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def addResultAndMaybeSendResponse(result: AddPartitionsToTxnResult): Unit = {
+      val canSend = responses.synchronized {
+        responses.add(result)
+        responses.size() == txns.size()

Review Comment:
   nit: You can remove the `()`.



##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -330,44 +363,53 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
     } else {
       // try to update the transaction metadata and append the updated metadata to txn log;
       // if there is no such metadata treat it as invalid producerId mapping error.
-      val result: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap {
-        case None => Left(Errors.INVALID_PRODUCER_ID_MAPPING)
-
-        case Some(epochAndMetadata) =>
-          val coordinatorEpoch = epochAndMetadata.coordinatorEpoch
-          val txnMetadata = epochAndMetadata.transactionMetadata
-
-          // generate the new transaction metadata with added partitions
-          txnMetadata.inLock {
-            if (txnMetadata.producerId != producerId) {
-              Left(Errors.INVALID_PRODUCER_ID_MAPPING)
-            } else if (txnMetadata.producerEpoch != producerEpoch) {
-              Left(Errors.PRODUCER_FENCED)
-            } else if (txnMetadata.pendingTransitionInProgress) {
-              // return a retriable exception to let the client backoff and retry
-              Left(Errors.CONCURRENT_TRANSACTIONS)
-            } else if (txnMetadata.state == PrepareCommit || txnMetadata.state == PrepareAbort) {
-              Left(Errors.CONCURRENT_TRANSACTIONS)
-            } else if (txnMetadata.state == Ongoing && partitions.subsetOf(txnMetadata.topicPartitions)) {
-              // this is an optimization: if the partitions are already in the metadata reply OK immediately
-              Left(Errors.NONE)
-            } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
-            }
-          }
-      }
+      val result: ApiResult[(Int, TransactionMetadata)] = getTransactionMetadata(transactionalId, producerId, producerEpoch, partitions)
 
       result match {
         case Left(err) =>
           debug(s"Returning $err error code to client for $transactionalId's AddPartitions request")
           responseCallback(err)
 
-        case Right((coordinatorEpoch, newMetadata)) =>
-          txnManager.appendTransactionToLog(transactionalId, coordinatorEpoch, newMetadata,
+        case Right((coordinatorEpoch, txnMetadata)) =>
+          txnManager.appendTransactionToLog(transactionalId, coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()),
             responseCallback, requestLocal = requestLocal)
       }
     }
   }
+  
+  private def getTransactionMetadata(transactionalId: String,
+                             producerId: Long,
+                             producerEpoch: Short,
+                             partitions: collection.Set[TopicPartition]): ApiResult[(Int, TransactionMetadata)] = {

Review Comment:
   nit: Indentation is off.



##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -330,44 +363,53 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
     } else {
       // try to update the transaction metadata and append the updated metadata to txn log;
       // if there is no such metadata treat it as invalid producerId mapping error.
-      val result: ApiResult[(Int, TxnTransitMetadata)] = txnManager.getTransactionState(transactionalId).flatMap {
-        case None => Left(Errors.INVALID_PRODUCER_ID_MAPPING)
-
-        case Some(epochAndMetadata) =>
-          val coordinatorEpoch = epochAndMetadata.coordinatorEpoch
-          val txnMetadata = epochAndMetadata.transactionMetadata
-
-          // generate the new transaction metadata with added partitions
-          txnMetadata.inLock {
-            if (txnMetadata.producerId != producerId) {
-              Left(Errors.INVALID_PRODUCER_ID_MAPPING)
-            } else if (txnMetadata.producerEpoch != producerEpoch) {
-              Left(Errors.PRODUCER_FENCED)
-            } else if (txnMetadata.pendingTransitionInProgress) {
-              // return a retriable exception to let the client backoff and retry
-              Left(Errors.CONCURRENT_TRANSACTIONS)
-            } else if (txnMetadata.state == PrepareCommit || txnMetadata.state == PrepareAbort) {
-              Left(Errors.CONCURRENT_TRANSACTIONS)
-            } else if (txnMetadata.state == Ongoing && partitions.subsetOf(txnMetadata.topicPartitions)) {
-              // this is an optimization: if the partitions are already in the metadata reply OK immediately
-              Left(Errors.NONE)
-            } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
-            }
-          }
-      }
+      val result: ApiResult[(Int, TransactionMetadata)] = getTransactionMetadata(transactionalId, producerId, producerEpoch, partitions)
 
       result match {
         case Left(err) =>
           debug(s"Returning $err error code to client for $transactionalId's AddPartitions request")
           responseCallback(err)
 
-        case Right((coordinatorEpoch, newMetadata)) =>
-          txnManager.appendTransactionToLog(transactionalId, coordinatorEpoch, newMetadata,
+        case Right((coordinatorEpoch, txnMetadata)) =>
+          txnManager.appendTransactionToLog(transactionalId, coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()),
             responseCallback, requestLocal = requestLocal)
       }
     }
   }
+  
+  private def getTransactionMetadata(transactionalId: String,
+                             producerId: Long,
+                             producerEpoch: Short,
+                             partitions: collection.Set[TopicPartition]): ApiResult[(Int, TransactionMetadata)] = {
+    // try to update the transaction metadata and append the updated metadata to txn log;
+    // if there is no such metadata treat it as invalid producerId mapping error.

Review Comment:
   nit: Is this comment relevant here?



##########
core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala:
##########
@@ -55,22 +65,146 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
     val producerId = 1000L
     val producerEpoch: Short = 0
 
-    val request = new AddPartitionsToTxnRequest.Builder(
-      transactionalId,
-      producerId,
-      producerEpoch,
-      List(createdTopicPartition, nonExistentTopic).asJava)
-      .build()
+    val request =
+      if (version < 4) {
+        AddPartitionsToTxnRequest.Builder.forClient(
+          transactionalId,
+          producerId,
+          producerEpoch,
+          List(createdTopicPartition, nonExistentTopic).asJava
+        ).build()

Review Comment:
   Should we set the version here?



##########
core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala:
##########
@@ -1185,4 +1213,8 @@ class TransactionCoordinatorTest {
   def errorsCallback(ret: Errors): Unit = {
     error = ret
   }
+
+  def verifyPartitionsInTxnCallback(result: AddPartitionsToTxnResult): Unit = {
+    errors = AddPartitionsToTxnResponse.errorsForTransaction(result.topicResults()).asScala.toMap

Review Comment:
   Relying on a global variable is risky here. It would be much better to define the callback within the test itself.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1124831869


##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -317,6 +322,34 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
       }
     }
   }
+  
+  def handleVerifyPartitionsInTransaction(transactionalId: String,
+                                          producerId: Long,
+                                          producerEpoch: Short,
+                                          partitions: collection.Set[TopicPartition],
+                                          responseCallback: VerifyPartitionsCallback): Unit = {
+    if (transactionalId == null || transactionalId.isEmpty) {
+      debug(s"Returning ${Errors.INVALID_REQUEST} error code to client for $transactionalId's AddPartitions request")
+      responseCallback(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitions.map(_ -> Errors.INVALID_REQUEST).toMap.asJava))
+    } else {
+      val result: ApiResult[(Int, TransactionMetadata)] = getTransactionMetadata(transactionalId, producerId, producerEpoch, partitions)
+      
+      result match {
+        case Left(err) =>
+          debug(s"Returning $err error code to client for $transactionalId's AddPartitions request")
+          responseCallback(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitions.map(_ -> err).toMap.asJava))
+
+        case Right((_, txnMetadata)) =>
+          val txnMetadataPartitions = txnMetadata.topicPartitions
+          val addedPartitions = partitions.intersect(txnMetadataPartitions)
+          val nonAddedPartitions = partitions.diff(txnMetadataPartitions)
+          val errors = mutable.Map[TopicPartition, Errors]()
+          addedPartitions.foreach(errors.put(_, Errors.NONE))
+          nonAddedPartitions.foreach(errors.put(_, Errors.INVALID_TXN_STATE))

Review Comment:
   sure



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1128091331


##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -2030,13 +2032,87 @@ class KafkaApisTest {
       val response = capturedResponse.getValue
 
       if (version < 2) {
-        assertEquals(Collections.singletonMap(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors())
+        assertEquals(Collections.singletonMap(topicPartition, Errors.INVALID_PRODUCER_EPOCH), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID))
       } else {
-        assertEquals(Collections.singletonMap(topicPartition, Errors.PRODUCER_FENCED), response.errors())
+        assertEquals(Collections.singletonMap(topicPartition, Errors.PRODUCER_FENCED), response.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID))
       }
     }
   }
 
+  @Test
+  def testBatchedAddPartitionsToTxnRequest(): Unit = {

Review Comment:
   As a follow-up: It seems that the test coverage is pretty low for this API here. It would be great if we could extend it. e.g. authorization failures, validation failures, etc.



##########
core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala:
##########
@@ -231,7 +231,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       resp.errors.get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)).error),
     ApiKeys.INIT_PRODUCER_ID -> ((resp: InitProducerIdResponse) => resp.error),
     ApiKeys.WRITE_TXN_MARKERS -> ((resp: WriteTxnMarkersResponse) => resp.errorsByProducerId.get(producerId).get(tp)),
-    ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(tp)),
+    ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID).get(tp)),

Review Comment:
   As a follow-up: We should cover the new version here as well.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on PR #13231:
URL: https://github.com/apache/kafka/pull/13231#issuecomment-1458557507

   for follow ups: https://issues.apache.org/jira/browse/KAFKA-14790


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107745160


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4
+        public Builder(String transactionalId,
+                       long producerId,
+                       short producerEpoch,
+                       List<TopicPartition> partitions) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
-            this.data = data;
+            this.isClientRequest = true;
+
+            AddPartitionsToTxnTopicCollection topics = compileTopics(partitions);
+
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactionalId(transactionalId)

Review Comment:
   We can do that in the "fromClient" factory method 👍 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107747037


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -66,24 +97,22 @@ public Builder(final String transactionalId,
             AddPartitionsToTxnTopicCollection topics = new AddPartitionsToTxnTopicCollection();
             for (Map.Entry<String, List<Integer>> partitionEntry : partitionMap.entrySet()) {
                 topics.add(new AddPartitionsToTxnTopic()
-                               .setName(partitionEntry.getKey())
-                               .setPartitions(partitionEntry.getValue()));
+                        .setName(partitionEntry.getKey())
+                        .setPartitions(partitionEntry.getValue()));

Review Comment:
   Are you saying it shouldn't be indented here? I removed 8 spaces.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1121005648


##########
core/src/test/scala/unit/kafka/server/KafkaApisTest.scala:
##########
@@ -1962,7 +1962,9 @@ class KafkaApisTest {
         ArgumentMatchers.eq(producerId),
         ArgumentMatchers.eq(epoch),
         ArgumentMatchers.eq(Set(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, partition))),
+        ArgumentMatchers.eq(false),

Review Comment:
   I added tests in RequestServerTest but I guess those are more integration tests. I can add some in KafkaApisTest too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118948279


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   As discussed by @hachikuji and @guozhangwang the idea was if a top level error was set we could skip the rest of the handling. I just set all the fields to the same error for consistency. What is your suggestion? To remove these



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123064502


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   I am not sure whether I like it or not. Those two values in the callback may be error prone in the future. Reading `handleAddPartitionsToTransaction` again, the add and verify steps only have the code validation in common. How feasible would it be to actually add another method for the verification part instead of trying to do everything in `handleAddPartitionsToTransaction`? I suppose that we could extract the code part into a common method used by both.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107589884


##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponseTest.java:
##########
@@ -84,16 +88,59 @@ public void testParse() {
 
         topicCollection.add(topicResult);
 
-        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
-                                                  .setResults(topicCollection)
-                                                  .setThrottleTimeMs(throttleTimeMs);
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
-
         for (short version : ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions()) {

Review Comment:
   I thought maybe we didn't want to redo the top steps every time but sure i can change it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107748914


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +193,41 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        if (version < 4) {
+            final HashMap<TopicPartition, Errors> errors = new HashMap<>();
+            for (TopicPartition partition : partitions()) {
+                errors.put(partition, error);
+            }
+            return new AddPartitionsToTxnResponse(throttleTimeMs, errors);
+        } else {
+            AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();

Review Comment:
   There is 1 error we can return as top level errors if that helps. The rest are topic or partition level.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107757784


##########
clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json:
##########
@@ -22,22 +22,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
       "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
-    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
-      "about": "The results for each topic.", "fields": [
+    { "name": "ResultsByTransaction", "type": "[]AddPartitionsToTxnResult", "versions": "4+",
+      "about": "Results categorized by transactional ID.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction."},
+      { "name": "TopicResults", "type": "[]AddPartitionsToTxnTopicResult", "versions": "4+",
+        "about": "The results for each topic." }
+    ]},
+    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0-3",

Review Comment:
   That's fair. We've done something similiar in the past.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110443918


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   Yep, you got there first 😄. I would probably vote to make the change. Tracking separate connections does not sound attractive. We could still do that even with the modified protocol if there is a noticeable regression for old clients.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1105130904


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4
+        public Builder(String transactionalId,
+                       long producerId,
+                       short producerEpoch,
+                       List<TopicPartition> partitions) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
-            this.data = data;
+            this.isClientRequest = true;
+
+            AddPartitionsToTxnTopicCollection topics = compileTopics(partitions);
+
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactionalId(transactionalId)
+                    .setProducerId(producerId)
+                    .setProducerEpoch(producerEpoch)
+                    .setTopics(topics);
         }
 
-        public Builder(final String transactionalId,
-                       final long producerId,
-                       final short producerEpoch,
-                       final List<TopicPartition> partitions) {
+        public Builder(AddPartitionsToTxnTransactionCollection transactions,
+                       boolean verifyOnly) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
+            this.isClientRequest = false;
 
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactions(transactions)

Review Comment:
   same on the indentation



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4
+        public Builder(String transactionalId,
+                       long producerId,
+                       short producerEpoch,
+                       List<TopicPartition> partitions) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
-            this.data = data;
+            this.isClientRequest = true;
+
+            AddPartitionsToTxnTopicCollection topics = compileTopics(partitions);
+
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactionalId(transactionalId)
+                    .setProducerId(producerId)
+                    .setProducerEpoch(producerEpoch)
+                    .setTopics(topics);

Review Comment:
   the indentation looks off here. i think intellij does this to me too



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {
+        if (cachedPartitionsByTransaction == null) {
+            cachedPartitionsByTransaction = new HashMap<>();
+        }
+        
+        return cachedPartitionsByTransaction.computeIfAbsent(transaction, txn -> {
+            List<TopicPartition> partitions = new ArrayList<>();
+            for (AddPartitionsToTxnTopic topicCollection : data.transactions().find(txn).topics()) {
+                for (Integer partition : topicCollection.partitions()) {
+                    partitions.add(new TopicPartition(topicCollection.name(), partition));
+                }
+            }
+            return partitions;
+        });
+    }
+    
+    public Map<String, List<TopicPartition>> partitionsByTransaction() {
+        if (cachedPartitionsByTransaction != null && cachedPartitionsByTransaction.size() == data.transactions().size()) {

Review Comment:
   can there be any case where the two maps have the same size but store different items?



##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponseTest.java:
##########
@@ -84,16 +88,59 @@ public void testParse() {
 
         topicCollection.add(topicResult);
 
-        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
-                                                  .setResults(topicCollection)
-                                                  .setThrottleTimeMs(throttleTimeMs);
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
-
         for (short version : ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions()) {

Review Comment:
   can we use the parameterized test for the api versions?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -49,28 +52,37 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
     private final AddPartitionsToTxnResponseData data;
 
     private Map<TopicPartition, Errors> cachedErrorsMap = null;
+    
+    private Map<String, Map<TopicPartition, Errors>> cachedAllErrorsMap = null;
 
     public AddPartitionsToTxnResponse(AddPartitionsToTxnResponseData data) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN);
         this.data = data;
     }
 
+    // Only used for versions < 4
     public AddPartitionsToTxnResponse(int throttleTimeMs, Map<TopicPartition, Errors> errors) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN);
 
+        this.data = new AddPartitionsToTxnResponseData()
+                .setThrottleTimeMs(throttleTimeMs)
+                .setResults(topicCollectionForErrors(errors));
+    }
+    
+    private static AddPartitionsToTxnTopicResultCollection topicCollectionForErrors(Map<TopicPartition, Errors> errors) {
         Map<String, AddPartitionsToTxnPartitionResultCollection> resultMap = new HashMap<>();
-
+        

Review Comment:
   nit: maybe just a newline?



##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -352,7 +353,12 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
               // this is an optimization: if the partitions are already in the metadata reply OK immediately
               Left(Errors.NONE)
             } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
+              // If verifyOnly, we should have returned in the step above. If we didn't the partitions are not present in the transaction.

Review Comment:
   does this comment meant that if the request was to verifyOnly, then we should have returned with Left(Errors.NONE) because there should have been a matching transaction with all of the partitions?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4

Review Comment:
   does this mean this constructor is only used for less than 4? how are we enforcing this?



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   are we using this return type anywhere?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2383,68 +2384,101 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val lock = new Object
+    val addPartitionsToTxnRequest = if (request.context.apiVersion() < 4) request.body[AddPartitionsToTxnRequest].normalizeRequest() else request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResults(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      lock synchronized {
+        if (responses.size() == txns.size()) {
+          requestHelper.sendResponseMaybeThrottle(request, createResponse)

Review Comment:
   do we need to hold the lock when sending the response?



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2383,68 +2384,101 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val lock = new Object
+    val addPartitionsToTxnRequest = if (request.context.apiVersion() < 4) request.body[AddPartitionsToTxnRequest].normalizeRequest() else request.body[AddPartitionsToTxnRequest]

Review Comment:
   nit: i think we can separate the if else here



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2383,68 +2384,101 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val lock = new Object
+    val addPartitionsToTxnRequest = if (request.context.apiVersion() < 4) request.body[AddPartitionsToTxnRequest].normalizeRequest() else request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResults(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      lock synchronized {
+        if (responses.size() == txns.size()) {
+          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+        }
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId()

Review Comment:
   nit: parentheses



##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2383,68 +2384,101 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val lock = new Object

Review Comment:
   any reason we don't use `val responses` as the synchronization object?



##########
core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala:
##########
@@ -302,17 +302,42 @@ class TransactionCoordinatorTest {
       any()
     )
   }
-
+  

Review Comment:
   nit: indentation



##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -66,24 +97,22 @@ public Builder(final String transactionalId,
             AddPartitionsToTxnTopicCollection topics = new AddPartitionsToTxnTopicCollection();
             for (Map.Entry<String, List<Integer>> partitionEntry : partitionMap.entrySet()) {
                 topics.add(new AddPartitionsToTxnTopic()
-                               .setName(partitionEntry.getKey())
-                               .setPartitions(partitionEntry.getValue()));
+                        .setName(partitionEntry.getKey())
+                        .setPartitions(partitionEntry.getValue()));

Review Comment:
   i'm seeing a bunch of places with double indentation - not sure if it's intentional or it's done by your editor



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1113450545


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   Actually -- the way this is implemented, we check on a per-transaction basis anyway. Since this is 1:1 corresponded to producer ID (and therefore client version) the only time we could see changes between verify only and not would be when the client upgrades. I may need to think about handling that as a special case.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1113435053


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   I'm wondering do we want to keep the top level verifyOnly as an optimization? Or is it just more trouble than it's worth and maybe won't be used too often.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123503881


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   To clarify -- I'm not sure if being clear on how we want to use it (ie the errors it is used for) is the same question as if we want to include the error on the txns as well. 
   
   As for the latter, I think we are more error prone if we set the error to none. I guess I'm just not sure what we are trying to accomplish by removing them from the lower fields. To be clear, I think I've also seen this pattern on other responses with top level errors.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123501064


##########
clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java:
##########
@@ -1303,11 +1305,13 @@ public void testCommitWithTopicAuthorizationFailureInAddPartitionsInFlight() thr
         Map<TopicPartition, Errors> errors = new HashMap<>();
         errors.put(tp0, Errors.TOPIC_AUTHORIZATION_FAILED);
         errors.put(tp1, Errors.OPERATION_NOT_ATTEMPTED);
+        AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors);
+        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0);
         client.respond(body -> {
             AddPartitionsToTxnRequest request = (AddPartitionsToTxnRequest) body;
-            assertEquals(new HashSet<>(request.partitions()), new HashSet<>(errors.keySet()));
+            assertEquals(new HashSet<>(AddPartitionsToTxnRequest.getPartitions(request.data().v3AndBelowTopics())), new HashSet<>(errors.keySet()));

Review Comment:
   I can take a look. I also used internally to build the request and I think the v4+ version in KafkaApis itself. I can try to make a helper for these tests though.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123517021


##########
clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java:
##########
@@ -1303,11 +1305,13 @@ public void testCommitWithTopicAuthorizationFailureInAddPartitionsInFlight() thr
         Map<TopicPartition, Errors> errors = new HashMap<>();
         errors.put(tp0, Errors.TOPIC_AUTHORIZATION_FAILED);
         errors.put(tp1, Errors.OPERATION_NOT_ATTEMPTED);
+        AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors);
+        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0);
         client.respond(body -> {
             AddPartitionsToTxnRequest request = (AddPartitionsToTxnRequest) body;
-            assertEquals(new HashSet<>(request.partitions()), new HashSet<>(errors.keySet()));
+            assertEquals(new HashSet<>(AddPartitionsToTxnRequest.getPartitions(request.data().v3AndBelowTopics())), new HashSet<>(errors.keySet()));

Review Comment:
   If those are used internally the request as well, we can't really move them.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1120967466


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1328,7 +1328,7 @@ Priority priority() {
         @Override
         public void handleResponse(AbstractResponse response) {
             AddPartitionsToTxnResponse addPartitionsToTxnResponse = (AddPartitionsToTxnResponse) response;
-            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors();
+            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID);

Review Comment:
   What should we do if the check fails? Just have a better error message thrown?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] guozhangwang commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1113415976


##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequestTest.java:
##########
@@ -17,43 +17,138 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopic;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTransaction;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTransactionCollection;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopicCollection;
+import org.apache.kafka.common.message.AddPartitionsToTxnResponseData;
 import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 
 import java.util.ArrayList;
-
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class AddPartitionsToTxnRequestTest {
-
-    private static String transactionalId = "transactionalId";
+    private final String transactionalId1 = "transaction1";
+    private final String transactionalId2 = "transaction2";
     private static int producerId = 10;
     private static short producerEpoch = 1;
     private static int throttleTimeMs = 10;
+    private static TopicPartition tp0 = new TopicPartition("topic", 0);
+    private static TopicPartition tp1 = new TopicPartition("topic", 1);
 
     @ParameterizedTest
     @ApiKeyVersionsSource(apiKey = ApiKeys.ADD_PARTITIONS_TO_TXN)
     public void testConstructor(short version) {
-        List<TopicPartition> partitions = new ArrayList<>();
-        partitions.add(new TopicPartition("topic", 0));
-        partitions.add(new TopicPartition("topic", 1));
+        

Review Comment:
   thx!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13231:
URL: https://github.com/apache/kafka/pull/13231#issuecomment-1441262764

   > Took a quick look at the unstable api change. Looks like some integration tests built specifically for v4 fail with `org.apache.kafka.common.errors.InvalidRequestException: Received request api key ADD_PARTITIONS_TO_TXN with version 4 which is not enabled`
   > 
   > I will need to look into this.
   
   @jolshan I suppose that you have to enable unstable apis in your new integration tests.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123499195


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   I suppose I can do that but it looked like a lot of code duplication. There wasn't a great way to extract out the commonalities but I can try again.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1122921475


##########
clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java:
##########
@@ -1328,7 +1328,7 @@ Priority priority() {
         @Override
         public void handleResponse(AbstractResponse response) {
             AddPartitionsToTxnResponse addPartitionsToTxnResponse = (AddPartitionsToTxnResponse) response;
-            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors();
+            Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors().get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID);

Review Comment:
   Yeah, the check is probably not necessary. By the way, I find the idea of having `V3_AND_BELOW_TXN_ID` for old version a bit confusing. I was wondering if using `addPartitionsToTxnResponse.data().resultsByTopicV3AndBelow()` would be a better alternative here. We only iterate over the Map so the Map is not strictly required here. Have you considered something like this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1128154589


##########
core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala:
##########
@@ -231,7 +231,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       resp.errors.get(new ConfigResource(ConfigResource.Type.TOPIC, tp.topic)).error),
     ApiKeys.INIT_PRODUCER_ID -> ((resp: InitProducerIdResponse) => resp.error),
     ApiKeys.WRITE_TXN_MARKERS -> ((resp: WriteTxnMarkersResponse) => resp.errorsByProducerId.get(producerId).get(tp)),
-    ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(tp)),
+    ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID).get(tp)),

Review Comment:
   The new version doesn't really use authorizer, so I wasn't sure if it was needed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107744819


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4

Review Comment:
   Maybe Jason's suggestion below will help -- using factory methods.
   I'm not sure there is any way to "enforce" but to guide folks to use the right methods



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107745666


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -35,21 +44,43 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
     private final AddPartitionsToTxnRequestData data;
 
     private List<TopicPartition> cachedPartitions = null;
+    
+    private Map<String, List<TopicPartition>> cachedPartitionsByTransaction = null;
+
+    private final short version;
 
     public static class Builder extends AbstractRequest.Builder<AddPartitionsToTxnRequest> {
         public final AddPartitionsToTxnRequestData data;
+        public final boolean isClientRequest;
 
-        public Builder(final AddPartitionsToTxnRequestData data) {
+        // Only used for versions < 4
+        public Builder(String transactionalId,
+                       long producerId,
+                       short producerEpoch,
+                       List<TopicPartition> partitions) {
             super(ApiKeys.ADD_PARTITIONS_TO_TXN);
-            this.data = data;
+            this.isClientRequest = true;
+
+            AddPartitionsToTxnTopicCollection topics = compileTopics(partitions);
+
+            this.data = new AddPartitionsToTxnRequestData()
+                    .setTransactionalId(transactionalId)
+                    .setProducerId(producerId)
+                    .setProducerEpoch(producerEpoch)
+                    .setTopics(topics);

Review Comment:
   Are you just saying I should do 4 instead of 8? I didn't know if there was a formalized decision there



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107751395


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -49,28 +52,37 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
     private final AddPartitionsToTxnResponseData data;
 
     private Map<TopicPartition, Errors> cachedErrorsMap = null;
+    
+    private Map<String, Map<TopicPartition, Errors>> cachedAllErrorsMap = null;
 
     public AddPartitionsToTxnResponse(AddPartitionsToTxnResponseData data) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN);
         this.data = data;
     }
 
+    // Only used for versions < 4

Review Comment:
   I left it this way for minimal code changes but we could get rid of this one.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] guozhangwang commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107858293


##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequestTest.java:
##########
@@ -17,43 +17,138 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopic;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTransaction;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTransactionCollection;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopicCollection;
+import org.apache.kafka.common.message.AddPartitionsToTxnResponseData;
 import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 
 import java.util.ArrayList;
-
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class AddPartitionsToTxnRequestTest {
-
-    private static String transactionalId = "transactionalId";
+    private final String transactionalId1 = "transaction1";
+    private final String transactionalId2 = "transaction2";
     private static int producerId = 10;
     private static short producerEpoch = 1;
     private static int throttleTimeMs = 10;
+    private static TopicPartition tp0 = new TopicPartition("topic", 0);
+    private static TopicPartition tp1 = new TopicPartition("topic", 1);
 
     @ParameterizedTest
     @ApiKeyVersionsSource(apiKey = ApiKeys.ADD_PARTITIONS_TO_TXN)
     public void testConstructor(short version) {
-        List<TopicPartition> partitions = new ArrayList<>();
-        partitions.add(new TopicPartition("topic", 0));
-        partitions.add(new TopicPartition("topic", 1));
+        

Review Comment:
   Do we already have a test coverage with the old `Builder` constructor to check that `verifyOnly` would default to `false`?



##########
clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json:
##########
@@ -22,22 +22,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
       "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
-    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
-      "about": "The results for each topic.", "fields": [
+    { "name": "ResultsByTransaction", "type": "[]AddPartitionsToTxnResult", "versions": "4+",
+      "about": "Results categorized by transactional ID.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction."},
+      { "name": "TopicResults", "type": "[]AddPartitionsToTxnTopicResult", "versions": "4+",
+        "about": "The results for each topic." }
+    ]},
+    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0-3",

Review Comment:
   nit: maybe rename this field from `Results` to `TransactionResultsByTopic`?



##########
clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json:
##########
@@ -22,22 +22,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
       "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
-    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
-      "about": "The results for each topic.", "fields": [
+    { "name": "ResultsByTransaction", "type": "[]AddPartitionsToTxnResult", "versions": "4+",
+      "about": "Results categorized by transactional ID.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction."},
+      { "name": "TopicResults", "type": "[]AddPartitionsToTxnTopicResult", "versions": "4+",
+        "about": "The results for each topic." }
+    ]},
+    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0-3",
+      "about": "The results for each topic." }
+  ],
+  "commonStructs": [
+    { "name": "AddPartitionsToTxnTopicResult", "versions": "0+", "fields": [
       { "name": "Name", "type": "string", "versions": "0+", "mapKey": true, "entityType": "topicName",
         "about": "The topic name." },
-      { "name": "Results", "type": "[]AddPartitionsToTxnPartitionResult", "versions": "0+", 
-        "about": "The results for each partition", "fields": [
-        { "name": "PartitionIndex", "type": "int32", "versions": "0+", "mapKey": true,
-          "about": "The partition indexes." },
-        { "name": "ErrorCode", "type": "int16", "versions": "0+",
-          "about": "The response error code."}
-      ]}
+      { "name": "Results", "type": "[]AddPartitionsToTxnPartitionResult", "versions": "0+",

Review Comment:
   ditto here, maybe `TopicResultsByPartition`?



##########
core/src/main/scala/kafka/coordinator/transaction/TransactionCoordinator.scala:
##########
@@ -352,7 +353,12 @@ class TransactionCoordinator(txnConfig: TransactionConfig,
               // this is an optimization: if the partitions are already in the metadata reply OK immediately
               Left(Errors.NONE)
             } else {
-              Right(coordinatorEpoch, txnMetadata.prepareAddPartitions(partitions.toSet, time.milliseconds()))
+              // If verifyOnly, we should have returned in the step above. If we didn't the partitions are not present in the transaction.
+              if (verifyOnly) {
+                Left(Errors.INVALID_TXN_STATE)

Review Comment:
   +1 here, since we do have the finest error-code level at the per-partition level we can leverage it to be more detailed.
   
   As for whether we should also add a global error code: in general I'd like that idea, and I think there are a few error codes: auth, unsupported_version, unsupported_format (we also have a `UNSUPPORTED_FOR_MESSAGE_FORMAT` for that), that should always be global to save the sender time to dig into lower-leveled ones. And then for different request type there are some global errors based on just the global fields --- for this case, the only global field left now is `verifyOnly` so that we probably do not have additional error codes. I vaguely remember there's a KIP proposing such a principle but I cannot find the KIP now.
   
   If we agree this is a generally good idea then let's start with this one first :P



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1118981498


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,39 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  // The AddPartitionsToTxnRequest version 4 API is added as part of KIP-890 and is still
+  // under developement. Hence, the API is not exposed by default by brokers
+  // unless explicitely enabled.
+  "latestVersionUnstable": true,
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
-      "about": "The transactional id corresponding to the transaction."},
-    { "name": "ProducerId", "type": "int64", "versions": "0+", "entityType": "producerId",
+    { "name": "Transactions", "type": "[]AddPartitionsToTxnTransaction", "versions":  "4+",
+      "about": "List of transactions to add partitions to.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction." },
+      { "name": "ProducerId", "type": "int64", "versions": "4+", "entityType": "producerId",
+        "about": "Current producer id in use by the transactional id." },
+      { "name": "ProducerEpoch", "type": "int16", "versions": "4+",
+        "about": "Current epoch associated with the producer id." },
+      { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,
+        "about": "Boolean to signify if we want to check if the partition is in the transaction rather than add it." },
+      { "name": "Topics", "type": "[]AddPartitionsToTxnTopic", "versions": "4+",
+        "about": "The partitions to add to the transaction." }
+    ]},
+    { "name": "V3AndBelowTransactionalId", "type": "string", "versions": "0-3", "entityType": "transactionalId",

Review Comment:
   Ack. That seems fine.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1122411269


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()
           }
 
-          requestHelper.sendResponseMaybeThrottle(request, createResponse)
+          txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+            transaction.producerId,
+            transaction.producerEpoch,
+            authorizedPartitions,
+            transaction.verifyOnly,
+            sendResponseCallback,
+            sendVerifyResponseCallback,

Review Comment:
   I've changed to one callback, but I decided to go with a different approach where we either provide one error or the map. I think it would be annoying to convert the errors for all partitions and then potentially have to redo it to convert the error for v2 and below. Let me know if there is an issue with this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] dajac commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1122925511


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -118,11 +123,78 @@ public AddPartitionsToTxnRequestData data() {
 
     @Override
     public AddPartitionsToTxnResponse getErrorResponse(int throttleTimeMs, Throwable e) {
-        final HashMap<TopicPartition, Errors> errors = new HashMap<>();
-        for (TopicPartition partition : partitions()) {
-            errors.put(partition, Errors.forException(e));
+        Errors error = Errors.forException(e);
+        AddPartitionsToTxnResponseData response = new AddPartitionsToTxnResponseData();
+        if (version < 4) {
+            response.setResultsByTopicV3AndBelow(errorResponseForTopics(data.v3AndBelowTopics(), error));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            for (AddPartitionsToTxnTransaction transaction : data().transactions()) {
+                results.add(errorResponseForTransaction(transaction.transactionalId(), error));
+            }
+            response.setResultsByTransaction(results);
+            response.setErrorCode(error.code());

Review Comment:
   Yeah, I like the top level error code for that reason. However, we must be clear on how we want to use it. I suppose that we can only use it for cluster authorization failure and unexpected errors failing the entire request/response.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107771622


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -66,24 +97,22 @@ public Builder(final String transactionalId,
             AddPartitionsToTxnTopicCollection topics = new AddPartitionsToTxnTopicCollection();
             for (Map.Entry<String, List<Integer>> partitionEntry : partitionMap.entrySet()) {
                 topics.add(new AddPartitionsToTxnTopic()
-                               .setName(partitionEntry.getKey())
-                               .setPartitions(partitionEntry.getValue()));
+                        .setName(partitionEntry.getKey())
+                        .setPartitions(partitionEntry.getValue()));
             }
-
-            this.data = new AddPartitionsToTxnRequestData()
-                            .setTransactionalId(transactionalId)
-                            .setProducerId(producerId)
-                            .setProducerEpoch(producerEpoch)
-                            .setTopics(topics);
+            return topics;
         }
 
         @Override
         public AddPartitionsToTxnRequest build(short version) {
-            return new AddPartitionsToTxnRequest(data, version);
+            short clampedVersion = (isClientRequest && version > 3) ? 3 : version;

Review Comment:
   I didn't see if we directly built the request for the AddPartitionsToTxn requst or passed the abstract request to a queue (where we lose the type 😅 ) I will check again though



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107626067


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   ?? are you saying the method isn't used? It's used on line 168



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107649386


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   the method can be of type void, no?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110268297


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   It's a fair point. I'll think on it a bit.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110442234


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",

Review Comment:
   Would it make sense to set `latestVersionUnstable` so that we reserve some flexibility to change the API after we merge this?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1123774975


##########
clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java:
##########
@@ -1303,11 +1305,13 @@ public void testCommitWithTopicAuthorizationFailureInAddPartitionsInFlight() thr
         Map<TopicPartition, Errors> errors = new HashMap<>();
         errors.put(tp0, Errors.TOPIC_AUTHORIZATION_FAILED);
         errors.put(tp1, Errors.OPERATION_NOT_ATTEMPTED);
+        AddPartitionsToTxnResult result = AddPartitionsToTxnResponse.resultForTransaction(AddPartitionsToTxnResponse.V3_AND_BELOW_TXN_ID, errors);
+        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData().setResultsByTopicV3AndBelow(result.topicResults()).setThrottleTimeMs(0);
         client.respond(body -> {
             AddPartitionsToTxnRequest request = (AddPartitionsToTxnRequest) body;
-            assertEquals(new HashSet<>(request.partitions()), new HashSet<>(errors.keySet()));
+            assertEquals(new HashSet<>(AddPartitionsToTxnRequest.getPartitions(request.data().v3AndBelowTopics())), new HashSet<>(errors.keySet()));

Review Comment:
   The one in the request is used in a place that iterates through the topic objects so it is actually ok. I made a helper in the TransactionManagerTest since it uses the phrase 3 times. I left the final remaining usage of this in the last test file since I think its ok to have once and doesn't make sense to use a helper. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1124831349


##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponseTest.java:
##########
@@ -58,42 +65,75 @@ public void setUp() {
         errorsMap.put(tp2, errorTwo);
     }
 
-    @Test
-    public void testConstructorWithErrorResponse() {
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(throttleTimeMs, errorsMap);
-
-        assertEquals(expectedErrorCounts, response.errorCounts());
-        assertEquals(throttleTimeMs, response.throttleTimeMs());
-    }
-
-    @Test
-    public void testParse() {
-
+    @ParameterizedTest
+    @ApiKeyVersionsSource(apiKey = ApiKeys.ADD_PARTITIONS_TO_TXN)
+    public void testParse(short version) {
         AddPartitionsToTxnTopicResultCollection topicCollection = new AddPartitionsToTxnTopicResultCollection();
 
         AddPartitionsToTxnTopicResult topicResult = new AddPartitionsToTxnTopicResult();
         topicResult.setName(topicOne);
 
-        topicResult.results().add(new AddPartitionsToTxnPartitionResult()
-                                      .setErrorCode(errorOne.code())
+        topicResult.resultsByPartition().add(new AddPartitionsToTxnPartitionResult()
+                                      .setPartitionErrorCode(errorOne.code())
                                       .setPartitionIndex(partitionOne));
 
-        topicResult.results().add(new AddPartitionsToTxnPartitionResult()
-                                      .setErrorCode(errorTwo.code())
+        topicResult.resultsByPartition().add(new AddPartitionsToTxnPartitionResult()
+                                      .setPartitionErrorCode(errorTwo.code())
                                       .setPartitionIndex(partitionTwo));
 
         topicCollection.add(topicResult);
+            
+        if (version < 4) {
+            AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
+                    .setResultsByTopicV3AndBelow(topicCollection)
+                    .setThrottleTimeMs(throttleTimeMs);
+            AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
 
-        AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
-                                                  .setResults(topicCollection)
-                                                  .setThrottleTimeMs(throttleTimeMs);
-        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
-
-        for (short version : ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions()) {
             AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serialize(version), version);
             assertEquals(expectedErrorCounts, parsedResponse.errorCounts());
             assertEquals(throttleTimeMs, parsedResponse.throttleTimeMs());
             assertEquals(version >= 1, parsedResponse.shouldClientThrottle(version));
+        } else {
+            AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+            results.add(new AddPartitionsToTxnResult().setTransactionalId("txn1").setTopicResults(topicCollection));
+            
+            // Create another transaction with new name and errorOne for a single partition.
+            Map<TopicPartition, Errors> txnTwoExpectedErrors = Collections.singletonMap(tp2, errorOne);
+            results.add(AddPartitionsToTxnResponse.resultForTransaction("txn2", txnTwoExpectedErrors));
+
+            AddPartitionsToTxnResponseData data = new AddPartitionsToTxnResponseData()
+                    .setResultsByTransaction(results)
+                    .setThrottleTimeMs(throttleTimeMs);
+            AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(data);
+
+            Map<Errors, Integer> newExpectedErrorCounts = new HashMap<>();
+            newExpectedErrorCounts.put(Errors.NONE, 1); // top level error
+            newExpectedErrorCounts.put(errorOne, 2);
+            newExpectedErrorCounts.put(errorTwo, 1);
+            
+            AddPartitionsToTxnResponse parsedResponse = AddPartitionsToTxnResponse.parse(response.serialize(version), version);
+            assertEquals(txnTwoExpectedErrors, errorsForTransaction(response.getTransactionTopicResults("txn2")));
+            assertEquals(newExpectedErrorCounts, parsedResponse.errorCounts());
+            assertEquals(throttleTimeMs, parsedResponse.throttleTimeMs());
+            assertTrue(parsedResponse.shouldClientThrottle(version));
         }
     }
+    
+    @Test
+    public void testBatchedErrors() {
+        Map<TopicPartition, Errors> txn1Errors = Collections.singletonMap(tp1, errorOne);
+        Map<TopicPartition, Errors> txn2Errors = Collections.singletonMap(tp1, errorOne);
+        
+        AddPartitionsToTxnResult transaction1 = AddPartitionsToTxnResponse.resultForTransaction("txn1", txn1Errors);
+        AddPartitionsToTxnResult transaction2 = AddPartitionsToTxnResponse.resultForTransaction("txn2", txn2Errors);
+        
+        AddPartitionsToTxnResultCollection results = new AddPartitionsToTxnResultCollection();
+        results.add(transaction1);
+        results.add(transaction2);
+        
+        AddPartitionsToTxnResponse response = new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setResultsByTransaction(results));
+        
+        assertEquals(txn1Errors, errorsForTransaction(response.getTransactionTopicResults("txn1")));
+        assertEquals(txn2Errors, errorsForTransaction(response.getTransactionTopicResults("txn2")));
+    }

Review Comment:
   (ErrorCounts also uses this method)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110117043


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   Is the idea that we will have some old producers (verifyOnly) and some new ones?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1113435053


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   I'm wondering do we want to keep the top level verifyOnly as an optimization? Or is it just more trouble than it's worth and maybe won't be used too often.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1120991153


##########
core/src/main/scala/kafka/server/KafkaApis.scala:
##########
@@ -2384,68 +2385,116 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (config.interBrokerProtocolVersion.isLessThan(version))
       throw new UnsupportedVersionException(s"inter.broker.protocol.version: ${config.interBrokerProtocolVersion.version} is less than the required version: ${version.version}")
   }
-
-  def handleAddPartitionToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
+  def handleAddPartitionsToTxnRequest(request: RequestChannel.Request, requestLocal: RequestLocal): Unit = {
     ensureInterBrokerVersion(IBP_0_11_0_IV0)
-    val addPartitionsToTxnRequest = request.body[AddPartitionsToTxnRequest]
-    val transactionalId = addPartitionsToTxnRequest.data.transactionalId
-    val partitionsToAdd = addPartitionsToTxnRequest.partitions.asScala
-    if (!authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId))
-      requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
-      val authorizedPartitions = mutable.Set[TopicPartition]()
-
-      val authorizedTopics = authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
-        partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic)
-      for (topicPartition <- partitionsToAdd) {
-        if (!authorizedTopics.contains(topicPartition.topic))
-          unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
-        else if (!metadataCache.contains(topicPartition))
-          nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
-        else
-          authorizedPartitions.add(topicPartition)
+    val addPartitionsToTxnRequest =
+      if (request.context.apiVersion() < 4) 
+        request.body[AddPartitionsToTxnRequest].normalizeRequest() 
+      else 
+        request.body[AddPartitionsToTxnRequest]
+    val version = addPartitionsToTxnRequest.version
+    val responses = new AddPartitionsToTxnResultCollection()
+    val partitionsByTransaction = addPartitionsToTxnRequest.partitionsByTransaction()
+    
+    // Newer versions of the request should only come from other brokers.
+    if (version >= 4) authHelper.authorizeClusterOperation(request, CLUSTER_ACTION)
+
+    // V4 requests introduced batches of transactions. We need all transactions to be handled before sending the 
+    // response so there are a few differences in handling errors and sending responses.
+    def createResponse(requestThrottleMs: Int): AbstractResponse = {
+      if (version < 4) {
+        // There will only be one response in data. Add it to the response data object.
+        val data = new AddPartitionsToTxnResponseData()
+        responses.forEach(result => {
+          data.setResultsByTopicV3AndBelow(result.topicResults())
+          data.setThrottleTimeMs(requestThrottleMs)
+        })
+        new AddPartitionsToTxnResponse(data)
+      } else {
+        new AddPartitionsToTxnResponse(new AddPartitionsToTxnResponseData().setThrottleTimeMs(requestThrottleMs).setResultsByTransaction(responses))
       }
+    }
 
-      if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
-        // Any failed partition check causes the entire request to fail. We send the appropriate error codes for the
-        // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
-        // the authorization check to indicate that they were not added to the transaction.
-        val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
-          authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
-        requestHelper.sendResponseMaybeThrottle(request, requestThrottleMs =>
-          new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
+    val txns = addPartitionsToTxnRequest.data.transactions
+    def maybeSendResponse(): Unit = {
+      var canSend = false
+      responses.synchronized {
+        if (responses.size() == txns.size()) {
+          canSend = true
+        }
+      }
+      if (canSend) {
+        requestHelper.sendResponseMaybeThrottle(request, createResponse)
+      }
+    }
+
+    txns.forEach( transaction => {
+      val transactionalId = transaction.transactionalId
+      val partitionsToAdd = partitionsByTransaction.get(transactionalId).asScala
+      
+      // Versions < 4 come from clients and must be authorized to write for the given transaction and for the given topics.
+      if (version < 4 && !authHelper.authorize(request.context, WRITE, TRANSACTIONAL_ID, transactionalId)) {
+        responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+        maybeSendResponse()
       } else {
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val finalError =
-              if (addPartitionsToTxnRequest.version < 2 && error == Errors.PRODUCER_FENCED) {
+        val unauthorizedTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val nonExistingTopicErrors = mutable.Map[TopicPartition, Errors]()
+        val authorizedPartitions = mutable.Set[TopicPartition]()
+
+        val authorizedTopics = if (version < 4) authHelper.filterByAuthorized(request.context, WRITE, TOPIC,
+          partitionsToAdd.filterNot(tp => Topic.isInternal(tp.topic)))(_.topic) else partitionsToAdd.map(_.topic).toSet
+        for (topicPartition <- partitionsToAdd) {
+          if (!authorizedTopics.contains(topicPartition.topic))
+            unauthorizedTopicErrors += topicPartition -> Errors.TOPIC_AUTHORIZATION_FAILED
+          else if (!metadataCache.contains(topicPartition))
+            nonExistingTopicErrors += topicPartition -> Errors.UNKNOWN_TOPIC_OR_PARTITION
+          else
+            authorizedPartitions.add(topicPartition)
+        }
+
+        if (unauthorizedTopicErrors.nonEmpty || nonExistingTopicErrors.nonEmpty) {
+          // Any failed partition check causes the entire transaction to fail. We send the appropriate error codes for the
+          // partitions which failed, and an 'OPERATION_NOT_ATTEMPTED' error code for the partitions which succeeded
+          // the authorization check to indicate that they were not added to the transaction.
+          val partitionErrors = unauthorizedTopicErrors ++ nonExistingTopicErrors ++
+            authorizedPartitions.map(_ -> Errors.OPERATION_NOT_ATTEMPTED)
+          responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, partitionErrors.asJava))
+          maybeSendResponse()
+        } else {
+          def sendResponseCallback(error: Errors): Unit = {
+            val finalError = {
+              if (version < 2 && error == Errors.PRODUCER_FENCED) {
                 // For older clients, they could not understand the new PRODUCER_FENCED error code,
                 // so we need to return the old INVALID_PRODUCER_EPOCH to have the same client handling logic.
                 Errors.INVALID_PRODUCER_EPOCH
               } else {
                 error
               }
-
-            val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
-              partitionsToAdd.map{tp => (tp, finalError)}.toMap.asJava)
-            trace(s"Completed $transactionalId's AddPartitionsToTxnRequest with partitions $partitionsToAdd: errors: $error from client ${request.header.clientId}")
-            responseBody
+            }
+            responses.synchronized {
+              responses.add(addPartitionsToTxnRequest.errorResponseForTransaction(transactionalId, finalError))
+            }
+            maybeSendResponse()
+          }
+          
+          def sendVerifyResponseCallback(errors: Map[TopicPartition, Errors]): Unit = {
+            responses.synchronized {
+              responses.add(AddPartitionsToTxnResponse.resultForTransaction(transactionalId, errors.asJava))
+            }
+            maybeSendResponse()

Review Comment:
   I guess I can just supply what to add to the response. I will redo this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1124937664


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -80,45 +105,44 @@ topicName, new AddPartitionsToTxnPartitionResultCollection()
         AddPartitionsToTxnTopicResultCollection topicCollection = new AddPartitionsToTxnTopicResultCollection();
         for (Map.Entry<String, AddPartitionsToTxnPartitionResultCollection> entry : resultMap.entrySet()) {
             topicCollection.add(new AddPartitionsToTxnTopicResult()
-                                    .setName(entry.getKey())
-                                    .setResults(entry.getValue()));
+                .setName(entry.getKey())
+                .setResultsByPartition(entry.getValue()));
         }
-
-        this.data = new AddPartitionsToTxnResponseData()
-                        .setThrottleTimeMs(throttleTimeMs)
-                        .setResults(topicCollection);
+        return topicCollection;
     }
 
-    @Override
-    public int throttleTimeMs() {
-        return data.throttleTimeMs();
+    public static AddPartitionsToTxnResult resultForTransaction(String transactionalId, Map<TopicPartition, Errors> errors) {
+        return new AddPartitionsToTxnResult().setTransactionalId(transactionalId).setTopicResults(topicCollectionForErrors(errors));
     }
 
-    @Override
-    public void maybeSetThrottleTimeMs(int throttleTimeMs) {
-        data.setThrottleTimeMs(throttleTimeMs);
+    public AddPartitionsToTxnTopicResultCollection getTransactionTopicResults(String transactionalId) {
+        return data.resultsByTransaction().find(transactionalId).topicResults();
     }
 
-    public Map<TopicPartition, Errors> errors() {
-        if (cachedErrorsMap != null) {
-            return cachedErrorsMap;
-        }
-
-        cachedErrorsMap = new HashMap<>();
-
-        for (AddPartitionsToTxnTopicResult topicResult : this.data.results()) {
-            for (AddPartitionsToTxnPartitionResult partitionResult : topicResult.results()) {
-                cachedErrorsMap.put(new TopicPartition(
-                        topicResult.name(), partitionResult.partitionIndex()),
-                    Errors.forCode(partitionResult.errorCode()));
+    public static Map<TopicPartition, Errors> errorsForTransaction(AddPartitionsToTxnTopicResultCollection topicCollection) {
+        Map<TopicPartition, Errors> topicResults = new HashMap<>();
+        for (AddPartitionsToTxnTopicResult topicResult : topicCollection) {
+            for (AddPartitionsToTxnPartitionResult partitionResult : topicResult.resultsByPartition()) {
+                topicResults.put(
+                    new TopicPartition(topicResult.name(), partitionResult.partitionIndex()), Errors.forCode(partitionResult.partitionErrorCode()));
             }
         }
-        return cachedErrorsMap;
+        return topicResults;
     }
 
     @Override
     public Map<Errors, Integer> errorCounts() {
-        return errorCounts(errors().values());
+        List<Errors> allErrors = new ArrayList<>();
+
+        // If we are not using this field, we have request 4 or later
+        if (this.data.resultsByTopicV3AndBelow().size() == 0) {
+            allErrors.add(Errors.forCode(data.errorCode()));

Review Comment:
   I create allErrors because I use addAll for the individual transactions. I can place this code after I create error counts, but it doesn't really seem to accomplish much. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107662689


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -66,24 +97,22 @@ public Builder(final String transactionalId,
             AddPartitionsToTxnTopicCollection topics = new AddPartitionsToTxnTopicCollection();
             for (Map.Entry<String, List<Integer>> partitionEntry : partitionMap.entrySet()) {
                 topics.add(new AddPartitionsToTxnTopic()
-                               .setName(partitionEntry.getKey())
-                               .setPartitions(partitionEntry.getValue()));
+                        .setName(partitionEntry.getKey())
+                        .setPartitions(partitionEntry.getValue()));
             }
-
-            this.data = new AddPartitionsToTxnRequestData()
-                            .setTransactionalId(transactionalId)
-                            .setProducerId(producerId)
-                            .setProducerEpoch(producerEpoch)
-                            .setTopics(topics);
+            return topics;
         }
 
         @Override
         public AddPartitionsToTxnRequest build(short version) {
-            return new AddPartitionsToTxnRequest(data, version);
+            short clampedVersion = (isClientRequest && version > 3) ? 3 : version;

Review Comment:
   Hmm I didn't see such an option in the builder but maybe I'm missing something. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107662160


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   not as it is written. We return on line 150. I originally thought this could be useful if we just wanted the list of the partitions. but since it is private I can change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jeffkbkim commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107649386


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -101,15 +130,61 @@ public String toString() {
     public AddPartitionsToTxnRequest(final AddPartitionsToTxnRequestData data, short version) {
         super(ApiKeys.ADD_PARTITIONS_TO_TXN, version);
         this.data = data;
+        this.version = version;
     }
-
+    
+    // Only used for versions < 4
     public List<TopicPartition> partitions() {
         if (cachedPartitions != null) {
             return cachedPartitions;
         }
         cachedPartitions = Builder.getPartitions(data);
         return cachedPartitions;
     }
+    
+    private List<TopicPartition> partitionsForTransaction(String transaction) {

Review Comment:
   i'm saying the method can be of type void, no?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107759467


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java:
##########
@@ -66,24 +97,22 @@ public Builder(final String transactionalId,
             AddPartitionsToTxnTopicCollection topics = new AddPartitionsToTxnTopicCollection();
             for (Map.Entry<String, List<Integer>> partitionEntry : partitionMap.entrySet()) {
                 topics.add(new AddPartitionsToTxnTopic()
-                               .setName(partitionEntry.getKey())
-                               .setPartitions(partitionEntry.getValue()));
+                        .setName(partitionEntry.getKey())
+                        .setPartitions(partitionEntry.getValue()));
             }
-
-            this.data = new AddPartitionsToTxnRequestData()
-                            .setTransactionalId(transactionalId)
-                            .setProducerId(producerId)
-                            .setProducerEpoch(producerEpoch)
-                            .setTopics(topics);
+            return topics;
         }
 
         @Override
         public AddPartitionsToTxnRequest build(short version) {
-            return new AddPartitionsToTxnRequest(data, version);
+            short clampedVersion = (isClientRequest && version > 3) ? 3 : version;

Review Comment:
   It comes from `AbstractRequest.Builder`. You can pass the min and max versions in the constructor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107757784


##########
clients/src/main/resources/common/message/AddPartitionsToTxnResponse.json:
##########
@@ -22,22 +22,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
     { "name": "ThrottleTimeMs", "type": "int32", "versions": "0+",
       "about": "Duration in milliseconds for which the request was throttled due to a quota violation, or zero if the request did not violate any quota." },
-    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0+",
-      "about": "The results for each topic.", "fields": [
+    { "name": "ResultsByTransaction", "type": "[]AddPartitionsToTxnResult", "versions": "4+",
+      "about": "Results categorized by transactional ID.", "fields": [
+      { "name": "TransactionalId", "type": "string", "versions": "4+", "mapKey": true, "entityType": "transactionalId",
+        "about": "The transactional id corresponding to the transaction."},
+      { "name": "TopicResults", "type": "[]AddPartitionsToTxnTopicResult", "versions": "4+",
+        "about": "The results for each topic." }
+    ]},
+    { "name": "Results", "type": "[]AddPartitionsToTxnTopicResult", "versions": "0-3",

Review Comment:
   That's fair. We've done something similar in the past.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1107751863


##########
clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java:
##########
@@ -99,6 +112,7 @@ public void maybeSetThrottleTimeMs(int throttleTimeMs) {
         data.setThrottleTimeMs(throttleTimeMs);
     }
 
+    // Only used for versions < 4

Review Comment:
   I'm not sure how to get around some of these since the return type is different.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan merged pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan merged PR #13231:
URL: https://github.com/apache/kafka/pull/13231


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1124925652


##########
core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala:
##########
@@ -1185,4 +1213,8 @@ class TransactionCoordinatorTest {
   def errorsCallback(ret: Errors): Unit = {
     error = ret
   }
+
+  def verifyPartitionsInTxnCallback(result: AddPartitionsToTxnResult): Unit = {
+    errors = AddPartitionsToTxnResponse.errorsForTransaction(result.topicResults()).asScala.toMap

Review Comment:
   I'm just followed the pattern of the errorsCallback. 😅 I'm not going to redo that, but I can redo this if it's an issue.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110247044


##########
clients/src/test/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequestTest.java:
##########
@@ -17,43 +17,138 @@
 package org.apache.kafka.common.requests;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopic;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTransaction;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTransactionCollection;
+import org.apache.kafka.common.message.AddPartitionsToTxnRequestData.AddPartitionsToTxnTopicCollection;
+import org.apache.kafka.common.message.AddPartitionsToTxnResponseData;
 import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 
 import java.util.ArrayList;
-
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+
+import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class AddPartitionsToTxnRequestTest {
-
-    private static String transactionalId = "transactionalId";
+    private final String transactionalId1 = "transaction1";
+    private final String transactionalId2 = "transaction2";
     private static int producerId = 10;
     private static short producerEpoch = 1;
     private static int throttleTimeMs = 10;
+    private static TopicPartition tp0 = new TopicPartition("topic", 0);
+    private static TopicPartition tp1 = new TopicPartition("topic", 1);
 
     @ParameterizedTest
     @ApiKeyVersionsSource(apiKey = ApiKeys.ADD_PARTITIONS_TO_TXN)
     public void testConstructor(short version) {
-        List<TopicPartition> partitions = new ArrayList<>();
-        partitions.add(new TopicPartition("topic", 0));
-        partitions.add(new TopicPartition("topic", 1));
+        

Review Comment:
   We don't but it's easy to add



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] hachikuji commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "hachikuji (via GitHub)" <gi...@apache.org>.
hachikuji commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1110441561


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",
   "flexibleVersions": "3+",
   "fields": [
-    { "name": "TransactionalId", "type": "string", "versions": "0+", "entityType": "transactionalId",
+    { "name": "VerifyOnly", "type": "bool", "versions": "4+", "default": false,

Review Comment:
   Yeah, interesting point. I could see moving `VerifyOnly` to the level of `transactionalId`. An interesting corollary if we do batch both modes together is that verify-only requests may end up blocking on replication even though we are only checking the state in memory. This would kind of penalize old clients, but maybe that's acceptable. Unless we used separate connections for each mode, perhaps it is unavoidable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on PR #13231:
URL: https://github.com/apache/kafka/pull/13231#issuecomment-1441095454

   Took a quick look at the unstable api change. Looks like some integration tests built specifically for v4 fail with `org.apache.kafka.common.errors.InvalidRequestException: Received request api key ADD_PARTITIONS_TO_TXN with version 4 which is not enabled` 
   
   I will need to look into this.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] guozhangwang commented on pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "guozhangwang (via GitHub)" <gi...@apache.org>.
guozhangwang commented on PR #13231:
URL: https://github.com/apache/kafka/pull/13231#issuecomment-1444715595

   @dajac @hachikuji if you do not have further comments, we can proceed and merge it then?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] jolshan commented on a diff in pull request #13231: KAFKA-14402: Update AddPartitionsToTxn protocol to batch and handle verifyOnly requests

Posted by "jolshan (via GitHub)" <gi...@apache.org>.
jolshan commented on code in PR #13231:
URL: https://github.com/apache/kafka/pull/13231#discussion_r1113432465


##########
clients/src/main/resources/common/message/AddPartitionsToTxnRequest.json:
##########
@@ -23,17 +23,35 @@
   // Version 2 adds the support for new error code PRODUCER_FENCED.
   //
   // Version 3 enables flexible versions.
-  "validVersions": "0-3",
+  //
+  // Version 4 adds VerifyOnly field to check if partitions are already in transaction and adds support to batch multiple transactions.
+  "validVersions": "0-4",

Review Comment:
   Yeah makes sense



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org