[ https://issues.apache.org/jira/browse/KAFKA-9307?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16997882#comment-16997882 ]
Dhruvil Shah commented on KAFKA-9307: ------------------------------------- Exception during step 6 that led to partial completion of become-follower transition: {code:java} 2019-12-12 03:11:56,320] ERROR [Transaction State Manager 3]: The metadata cache for txn partition 41 has already exist with epoch 111 and 6 entries while trying to add to it; this should not happen (kafka.coordinator.transaction.TransactionStateManager) [2019-12-12 03:11:56,320] ERROR Uncaught exception in scheduled task 'load-txns-for-partition-__transaction_state-41' (kafka.utils.KafkaScheduler) java.lang.IllegalStateException: The metadata cache for txn partition 41 has already exist with epoch 111 and 6 entries while trying to add to it; this should not happen at kafka.coordinator.transaction.TransactionStateManager.addLoadedTransactionsToCache(TransactionStateManager.scala:369) at kafka.coordinator.transaction.TransactionStateManager$$anonfun$kafka$coordinator$transaction$TransactionStateManager$$loadTransactions$1$1.apply$mcV$sp(TransactionStateManager.scala:394) at kafka.coordinator.transaction.TransactionStateManager$$anonfun$kafka$coordinator$transaction$TransactionStateManager$$loadTransactions$1$1.apply(TransactionStateManager.scala:393) at kafka.coordinator.transaction.TransactionStateManager$$anonfun$kafka$coordinator$transaction$TransactionStateManager$$loadTransactions$1$1.apply(TransactionStateManager.scala:393) at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:259) at kafka.coordinator.transaction.TransactionStateManager.kafka$coordinator$transaction$TransactionStateManager$$loadTransactions$1(TransactionStateManager.scala:392) at kafka.coordinator.transaction.TransactionStateManager$$anonfun$loadTransactionsForTxnTopicPartition$2.apply$mcV$sp(TransactionStateManager.scala:426) at kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:114) at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:63) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java) at java.lang.Thread.run(Thread.java:748) {code} > Transaction coordinator could be left in unknown state after ZK session > timeout > ------------------------------------------------------------------------------- > > Key: KAFKA-9307 > URL: https://issues.apache.org/jira/browse/KAFKA-9307 > Project: Kafka > Issue Type: Bug > Components: core > Reporter: Dhruvil Shah > Assignee: Dhruvil Shah > Priority: Major > > We observed a case where the transaction coordinator could not load > transaction state from __transaction-state topic partition. Clients would > continue seeing COORDINATOR_LOAD_IN_PROGRESS exceptions until the broker > hosting the coordinator is restarted. > This is the sequence of events that leads to the issue: > # The broker is the leader of one (or more) transaction state topic > partitions. > # The broker loses its ZK session due to a network issue. > # Broker reestablishes session with ZK, though there are still transient > network issues. > # Broker is made follower of the transaction state topic partition it was > leading earlier. > # During the become-follower transition, the broker loses its ZK session > again. > # The become-follower transition for this broker fails in-between, leaving > us in a partial leader / partial follower state for the transaction topic. > This meant that we could not unload the transaction metadata. However, the > broker successfully caches the leader epoch of associated with the > LeaderAndIsrRequest. > # Later, when the ZK session is finally established successfully, the broker > ignores the become-follower transition as the leader epoch was same as the > one it had cached. This prevented the transaction metadata from being > unloaded. > # Because this partition was a partial follower, we had setup replica > fetchers. The partition continued to fetch from the leader until it was made > part of the ISR. > # Once it was part of the ISR, preferred leader election kicked in and > elected this broker as the leader. > # When processing the become-leader transition, the operation failed as we > already had transaction metadata loaded at a previous epoch. > # This meant that this partition was left in the "loading" state and we thus > returned COORDINATOR_LOAD_IN_PROGRESS errors. > # Broker restart fixed this partial in-memory state and we were able to > resume processing for transactions. -- This message was sent by Atlassian Jira (v8.3.4#803005)