[ 
https://issues.apache.org/jira/browse/KAFKA-16217?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Calvin Liu updated KAFKA-16217:
-------------------------------
    Description: 
The producer is stuck during the close. It keeps retrying to abort the 
transaction but it never succeeds. 
{code:java}
[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) 
{code}
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].
 

  was:
The producer is stuck during the close. It keeps retrying to abort the 
transaction but it never succeeds. 
{code:java}
[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) 
{code}
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), before 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].
 


> Transactional producer stuck in IllegalStateException during close
> ------------------------------------------------------------------
>
>                 Key: KAFKA-16217
>                 URL: https://issues.apache.org/jira/browse/KAFKA-16217
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>            Reporter: Calvin Liu
>            Priority: Major
>
> The producer is stuck during the close. It keeps retrying to abort the 
> transaction but it never succeeds. 
> {code:java}
> [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) 
> {code}
> 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].
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to