CalvinConfluent commented on code in PR #18810:
URL: https://github.com/apache/kafka/pull/18810#discussion_r1945597015


##########
core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala:
##########
@@ -2252,6 +2252,75 @@ class ReplicaManagerTest {
     }
   }
 
+  @ParameterizedTest
+  @EnumSource(
+    value = classOf[Errors],
+    names = Array(
+      "NOT_COORDINATOR",
+      "CONCURRENT_TRANSACTIONS"
+    )
+  )
+  def testTransactionAddPartitionRetry(error: Errors): Unit = {
+    val tp0 = new TopicPartition(topic, 0)
+    val producerId = 24L
+    val producerEpoch = 0.toShort
+    val sequence = 6
+    val addPartitionsToTxnManager = mock(classOf[AddPartitionsToTxnManager])
+
+    val replicaManager = 
setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager,
 List(tp0))
+    try {
+      replicaManager.becomeLeaderOrFollower(1,
+        makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new 
LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
+        (_, _) => ())
+
+      // Append some transactional records.
+      val transactionalRecords = 
MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, 
producerEpoch, sequence,
+        new SimpleRecord("message".getBytes))
+
+      // We should add these partitions to the manager to verify.
+      val result = handleProduceAppend(replicaManager, tp0, 
transactionalRecords, origin = AppendOrigin.CLIENT,
+        transactionalId = transactionalId, transactionSupportedOperation = 
addPartition)
+      val appendCallback = 
ArgumentCaptor.forClass(classOf[AddPartitionsToTxnManager.AppendCallback])
+      verify(addPartitionsToTxnManager, times(1)).addOrVerifyTransaction(

Review Comment:
   For the NOT_COORDINATOR case, the addOrVerifyTransaction is only called 
once. 
   For the CONCURRENT_TRANSACTIONS, the first verify is consumed with the 
error. Later addOrVerifyTransaction is called another time which matches with 
the second verify.



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to