[ 
https://issues.apache.org/jira/browse/KAFKA-5415?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16043741#comment-16043741
 ] 

Apurva Mehta commented on KAFKA-5415:
-------------------------------------

Incriminating evidence from the logs.

1. Tail of the transaction log is PrepareCommit at offset 277
{noformat}
offset: 276 position: 46886 CreateTime: 1496957141444 isvalid: true keysize: 29 
valuesize: 95 magic: 2 compresscodec: NONE producerId: -1 sequence: -1 
isTransactional: false headerKeys: [] key: 
transactionalId=my-first-transactional-id payload: 
producerId:2000,producerEpoch:0,state=Ongoing,partitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, 
output-topic-1),lastUpdateTimestamp=1496957141444
offset: 277 position: 47080 CreateTime: 1496957141285 isvalid: true keysize: 29 
valuesize: 95 magic: 2 compresscodec: NONE producerId: -1 sequence: -1 
isTransactional: false headerKeys: [] key: 
transactionalId=my-first-transactional-id payload: 
producerId:2000,producerEpoch:0,state=PrepareCommit,partitions=Set(output-topic-2,
 __consumer_offsets-47, output-topic-0, 
output-topic-1),lastUpdateTimestamp=1496957141285
{noformat}

2. The client disconnected from the coordinator after sending the EndTxn request
{noformat}
[2017-06-08 21:25:41,474] TRACE Produced messages to topic-partition 
output-topic-2 with base offset offset 28618 and error: null. 
(org.apache.kafka.clients.producer.internals.ProducerBatch)
[2017-06-08 21:25:41,474] TRACE [TransactionalId my-first-transactional-id] 
Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, 
producerId=2000, producerEpoch=0, result=COMMIT) dequeued for sending 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:25:41,474] DEBUG [TransactionalId my-first-transactional-id] 
Sending transactional request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) to node worker12:9092 (id: 3 rack: null) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-06-08 21:26:11,533] DEBUG [TransactionalId my-first-transactional-id] 
Disconnected from 3. Will retry. 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,533] DEBUG [TransactionalId my-first-transactional-id] 
Enqueuing transactional request (type=FindCoordinatorRequest, 
coordinatorKey=my-first-transactional-id, coordinatorType=TRANSACTION) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,533] DEBUG [TransactionalId my-first-transactional-id] 
Enqueuing transactional request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,533] TRACE [TransactionalId my-first-transactional-id] 
Request (type=FindCoordinatorRequest, coordinatorKey=my-first-transactional-id, 
coordinatorType=TRANSACTION) dequeued for sending 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,534] DEBUG [TransactionalId my-first-transactional-id] 
Sending transactional request (type=FindCoordinatorRequest, 
coordinatorKey=my-first-transactional-id, coordinatorType=TRANSACTION) to node 
worker5:9092 (id: 2 rack: null) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-06-08 21:26:11,535] TRACE [TransactionalId my-first-transactional-id] 
Received transactional response FindCoordinatorResponse(throttleTimeMs=0, 
errorMessage='null', error=NONE, node=worker12:9092 (id: 3 rack: null)) for 
request (type=FindCoordinatorRequest, coordinatorKey=my-first-transactional-id, 
coordinatorType=TRANSACTION) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,535] TRACE [TransactionalId my-first-transactional-id] 
Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, 
producerId=2000, producerEpoch=0, result=COMMIT) dequeued for sending 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,535] DEBUG [TransactionalId my-first-transactional-id] 
Disconnect from worker12:9092 (id: 3 rack: null) while trying to send request 
(type=EndTxnRequest, transactionalId=my-first-transactional-id, 
producerId=2000, producerEpoch=0, result=COMMIT). Going to back off and retry 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-06-08 21:26:11,535] DEBUG [TransactionalId my-first-transactional-id] 
Enqueuing transactional request (type=FindCoordinatorRequest, 
coordinatorKey=my-first-transactional-id, coordinatorType=TRANSACTION) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,535] DEBUG [TransactionalId my-first-transactional-id] 
Enqueuing transactional request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,636] TRACE [TransactionalId my-first-transactional-id] 
Request (type=FindCoordinatorRequest, coordinatorKey=my-first-transactional-id, 
coordinatorType=TRANSACTION) dequeued for sending 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,636] DEBUG [TransactionalId my-first-transactional-id] 
Sending transactional request (type=FindCoordinatorRequest, 
coordinatorKey=my-first-transactional-id, coordinatorType=TRANSACTION) to node 
worker1:9092 (id: 1 rack: null) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-06-08 21:26:11,637] TRACE [TransactionalId my-first-transactional-id] 
Received transactional response FindCoordinatorResponse(throttleTimeMs=0, 
errorMessage='null', error=NONE, node=worker12:9092 (id: 3 rack: null)) for 
request (type=FindCoordinatorRequest, coordinatorKey=my-first-transactional-id, 
coordinatorType=TRANSACTION) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,637] TRACE [TransactionalId my-first-transactional-id] 
Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, 
producerId=2000, producerEpoch=0, result=COMMIT) dequeued for sending 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,739] DEBUG [TransactionalId my-first-transactional-id] 
Sending transactional request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) to node worker12:9092 (id: 3 rack: null) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-06-08 21:26:11,739] TRACE [TransactionalId my-first-transactional-id] 
Received transactional response EndTxnResponse(error=CONCURRENT_TRANSACTIONS, 
throttleTimeMs=0) for request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,739] DEBUG [TransactionalId my-first-transactional-id] 
Enqueuing transactional request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,739] TRACE [TransactionalId my-first-transactional-id] 
Request (type=EndTxnRequest, transactionalId=my-first-transactional-id, 
producerId=2000, producerEpoch=0, result=COMMIT) dequeued for sending 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,840] DEBUG [TransactionalId my-first-transactional-id] 
Sending transactional request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) to node worker12:9092 (id: 3 rack: null) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-06-08 21:26:11,840] TRACE [TransactionalId my-first-transactional-id] 
Received transactional response EndTxnResponse(error=CONCURRENT_TRANSACTIONS, 
throttleTimeMs=0) for request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-06-08 21:26:11,840] DEBUG [TransactionalId my-first-transactional-id] 
Enqueuing transactional request (type=EndTxnRequest, 
transactionalId=my-first-transactional-id, producerId=2000, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.TransactionManager)
{noformat}

3. Last message in the coordinator for the transactional id in question 
indicates a transition to PrepareCommit
{noformat}
[2017-06-08 21:25:41,449] DEBUG [Transaction Log Manager 3]: Updating 
my-first-transactional-id's transaction state to 
TxnTransitMetadata(producerId=2000, producerEpoch=0, txnTimeoutMs=60000, 
txnState=Ongoing, topicPartitions=Set(output-topic-2, __consumer_offsets-47, 
output-topic-0, output-topic-1), txnStartTimestamp=1496957141430, 
txnLastUpdateTimestamp=1496957141444) with coordinator epoch 3 for 
my-first-transactional-id succeeded 
(kafka.coordinator.transaction.TransactionStateManager)
[2017-06-08 21:25:41,285] DEBUG TransactionalId my-first-transactional-id 
prepare transition from Ongoing to TxnTransitMetadata(producerId=2000, 
producerEpoch=0, txnTimeoutMs=60000, txnState=PrepareCommit, 
topicPartitions=Set(output-topic-2, __consumer_offsets-47, output-topic-0, 
output-topic-1), txnStartTimestamp=1496957141430, 
txnLastUpdateTimestamp=1496957141285) 
(kafka.coordinator.transaction.TransactionMetadata)
{noformat}

4. The high water mark for the transaction log partition was updated to 278, 
indicating that the append callback must have been invoked.
{noformat}
[2017-06-08 21:25:41,449] DEBUG [Transaction Log Manager 3]: Updating 
my-first-transactional-id's transaction state to 
TxnTransitMetadata(producerId=2000, producerEpoch=0, txnTimeoutMs=60000, 
txnState=Ongoing, topicPartitions=Set(output-topic-2, __consumer_offsets-47, 
output-topic-0, output-topic-1), txnStartTimestamp=1496957141430, 
txnLastUpdateTimestamp=1496957141444) with coordinator epoch 3 for 
my-first-transactional-id succeeded 
(kafka.coordinator.transaction.TransactionStateManager)
[2017-06-08 21:25:41,285] DEBUG TransactionalId my-first-transactional-id 
prepare transition from Ongoing to TxnTransitMetadata(producerId=2000, 
producerEpoch=0, txnTimeoutMs=60000, txnState=PrepareCommit, 
topicPartitions=Set(output-topic-2, __consumer_offsets-47, output-topic-0, 
output-topic-1), txnStartTimestamp=1496957141430, 
txnLastUpdateTimestamp=1496957141285) 
(kafka.coordinator.transaction.TransactionMetadata)
{nodformat}


> TransactionCoordinator gets stuck in PrepareCommits state.
> ----------------------------------------------------------
>
>                 Key: KAFKA-5415
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5415
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Apurva Mehta
>            Assignee: Apurva Mehta
>            Priority: Blocker
>              Labels: exactly-once
>             Fix For: 0.11.0.0
>
>         Attachments: 6.tgz
>
>
> This has been revealed by the system test failures on jenkins. 
> The transaction coordinator seems to get into a path during the handling of 
> the EndTxnRequest where it returns an error (possibly a NOT_COORDINATOR or 
> COORDINATOR_NOT_AVAILABLE error, to be revealed by 
> https://github.com/apache/kafka/pull/3278) to the client. However, due to 
> network instability, the producer is disconnected before it receives this 
> error.
> As a result, the transaction remains in a `PrepareXX` state, and future 
> `EndTxn` requests sent by the client after reconnecting result in a 
> `CONCURRENT_TRANSACTION` error code. Hence the client gets stuck and the 
> transaction never finishes, as expiration isn't done from a PrepareXX state.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to