[jira] [Comment Edited] (KAFKA-5415) TransactionCoordinator gets stuck in PrepareCommits state.

2017-06-08 Thread Apurva Mehta (JIRA)

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

Apurva Mehta edited comment on KAFKA-5415 at 6/9/17 12:50 AM:
--

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, and got 'CONCURRENT_TRANSACTIONS' on retry.
{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-tra

[jira] [Comment Edited] (KAFKA-5415) TransactionCoordinator gets stuck in PrepareCommits state.

2017-06-08 Thread Apurva Mehta (JIRA)

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

Apurva Mehta edited comment on KAFKA-5415 at 6/9/17 12:51 AM:
--

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, and got 'CONCURRENT_TRANSACTIONS' on retry.
{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-tra