[
https://issues.apache.org/jira/browse/KAFKA-5416?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Apurva Mehta updated KAFKA-5416:
--------------------------------
Summary: TransactionCoordinator doesn't complete transition to
CompleteCommit (was: TransactionCoordinator: TransactionMarkerChannelManager
seems not to retry failed writes.)
> TransactionCoordinator doesn't complete transition to CompleteCommit
> --------------------------------------------------------------------
>
> 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
>
> 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)