[ https://issues.apache.org/jira/browse/KAFKA-5351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16031807#comment-16031807 ]
Apurva Mehta commented on KAFKA-5351: ------------------------------------- I found the bug. The problem is that the coordinator retains its pending state even though it sends a retriable error to the client. So when the client retries, the coordinator sees a pending state defined, and returns `CONCURRENT_TRANSACTIONS`, which is a state it can never get out of until it is bounced. The solution is to clear the pending state when coordinator returns a retriable error, because in reality there is nothing pending at that point. > Broker clean bounce test puts the broker into a 'CONCURRENT_TRANSACTIONS' > state permanently > ------------------------------------------------------------------------------------------- > > Key: KAFKA-5351 > URL: https://issues.apache.org/jira/browse/KAFKA-5351 > Project: Kafka > Issue Type: Sub-task > Components: clients, core, producer > Reporter: Apurva Mehta > Assignee: Apurva Mehta > Priority: Blocker > Labels: exactly-once > Fix For: 0.11.0.0 > > Attachments: kafka-5351.logs.tar.gz > > > In the broker clean bounce test, sometimes the consumer just hangs on a > request to the transactional coordinator because it keeps getting a > `CONCURRENT_TRANSACTIONS` error. This continues for 30 seconds, until the > process is killed. > {noformat} > [2017-05-31 04:54:14,053] DEBUG TransactionalId my-second-transactional-id -- > Received FindCoordinator response with error NONE > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,053] DEBUG TransactionalId: my-second-transactional-id > -- Sending transactional request (transactionalId=my-second-transactional-id, > producerId=2000, producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,053] TRACE TransactionalId: my-second-transactional-id > -- Waiting 100ms before resending a transactional request > (transactionalId=my-second-transactional-id, producerId=2000, > producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,154] TRACE TransactionalId: my-second-transactional-id > -- Sending transactional request (transactionalId=my-second-transactional-id, > producerId=2000, producerEpoch=0, result=COMMIT) to node 1 > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,191] TRACE Got transactional response for > request:(transactionalId=my-second-transactional-id, producerId=2000, > producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,191] DEBUG TransactionalId my-second-transactional-id -- > Received EndTxn response with error COORDINATOR_NOT_AVAILABLE > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,192] DEBUG TransactionalId: my-second-transactional-id > -- Sending transactional request (type=FindCoordinatorRequest, > coordinatorKey=my-second-transactional-id, coordinatorType=TRANSACTION) > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,192] TRACE TransactionalId: my-second-transactional-id > -- Sending transactional request (type=FindCoordinatorRequest, > coordinatorKey=my-second-transactional-id, coordinatorType=TRANSACTION) to > node 3 (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,193] TRACE Got transactional response for > request:(type=FindCoordinatorRequest, > coordinatorKey=my-second-transactional-id, coordinatorType=TRANSACTION) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,193] DEBUG TransactionalId my-second-transactional-id -- > Received FindCoordinator response with error NONE > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,193] DEBUG TransactionalId: my-second-transactional-id > -- Sending transactional request (transactionalId=my-second-transactional-id, > producerId=2000, producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,193] TRACE TransactionalId: my-second-transactional-id > -- Waiting 100ms before resending a transactional request > (transactionalId=my-second-transactional-id, producerId=2000, > producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,294] TRACE TransactionalId: my-second-transactional-id > -- Sending transactional request (transactionalId=my-second-transactional-id, > producerId=2000, producerEpoch=0, result=COMMIT) to node 1 > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,294] TRACE Got transactional response for > request:(transactionalId=my-second-transactional-id, producerId=2000, > producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,295] DEBUG TransactionalId my-second-transactional-id -- > Received EndTxn response with error CONCURRENT_TRANSACTIONS > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,295] DEBUG TransactionalId: my-second-transactional-id > -- Sending transactional request (transactionalId=my-second-transactional-id, > producerId=2000, producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,295] TRACE TransactionalId: my-second-transactional-id > -- Waiting 100ms before resending a transactional request > (transactionalId=my-second-transactional-id, producerId=2000, > producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,395] TRACE TransactionalId: my-second-transactional-id > -- Sending transactional request (transactionalId=my-second-transactional-id, > producerId=2000, producerEpoch=0, result=COMMIT) to node 1 > (org.apache.kafka.clients.producer.internals.Sender) > [2017-05-31 04:54:14,398] TRACE Got transactional response for > request:(transactionalId=my-second-transactional-id, producerId=2000, > producerEpoch=0, result=COMMIT) > (org.apache.kafka.clients.producer.internals.TransactionManager) > [2017-05-31 04:54:14,398] DEBUG TransactionalId my-second-transactional-id -- > Received EndTxn response with error CONCURRENT_TRANSACTIONS > (org.apache.kafka.clients.producer.internals.TransactionManager) > {noformat} -- This message was sent by Atlassian JIRA (v6.3.15#6346)