[ https://issues.apache.org/jira/browse/KAFKA-5427?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Apurva Mehta updated KAFKA-5427: -------------------------------- Labels: exactly-once (was: ) > Transactional producer cannot find coordinator when trying to abort > transaction after error > ------------------------------------------------------------------------------------------- > > Key: KAFKA-5427 > URL: https://issues.apache.org/jira/browse/KAFKA-5427 > Project: Kafka > Issue Type: Sub-task > Components: clients, core, producer > Reporter: Jason Gustafson > Assignee: Jason Gustafson > Priority: Blocker > Labels: exactly-once > Fix For: 0.11.0.0 > > > It can happen that we receive an abortable error while we are already > aborting a transaction. In this case, we have an EndTxnRequest queued for > sending when we transition to ABORTABLE_ERROR. It could be that we need to > find the coordinator before sending this EndTxnRequest. The problem is that > we will fail even the FindCoordinatorRequest because we are in an error > state. This causes the following endless loop: > {code} > [2017-06-10 19:29:33,436] DEBUG [TransactionalId my-first-transactional-id] > Enqueuing transactional request (type=FindCoordinatorRequest, > coordinatorKey=my-fi > rst-transactional-id, coordinatorType=TRANSACTION) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,436] DEBUG [TransactionalId my-first-transactional-id] > Enqueuing transactional request (type=EndTxnRequest, > transactionalId=my-first-tran > sactional-id, producerId=1000, producerEpoch=0, result=ABORT) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,536] TRACE [TransactionalId my-first-transactional-id] > Not sending transactional request (type=FindCoordinatorRequest, > coordinatorKey=my- > first-transactional-id, coordinatorType=TRANSACTION) because we are in an > error state (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,637] TRACE [TransactionalId my-first-transactional-id] > Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, > producerId > =1000, producerEpoch=0, result=ABORT) dequeued for sending > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,637] DEBUG [TransactionalId my-first-transactional-id] > Enqueuing transactional request (type=FindCoordinatorRequest, > coordinatorKey=my-fi > rst-transactional-id, coordinatorType=TRANSACTION) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,637] DEBUG [TransactionalId my-first-transactional-id] > Enqueuing transactional request (type=EndTxnRequest, > transactionalId=my-first-tran > sactional-id, producerId=1000, producerEpoch=0, result=ABORT) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,737] TRACE [TransactionalId my-first-transactional-id] > Not sending transactional request (type=FindCoordinatorRequest, > coordinatorKey=my- > first-transactional-id, coordinatorType=TRANSACTION) because we are in an > error state (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,837] TRACE [TransactionalId my-first-transactional-id] > Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, > producerId > =1000, producerEpoch=0, result=ABORT) dequeued for sending > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,838] DEBUG [TransactionalId my-first-transactional-id] > Enqueuing transactional request (type=FindCoordinatorRequest, > coordinatorKey=my-fi > rst-transactional-id, coordinatorType=TRANSACTION) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,838] DEBUG [TransactionalId my-first-transactional-id] > Enqueuing transactional request (type=EndTxnRequest, > transactionalId=my-first-tran > sactional-id, producerId=1000, producerEpoch=0, result=ABORT) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-06-10 19:29:33,938] TRACE [TransactionalId my-first-transactional-id] > Not sending transactional request (type=FindCoordinatorRequest, > coordinatorKey=my- > first-transactional-id, coordinatorType=TRANSACTION) because we are in an > error state (org.apache.kafka.clients.producer.internals.TransactionManager) > {code} > A couple suggested improvements: > 1. We should allow FindCoordinator requests regardless of the transaction > state. > 2. It is a bit confusing that we allow EndTxnRequest to be sent in both the > ABORTABLE_ERROR and the ABORTING_TRANSACTION states. Perhaps we should only > allow EndTxnRequest to be sent in ABORTING_TRANSACTION. If we hit an > abortable error and we are already aborting, then we should just stay in > ABORTING_TRANSACTION and perhaps log a warning. -- This message was sent by Atlassian JIRA (v6.4.14#64029)