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

Apurva Mehta edited comment on KAFKA-5351 at 5/31/17 7:03 AM:
--------------------------------------------------------------

I saw the following in the logs for two of the brokers. At 06:12:05 the first 
broker returns COORDINATOR_NOT_AVAILABLE, at which point the producer sends a 
FindCoordinator request and gets routed back to the same broker (knode09) 
below. The producer then keeps retrying the EndTxn request, and keeps getting 
CONCURRENT_TRANSACTIONS error code back. 

At 06:12:17,  the producer gets a NOT_COORDINATOR erro. It then sends another 
'FindCoordinator' request, and gets routed to knode03, at which point, it keeps 
getting CONCURRENT_TRANSACTIONS until the process is terminated. 

{noformat}
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592/knode09#
 find . -name 'server.log' -exec grep -Hni 'my-second-transactional-id' {} \;
./info/server.log:956:[2017-05-31 06:12:05,724] INFO TransactionalId 
my-second-transactional-id prepare transition from Ongoing to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit, topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211125723) 
(kafka.coordinator.transaction.TransactionMetadata)
./info/server.log:963:[2017-05-31 06:12:05,736] INFO [Transaction Log Manager 
3]: Appending transaction message TxnTransitMetadata(producerId=1001, 
producerEpoch=0, txnTimeoutMs=60000, txnState=PrepareCommit, 
topicPartitions=Set(output-topic-2, __consumer_offsets-47, output-topic-0, 
output-topic-1), txnStartTimestamp=1496211125265, 
txnLastUpdateTimestamp=1496211125723) for my-second-transactional-id failed due 
to org.apache.kafka.common.errors.NotEnoughReplicasException,returning 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionStateManager)
./info/server.log:964:[2017-05-31 06:12:05,737] INFO [Transaction Coordinator 
3]: Updating my-second-transactional-id's transaction state to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit,topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211125723) with 
coordinator epoch 1 for my-second-transactional-id failed since the transaction 
message cannot be appended to the log. Returning error code 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionCoordinator)
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592/knode09#
 cd ..
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592#
 cd knode03/
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592/knode03#
 find . -name 'server.log' -exec grep -Hni 'my-second-transactional-id' {} \;
./info/server.log:1737:[2017-05-31 06:12:17,906] INFO TransactionalId 
my-second-transactional-id prepare transition from Ongoing to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit, topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211137906) 
(kafka.coordinator.transaction.TransactionMetadata)
./info/server.log:1741:[2017-05-31 06:12:17,926] INFO [Transaction Log Manager 
1]: Appending transaction message TxnTransitMetadata(producerId=1001, 
producerEpoch=0, txnTimeoutMs=60000, txnState=PrepareCommit, 
topicPartitions=Set(output-topic-2, __consumer_offsets-47, output-topic-0, 
output-topic-1), txnStartTimestamp=1496211125265, 
txnLastUpdateTimestamp=1496211137906) for my-second-transactional-id failed due 
to org.apache.kafka.common.errors.NotEnoughReplicasException, returning 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionStateManager)
./info/server.log:1742:[2017-05-31 06:12:17,933] INFO [Transaction Coordinator 
1]: Updating my-second-transactional-id's transaction state to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit, topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211137906) with 
coordinator epoch 2 for my-second-transactional-id failed since the transaction 
message cannot be appended to the log. Returning error code 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionCoordinator)
{noformat}

Relevant portions of the client log: 
{noformat}
[2017-05-31 06:12:05,595] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,595] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 2 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,597] TRACE Got transactional response for 
request:(transactionalId=my-second-transactional-id, producerId=1001, 
producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,597] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error NOT_COORDINATOR 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,597] 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 06:12:05,597] 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 06:12:05,598] 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 06:12:05,598] DEBUG TransactionalId my-second-transactional-id -- 
Received FindCoordinator response with error NONE 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,598] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,598] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,699] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 3 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,738] TRACE Got transactional response for 
request:(transactionalId=my-second-transactional-id, producerId=1001, 
producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,738] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error COORDINATOR_NOT_AVAILABLE 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,738] 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 06:12:05,738] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (type=FindCoordinatorRequest, 
coordinatorKey=my-second-transactional-id, coordinatorType=TRANSACTION) to node 
2 (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,740] 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 06:12:05,740] DEBUG TransactionalId my-second-transactional-id -- 
Received FindCoordinator response with error NONE 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,740] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,740] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,841] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 3 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,842] TRACE Got transactional response for 
request:(transactionalId=my-second-transactional-id, producerId=1001, 
producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,842] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error CONCURRENT_TRANSACTIONS 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,842] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,842] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,942] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 3 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,945] TRACE Got transactional response for 
request:(transactionalId=my-second-transactional-id, producerId=1001, 
producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,945] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error CONCURRENT_TRANSACTIONS 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:05,945] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:05,945] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
 {noformat}

and, for the second time period:

{noformat}
[2017-05-31 06:12:16,648] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error CONCURRENT_TRANSACTIONS 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:16,648] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:16,648] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:16,749] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 3 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:16,763] TRACE Got transactional response for 
request:(transactionalId=my-second-transactional-id, producerId=1001, 
producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:16,763] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error NOT_COORDINATOR 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:16,786] 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 06:12:16,796] 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 06:12:17,086] 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 06:12:17,109] DEBUG TransactionalId my-second-transactional-id -- 
Received FindCoordinator response with error NONE 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:17,120] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:17,695] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:17,795] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 1 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:17,934] TRACE Got transactional response for 
request:(transactionalId=my-second-transactional-id, producerId=1001, 
producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:17,934] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error COORDINATOR_NOT_AVAILABLE 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:17,934] 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 06:12:17,934] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (type=FindCoordinatorRequest, 
coordinatorKey=my-second-transactional-id, coordinatorType=TRANSACTION) to node 
1 (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:17,941] 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 06:12:17,941] DEBUG TransactionalId my-second-transactional-id -- 
Received FindCoordinator response with error NONE 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:17,941] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:17,941] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:18,041] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 1 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:18,043] TRACE Got transactional response for 
request:(transactionalId=my-second-transactional-id, producerId=1001, 
producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:18,043] DEBUG TransactionalId my-second-transactional-id -- 
Received EndTxn response with error CONCURRENT_TRANSACTIONS 
(org.apache.kafka.clients.producer.internals.TransactionManager)
[2017-05-31 06:12:18,043] DEBUG TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) 
(org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:18,043] TRACE TransactionalId: my-second-transactional-id -- 
Waiting 100ms before resending a transactional request 
(transactionalId=my-second-transactional-id, producerId=1001, producerEpoch=0, 
result=COMMIT) (org.apache.kafka.clients.producer.internals.Sender)
[2017-05-31 06:12:18,143] TRACE TransactionalId: my-second-transactional-id -- 
Sending transactional request (transactionalId=my-second-transactional-id, 
producerId=1001, producerEpoch=0, result=COMMIT) to node 1 
(org.apache.kafka.clients.producer.internals.Sender)
{noformat}


was (Author: apurva):
I saw the following in the logs for two of the brokers. At 06:12:05 the first 
broker returns COORDINATOR_NOT_AVAILABLE, at which point the producer sends a 
FindCoordinator request and gets routed back to the same broker (knode09) 
below. The producer then keeps retrying the EndTxn request, and keeps getting 
CONCURRENT_TRANSACTIONS error code back. 

At 06:12:17,  the producer gets a NOT_COORDINATOR erro. It then sends another 
'FindCoordinator' request, and gets routed to knode03, at which point, it keeps 
getting CONCURRENT_TRANSACTIONS until the process is terminated. 

{noformat}
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592/knode09#
 find . -name 'server.log' -exec grep -Hni 'my-second-transactional-id' {} \;
./info/server.log:956:[2017-05-31 06:12:05,724] INFO TransactionalId 
my-second-transactional-id prepare transition from Ongoing to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit, topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211125723) 
(kafka.coordinator.transaction.TransactionMetadata)
./info/server.log:963:[2017-05-31 06:12:05,736] INFO [Transaction Log Manager 
3]: Appending transaction message TxnTransitMetadata(producerId=1001, 
producerEpoch=0, txnTimeoutMs=60000, txnState=PrepareCommit, 
topicPartitions=Set(output-topic-2, __consumer_offsets-47, output-topic-0, 
output-topic-1), txnStartTimestamp=1496211125265, 
txnLastUpdateTimestamp=1496211125723) for my-second-transactional-id failed due 
to org.apache.kafka.common.errors.NotEnoughReplicasException,returning 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionStateManager)
./info/server.log:964:[2017-05-31 06:12:05,737] INFO [Transaction Coordinator 
3]: Updating my-second-transactional-id's transaction state to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit,topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211125723) with 
coordinator epoch 1 for my-second-transactional-id failed since the transaction 
message cannot be appended to the log. Returning error code 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionCoordinator)
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592/knode09#
 cd ..
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592#
 cd knode03/
root@f82eb723774d:/opt/kafka-dev/results/latest/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/1/KafkaService-0-140388221024592/knode03#
 find . -name 'server.log' -exec grep -Hni 'my-second-transactional-id' {} \;
./info/server.log:1737:[2017-05-31 06:12:17,906] INFO TransactionalId 
my-second-transactional-id prepare transition from Ongoing to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit, topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211137906) 
(kafka.coordinator.transaction.TransactionMetadata)
./info/server.log:1741:[2017-05-31 06:12:17,926] INFO [Transaction Log Manager 
1]: Appending transaction message TxnTransitMetadata(producerId=1001, 
producerEpoch=0, txnTimeoutMs=60000, txnState=PrepareCommit, 
topicPartitions=Set(output-topic-2, __consumer_offsets-47, output-topic-0, 
output-topic-1), txnStartTimestamp=1496211125265, 
txnLastUpdateTimestamp=1496211137906) for my-second-transactional-id failed due 
to org.apache.kafka.common.errors.NotEnoughReplicasException, returning 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionStateManager)
./info/server.log:1742:[2017-05-31 06:12:17,933] INFO [Transaction Coordinator 
1]: Updating my-second-transactional-id's transaction state to 
TxnTransitMetadata(producerId=1001, producerEpoch=0, txnTimeoutMs=60000, 
txnState=PrepareCommit, topicPartitions=Set(output-topic-2, 
__consumer_offsets-47, output-topic-0, output-topic-1), 
txnStartTimestamp=1496211125265, txnLastUpdateTimestamp=1496211137906) with 
coordinator epoch 2 for my-second-transactional-id failed since the transaction 
message cannot be appended to the log. Returning error code 
COORDINATOR_NOT_AVAILABLE to the client 
(kafka.coordinator.transaction.TransactionCoordinator)
{noformat}

> 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
>
>
> 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)

Reply via email to