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

Dhruvil Shah updated KAFKA-9307:
--------------------------------
    Description: 
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 transaction state load 
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.

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


> 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 transaction state load 
> 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)

Reply via email to