You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Apurva Mehta (JIRA)" <ji...@apache.org> on 2017/06/09 06:43:18 UTC

[jira] [Updated] (KAFKA-5416) TransactionCoordinator seems to not return NOT_COORDINATOR error

     [ https://issues.apache.org/jira/browse/KAFKA-5416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Apurva Mehta updated KAFKA-5416:
--------------------------------
    Description: 
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. 



  was:
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:
# 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. 

Also, 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. 




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