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

Apurva Mehta commented on KAFKA-5416:
-------------------------------------

cc [~guozhang] [~damianguy] 

A second pair of eyes making sense of these logs would help a lot.

> TransactionCoordinator seems to not return NOT_COORDINATOR error
> ----------------------------------------------------------------
>
>                 Key: KAFKA-5416
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5416
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Apurva Mehta
>
> In regard to this system test: 
> http://confluent-kafka-branch-builder-system-test-results.s3-us-west-2.amazonaws.com/2017-06-09--001.1496974430--apurvam--MINOR-add-logging-to-transaction-coordinator-in-all-failure-cases--02b3816/TransactionsTest/test_transactions/failure_mode=clean_bounce.bounce_target=brokers/4.tgz
> There are two issues:
> First, a coordinator who is not the owner for a given partition of the 
> transaction log does not return NOT_COORDINATOR, but CONCURRENT_TRANSACTIONS 
> instead.
> Here are the ownership changes for __transaction_state-41: 
> {noformat}
> ./worker1/debug/server.log:3559:[2017-06-09 01:16:36,244] INFO [Transaction 
> Log Manager 2]: Loading transaction metadata from __transaction_state-41 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:38471:[2017-06-09 01:16:45,910] INFO [Transaction 
> Log Manager 2]: Removed 1 cached transaction metadata for 
> __transaction_state-41 on follower transition 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:40226:[2017-06-09 01:16:51,821] INFO [Transaction 
> Log Manager 2]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:41233:[2017-06-09 01:16:53,332] INFO [Transaction 
> Log Manager 2]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:42486:[2017-06-09 01:16:59,584] INFO [Transaction 
> Log Manager 2]: Loading transaction metadata from __transaction_state-41 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:42515:[2017-06-09 01:16:59,611] INFO [Transaction 
> Log Manager 2]: Finished loading 1 transaction metadata from 
> __transaction_state-41 in 27 milliseconds 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:153029:[2017-06-09 01:19:11,484] INFO [Transaction 
> Log Manager 2]: Removed 1 cached transaction metadata for 
> __transaction_state-41 on follower transition 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:3537:[2017-06-09 01:16:36,441] INFO [Transaction 
> Log Manager 1]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:25957:[2017-06-09 01:16:46,309] INFO [Transaction 
> Log Manager 1]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:26618:[2017-06-09 01:16:48,164] INFO [Transaction 
> Log Manager 1]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:27951:[2017-06-09 01:16:51,398] INFO [Transaction 
> Log Manager 1]: Loading transaction metadata from __transaction_state-41 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:27958:[2017-06-09 01:16:51,407] INFO [Transaction 
> Log Manager 1]: Finished loading 1 transaction metadata from 
> __transaction_state-41 in 9 milliseconds 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:80970:[2017-06-09 01:16:59,608] INFO [Transaction 
> Log Manager 1]: Removed 1 cached transaction metadata for 
> __transaction_state-41 on follower transition 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:2626:[2017-06-09 01:16:36,882] INFO [Transaction 
> Log Manager 3]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:14291:[2017-06-09 01:16:45,909] INFO [Transaction 
> Log Manager 3]: Loading transaction metadata from __transaction_state-41 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:14292:[2017-06-09 01:16:45,916] INFO [Transaction 
> Log Manager 3]: Finished loading 1 transaction metadata from 
> __transaction_state-41 in 7 milliseconds 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:52160:[2017-06-09 01:16:51,305] INFO [Transaction 
> Log Manager 3]: Removed 1 cached transaction metadata for 
> __transaction_state-41 on follower transition 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:54100:[2017-06-09 01:16:58,269] INFO [Transaction 
> Log Manager 3]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:54741:[2017-06-09 01:16:59,607] INFO [Transaction 
> Log Manager 3]: Trying to remove cached transaction metadata for 
> __transaction_state-41 on follower transition but there is no entries 
> remaining; it is likely that another process for removing the cached entries 
> has just executed earlier before 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:123782:[2017-06-09 01:19:11,499] INFO [Transaction 
> Log Manager 3]: Loading transaction metadata from __transaction_state-41 
> (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:123803:[2017-06-09 01:19:11,522] INFO [Transaction 
> Log Manager 3]: Finished loading 1 transaction metadata from 
> __transaction_state-41 in 23 milliseconds 
> (kafka.coordinator.transaction.TransactionStateManager)
> amehta-macbook-pro:KafkaService-0-140622383877776 apurva$
> {noformat}
> At 01:16:59, worker2 loses ownership of this partition.
> However, on the client log, the 'AddPartitions' request constantly gets a 
> CONCURRENT_TRANSACTIONS response from worker2 for 2 more minutes afterward. 
> This is amiss. 
> Second, the tail of the log for worker2 for the transactional id in question 
> 'my-first-transactional-id' is: 
> {noformat}
> [2017-06-09 01:16:50,782] DEBUG TransactionalId my-second-transactional-id 
> prepare transition from PrepareCommit to TxnTransitMetadata(producerId=1, 
> producerEpoch=0, txnTimeoutMs=60000, txnState=CompleteCommit, 
> topicPartitions=Set(), txnStartTimestamp=1496971010483, 
> txnLastUpdateTimestamp=1496971010776) 
> (kafka.coordinator.transaction.TransactionMetadata)
> [2017-06-09 01:16:50,844] DEBUG Updating my-second-transactional-id's 
> transaction state to 
> TransactionMetadata(transactionalId=my-second-transactional-id, producerId=1, 
> producerEpoch=0, txnTimeoutMs=60000, state=PrepareCommit, 
> pendingState=Some(CompleteCommit), topicPartitions=Set(), 
> txnStartTimestamp=1496971010483, txnLastUpdateTimestamp=1496971010664) with 
> coordinator epoch 3 for my-second-transactional-id succeeded 
> (kafka.coordinator.transaction.TransactionMarkerChannelManager)
> [2017-06-09 01:16:50,866] DEBUG [Transaction Log Manager 1]: Transaction 
> state update TxnTransitMetadata(producerId=1, producerEpoch=0, 
> txnTimeoutMs=60000, txnState=CompleteCommit, topicPartitions=Set(), 
> txnStartTimestamp=1496971010483, txnLastUpdateTimestamp=1496971010776) for 
> my-second-transactional-id failed when appending to log due to 
> org.apache.kafka.common.errors.NotEnoughReplicasException 
> (kafka.coordinator.transaction.TransactionStateManager)
> [2017-06-09 01:16:50,866] INFO [Transaction Log Manager 1]: Appending 
> transaction message TxnTransitMetadata(producerId=1, producerEpoch=0, 
> txnTimeoutMs=60000, txnState=CompleteCommit, topicPartitions=Set(), 
> txnStartTimestamp=1496971010483, txnLastUpdateTimestamp=1496971010776) 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)
> [2017-06-09 01:16:50,867] DEBUG [Transaction Log Manager 1]: TransactionalId 
> my-second-transactional-id, resetting pending state since we are returning 
> error COORDINATOR_NOT_AVAILABLE 
> (kafka.coordinator.transaction.TransactionStateManager)
> [2017-06-09 01:16:50,868] WARN Failed updating transaction state for 
> my-second-transactional-id when appending to transaction log due to 
> org.apache.kafka.common.errors.CoordinatorNotAvailableException. retrying 
> (kafka.coordinator.transaction.TransactionMarkerChannelManager)
> [2017-06-09 01:16:50,870] DEBUG [Transaction Log Manager 1]: Transaction 
> state update TxnTransitMetadata(producerId=1, producerEpoch=0, 
> txnTimeoutMs=60000, txnState=CompleteCommit, topicPartitions=Set(), 
> txnStartTimestamp=1496971010483, txnLastUpdateTimestamp=1496971010776) for 
> my-second-transactional-id failed when appending to log due to 
> org.apache.kafka.common.errors.NotEnoughReplicasException 
> (kafka.coordinator.transaction.TransactionStateManager)
> [2017-06-09 01:16:50,870] INFO [Transaction Log Manager 1]: Appending 
> transaction message TxnTransitMetadata(producerId=1, producerEpoch=0, 
> txnTimeoutMs=60000, txnState=CompleteCommit, topicPartitions=Set(), 
> txnStartTimestamp=1496971010483, txnLastUpdateTimestamp=1496971010776) 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)
> [2017-06-09 01:16:50,871] DEBUG [Transaction Log Manager 1]: TransactionalId 
> my-second-transactional-id, resetting pending state since we are returning 
> error COORDINATOR_NOT_AVAILABLE 
> (kafka.coordinator.transaction.TransactionStateManager)
> [2017-06-09 01:16:50,871] WARN Failed updating transaction state for 
> my-second-transactional-id when appending to transaction log due to 
> org.apache.kafka.common.errors.CoordinatorNotAvailableException. retrying 
> (kafka.coordinator.transaction.TransactionMarkerChannelManager)
> [2017-06-09 01:16:51,495] DEBUG [Transaction Log Manager 1]: Transaction 
> state update TxnTransitMetadata(producerId=1, producerEpoch=0, 
> txnTimeoutMs=60000, txnState=CompleteCommit, topicPartitions=Set(), 
> txnStartTimestamp=1496971010483, txnLastUpdateTimestamp=1496971010776) for 
> my-second-transactional-id failed when appending to log due to 
> org.apache.kafka.common.errors.NotEnoughReplicasException 
> (kafka.coordinator.transaction.TransactionStateManager)
> [2017-06-09 01:16:51,496] INFO [Transaction Log Manager 1]: Appending 
> transaction message TxnTransitMetadata(producerId=1, producerEpoch=0, 
> txnTimeoutMs=60000, txnState=CompleteCommit, topicPartitions=Set(), 
> txnStartTimestamp=1496971010483, txnLastUpdateTimestamp=1496971010776) 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)
> [2017-06-09 01:16:51,496] DEBUG [Transaction Log Manager 1]: TransactionalId 
> my-second-transactional-id, resetting pending state since we are returning 
> error COORDINATOR_NOT_AVAILABLE 
> (kafka.coordinator.transaction.TransactionStateManager)
> [2017-06-09 01:16:51,496] WARN Failed updating transaction state for 
> my-second-transactional-id when appending to transaction log due to 
> org.apache.kafka.common.errors.CoordinatorNotAvailableException. retrying 
> (kafka.coordinator.transaction.TransactionMarkerChannelManager)
> {noformat}
> It seems that the `TransactionManagerChannelManager` fails to retry the write 
> of `CompleteCommit` when it fails initially. the `retryLogAppends` method has 
> a debug log when messages are being retried, but that is absent from this 
> log. Could it be that the InterbrokerSendThread is in an inifinte poll? that 
> certainly looks possible from the code. 



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

Reply via email to