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 07:02:18 UTC

[jira] [Updated] (KAFKA-5416) TransactionCoordinator: TransactionMarkerChannelManager seems not to retry failed writes.

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

Apurva Mehta updated KAFKA-5416:
--------------------------------
    Summary: TransactionCoordinator: TransactionMarkerChannelManager seems not to retry failed writes.  (was: TransactionCoordinator seems to not return NOT_COORDINATOR error)

> TransactionCoordinator: TransactionMarkerChannelManager seems not to retry failed writes.
> -----------------------------------------------------------------------------------------
>
>                 Key: KAFKA-5416
>                 URL: https://issues.apache.org/jira/browse/KAFKA-5416
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Apurva Mehta
>            Priority: Blocker
>              Labels: exactly-once
>             Fix For: 0.11.0.0
>
>
> 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
> Here are the ownership changes for __transaction_state-37: 
> {noformat}
> ./worker1/debug/server.log:3623:[2017-06-09 01:16:36,551] INFO [Transaction Log Manager 2]: Trying to remove cached transaction metadata for __transaction_state-37 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:16174:[2017-06-09 01:16:39,963] INFO [Transaction Log Manager 2]: Loading transaction metadata from __transaction_state-37 (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:16194:[2017-06-09 01:16:39,978] INFO [Transaction Log Manager 2]: Finished loading 1 transaction metadata from __transaction_state-37 in 15 milliseconds (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:35633:[2017-06-09 01:16:45,308] INFO [Transaction Log Manager 2]: Removed 1 cached transaction metadata for __transaction_state-37 on follower transition (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:40353:[2017-06-09 01:16:52,218] INFO [Transaction Log Manager 2]: Trying to remove cached transaction metadata for __transaction_state-37 on follower transition but there is no entries remaining; it is likelythat another process for removing the cached entries has just executed earlier before (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:40664:[2017-06-09 01:16:52,683] INFO [Transaction Log Manager 2]: Trying to remove cached transaction metadata for __transaction_state-37 on follower transition but there is no entries remaining; it is likelythat another process for removing the cached entries has just executed earlier before (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:146044:[2017-06-09 01:19:08,844] INFO [Transaction Log Manager 2]: Loading transaction metadata from __transaction_state-37 (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:146048:[2017-06-09 01:19:08,850] INFO [Transaction Log Manager 2]: Finished loading 1 transaction metadata from __transaction_state-37 in 6 milliseconds (kafka.coordinator.transaction.TransactionStateManager)
> ./worker1/debug/server.log:150147:[2017-06-09 01:19:10,995] INFO [Transaction Log Manager 2]: Removed 1 cached transaction metadata for __transaction_state-37 on follower transition (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:3413:[2017-06-09 01:16:36,109] INFO [Transaction Log Manager 1]: Loading transaction metadata from __transaction_state-37 (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:20889:[2017-06-09 01:16:39,991] INFO [Transaction Log Manager 1]: Removed 1 cached transaction metadata for __transaction_state-37 on follower transition (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:26084:[2017-06-09 01:16:46,682] INFO [Transaction Log Manager 1]: Trying to remove cached transaction metadata for __transaction_state-37 on follower transition but there is no entries remaining; it is likelythat another process for removing the cached entries has just executed earlier before (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:26322:[2017-06-09 01:16:47,153] INFO [Transaction Log Manager 1]: Trying to remove cached transaction metadata for __transaction_state-37 on follower transition but there is no entries remaining; it is likelythat another process for removing the cached entries has just executed earlier before (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:27320:[2017-06-09 01:16:50,748] INFO [Transaction Log Manager 1]: Loading transaction metadata from __transaction_state-37 (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:27331:[2017-06-09 01:16:50,775] INFO [Transaction Log Manager 1]: Finished loading 1 transaction metadata from __transaction_state-37 in 27 milliseconds (kafka.coordinator.transaction.TransactionStateManager)
> ./worker2/debug/server.log:167429:[2017-06-09 01:19:08,857] INFO [Transaction Log Manager 1]: Removed 1 cached transaction metadata for __transaction_state-37 on follower transition (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:2456:[2017-06-09 01:16:36,631] INFO [Transaction Log Manager 3]: Trying to remove cached transaction metadata for __transaction_state-37 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:8650:[2017-06-09 01:16:39,976] INFO [Transaction Log Manager 3]: Trying to remove cached transaction metadata for __transaction_state-37 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:13865:[2017-06-09 01:16:45,311] INFO [Transaction Log Manager 3]: Loading transaction metadata from __transaction_state-37 (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:13879:[2017-06-09 01:16:45,336] INFO [Transaction Log Manager 3]: Finished loading 1 transaction metadata from __transaction_state-37 in 25 milliseconds (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:44921:[2017-06-09 01:16:50,751] INFO [Transaction Log Manager 3]: Removed 1 cached transaction metadata for __transaction_state-37 on follower transition (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:54228:[2017-06-09 01:16:58,681] INFO [Transaction Log Manager 3]: Trying to remove cached transaction metadata for __transaction_state-37 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:122047:[2017-06-09 01:19:08,855] INFO [Transaction Log Manager 3]: Trying to remove cached transaction metadata for __transaction_state-37 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:123425:[2017-06-09 01:19:11,014] INFO [Transaction Log Manager 3]: Loading transaction metadata from __transaction_state-37 (kafka.coordinator.transaction.TransactionStateManager)
> ./worker7/debug/server.log:123440:[2017-06-09 01:19:11,038] INFO [Transaction Log Manager 3]: Finished loading 1 transaction metadata from __transaction_state-37 in 24 milliseconds (kafka.coordinator.transaction.TransactionStateManager)
> {noformat}
> At 01:16:50, worker2 gains ownership of this partition and retains it till 01:19:08.
> On, the client, the 'AddPartitions' request constantly gets a CONCURRENT_TRANSACTIONS response from worker2 for 01:16:50 to 01:19:09 (when it finally gets a NOT_COORIDNATOR response due to brokers being shut down after test failure).
> The reason seems to be that the write of 'CompleteCommit' to the log by the TransactionMarkerChannelManager is not being retried after failure. Here is the tail of the log for the transactionalId in question 'my-second-transactional-id'
> {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 this message is absent from the log. Could it be that the InterbrokerSendThread is in an inifinte poll? that certainly looks possible from the code. 
> Further, since we return 'success' to the client after the `PrepareCommit` is written to the log, and return a `CONCURRENT_TRANSACTIONS` error on a future `AddPartitions` request, we never move out of the `CompleteCommit` state, resulting in a hung transaction.



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