You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "hachikuji (via GitHub)" <gi...@apache.org> on 2023/05/03 01:25:40 UTC

[GitHub] [kafka] hachikuji commented on a diff in pull request #13607: KAFKA-14916: Fix code that assumes transactional ID implies all records are transactional

hachikuji commented on code in PR #13607:
URL: https://github.com/apache/kafka/pull/13607#discussion_r1183173661


##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -744,6 +751,8 @@ class ReplicaManager(val config: KafkaConfig,
         }
 
         // map not yet verified partitions to a request object
+        // Since verification occurs on produce requests only, and each produce request has one batch per partition, we know the producer ID is transactional

Review Comment:
   But do we know that the producerId is consistent among all transactional batches? Seems like we are assuming it below, but where is it verified?



##########
core/src/main/scala/kafka/server/ReplicaManager.scala:
##########
@@ -642,7 +642,14 @@ class ReplicaManager(val config: KafkaConfig,
           (entriesPerPartition, Map.empty)
         else
           entriesPerPartition.partition { case (topicPartition, records) =>
-            getPartitionOrException(topicPartition).hasOngoingTransaction(records.firstBatch().producerId())
+            // Produce requests (only requests that require verification) should only have one batch per partition in "batches" but check all just to be safe.
+            val transactionalBatches = records.batches.asScala.filter(batch => batch.hasProducerId && batch.isTransactional)
+            if (!transactionalBatches.isEmpty) {

Review Comment:
   nit: `nonEmpty`?



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