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

Alyssa Huang commented on KAFKA-19354:
--------------------------------------

Hey Justin, I believe what José is saying that this can increase the likelihood 
of hitting this bug, but won't prevent it entirely. If you simulate the network 
delay you can certainly continue to see the issue. 

> KRaft observer unable to recover after re-bootstrapping to follower
> -------------------------------------------------------------------
>
>                 Key: KAFKA-19354
>                 URL: https://issues.apache.org/jira/browse/KAFKA-19354
>             Project: Kafka
>          Issue Type: Bug
>          Components: kraft
>    Affects Versions: 4.0.0
>            Reporter: Justin Chen
>            Assignee: Alyssa Huang
>            Priority: Major
>
> [Original dev mail 
> thread|https://lists.apache.org/thread/ws3390khsxhdg2b8cnv2mzv8slz5xq7q]
> If an observer's FETCH request to the quorum leader experiences a 
> failure/timeout, it is possible that when it re-bootstraps, it will connect 
> to a follower node (random selection). Subsequently, the observer node will 
> continually send FETCH requests to that follower, and in receive a response 
> with a "partitionError" errorCode=6 (NOT_LEADER_OR_FOLLOWER), which does not 
> trigger a re-bootstrap.
> Thus, the observer will be stuck sending FETCH requests to the follower 
> instead of the leader, halting metadata replication and causing it to fall 
> out of sync.
> To recover from this state, re-bootstrapping would need to occur by 
> restarting the affected observer or follower, until it connects to the 
> correct leader.
> *Steps to reproduce:*
> 1. Spin up Kafka cluster with 3 or 5 controllers. (ideally 5 to increase 
> likelihood of bootstrapping to a follower instead of the leader)
> 2. Enable a network delay on a particular observer broker (e.g. `tc qdisc add 
> dev eth0 root netem delay 2500ms`). I picked 2500ms since default timeout is 
> 2s for 
> `controller.quorum.fetch.timeout.ms`/`controller.quorum.request.timeout.ms`. 
> After a few seconds, disable the network delay (e.g. `tc qdisc del dev eth0 
> root netem`).
> 3. The observer node will re-bootstrap, potentially to a follower instead of 
> the leader. If so, the observer will continuously send fetch requests to the 
> follower node, receive `NOT_LEADER_OR_FOLLOWER` in response, and will no 
> longer replicate metadata.
> *Debug logs demonstrating this scenario:*
> - https://gist.github.com/justin-chen/1f3eee79d9a5066a467818a0b1bc006f
> - kraftcontroller-3 (leader), kraftcontroller-4 (follower), kafka-0 (observer)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to