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

[jira] [Commented] (KAFKA-5416) TransactionCoordinator doesn't complete transition to CompleteCommit

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

ASF GitHub Bot commented on KAFKA-5416:
---------------------------------------

GitHub user guozhangwang opened a pull request:

    https://github.com/apache/kafka/pull/3287

    KAFKA-5416: Re-prepare transition to CompleteCommit/Abort upon retrying append to log

    In `TransationStateManager`, we reset the pending state if an error occurred while appending to log; this is correct except that for the `TransactionMarkerChannelManager`, as it will retry appending to log and if eventually it succeeded, the transaction metadata's completing transition will throw an IllegalStateException since pending state is None, this will be thrown all the way to the `KafkaApis` and be swallowed.
    
    1. When re-enqueueing to the retry append queue, re-prepare transition to set its pending state.
    2. A bunch of log4j improvements based the debugging experience. The main principle is to make sure all error codes that is about to sent to the client will be logged, and unnecessary log4j entries to be removed.
    3. Also moved some log entries in ReplicationUtils.scala to `trace`: this is rather orthogonal to this PR but I found it rather annoying while debugging the logs.
    4. A couple of unrelated bug fixes as pointed by @hachikuji and @apurvam .

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/guozhangwang/kafka KHotfix-transaction-coordinator-append-callback

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/kafka/pull/3287.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3287
    
----
commit 755f01201774f6fb5ddcdff87caaa78634847ebe
Author: Guozhang Wang <wa...@gmail.com>
Date:   2017-06-09T23:02:44Z

    re-prepare transition to completeXX upon retrying append to log

----


> 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
>            Assignee: Guozhang Wang
>            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)