Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-16217

Transactional producer stuck in IllegalStateException during close

    XMLWordPrintableJSON

Details

    Description

      The producer is stuck during the close. It keeps retrying to abort the transaction but it never succeeds. 

      [ERROR] 2024-02-01 17:21:22,804 [kafka-producer-network-thread | producer-transaction-bench-transaction-id-f60SGdyRQGGFjdgg3vUgKg] org.apache.kafka.clients.producer.internals.Sender run - [Producer clientId=producer-transaction-ben
      ch-transaction-id-f60SGdyRQGGFjdgg3vUgKg, transactionalId=transaction-bench-transaction-id-f60SGdyRQGGFjdgg3vUgKg] Error in kafka producer I/O thread while aborting transaction:
      java.lang.IllegalStateException: Cannot attempt operation `abortTransaction` because the previous call to `commitTransaction` timed out and must be retried
              at org.apache.kafka.clients.producer.internals.TransactionManager.handleCachedTransactionRequestResult(TransactionManager.java:1138)
              at org.apache.kafka.clients.producer.internals.TransactionManager.beginAbort(TransactionManager.java:323)
              at org.apache.kafka.clients.producer.internals.Sender.run(Sender.java:274)
              at java.base/java.lang.Thread.run(Thread.java:1583)
              at org.apache.kafka.common.utils.KafkaThread.run(KafkaThread.java:66) 

      With the additional log, I found the root cause. If the producer is in a bad transaction state(in my case, the TransactionManager.pendingTransition was set to commitTransaction and did not get cleaned), then the producer calls close and tries to abort the existing transaction, the producer will get stuck in the transaction abortion. It is related to the fix https://github.com/apache/kafka/pull/13591.
       

      Attachments

        Issue Links

          Activity

            People

              calvinliu Calvin Liu
              calvinliu Calvin Liu
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: