[ https://issues.apache.org/jira/browse/KAFKA-2143?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14730109#comment-14730109 ]
Jun Rao commented on KAFKA-2143: -------------------------------- [~becket_qin], since before step (3), both A and B are in ISR, the last committed offset in A can't be larger than 3000. So, in step (3), if A becomes a follower, it has to first truncate its log to last committed offset before fetching. So, at that point, A's fetch offset can't be larger than 3000 and therefore won't be out of range. The following is a alternative scenario that can cause this. 1) Broker A (leader) receives messages to 5000 2) Broker B (follower) receives messages to 3000 (it is still in ISR because of replica.lag.max.messages) 3) For some reason, B is dropped out of ISR. 4) Broker A (the only one in ISR) commits messages to 5000. 5) For some reason, Broker A is considered dead and Broker B is live. 6) Broker B is selected as the new leader (unclean leader election) and is the only one in ISR. 7) Broker A is considered live again and starts fetching from 5000 (last committed offset) and gets OffsetOutOfRangeException. 8) In the mean time, B receives more messages to offset 6000. 9) Broker A tries to handle OffsetOutOfRangeException and finds out leader B's log end offset is now larger than its log end offset and truncates all its log. Your patch reduces the amount of the data that Broker A needs to replicate in step 9, which is probably fine. However, we probably should first verify if this is indeed what's happening since it seems that it should happen rarely. Also, KAFKA-2477 reports a similar issue w/o any leadership change. So, may be there is something else that can cause this. > Replicas get ahead of leader and fail > ------------------------------------- > > Key: KAFKA-2143 > URL: https://issues.apache.org/jira/browse/KAFKA-2143 > Project: Kafka > Issue Type: Bug > Components: replication > Affects Versions: 0.8.2.1 > Reporter: Evan Huus > Assignee: Jiangjie Qin > > On a cluster of 6 nodes, we recently saw a case where a single > under-replicated partition suddenly appeared, replication lag spiked, and > network IO spiked. The cluster appeared to recover eventually on its own, > Looking at the logs, the thing which failed was partition 7 of the topic > {{background_queue}}. It had an ISR of 1,4,3 and its leader at the time was > 3. Here are the interesting log lines: > On node 3 (the leader): > {noformat} > [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when > processing fetch request for partition [background_queue,7] offset 3722949957 > from follower with correlation id 148185816. Possible cause: Request for > offset 3722949957 but we only have log segments in the range 3648049863 to > 3722949955. (kafka.server.ReplicaManager) > [2015-04-23 16:50:05,879] ERROR [Replica Manager on Broker 3]: Error when > processing fetch request for partition [background_queue,7] offset 3722949957 > from follower with correlation id 156007054. Possible cause: Request for > offset 3722949957 but we only have log segments in the range 3648049863 to > 3722949955. (kafka.server.ReplicaManager) > [2015-04-23 16:50:13,960] INFO Partition [background_queue,7] on broker 3: > Shrinking ISR for partition [background_queue,7] from 1,4,3 to 3 > (kafka.cluster.Partition) > {noformat} > Note that both replicas suddenly asked for an offset *ahead* of the available > offsets. > And on nodes 1 and 4 (the replicas) many occurrences of the following: > {noformat} > [2015-04-23 16:50:05,935] INFO Scheduling log segment 3648049863 for log > background_queue-7 for deletion. (kafka.log.Log) (edited) > {noformat} > Based on my reading, this looks like the replicas somehow got *ahead* of the > leader, asked for an invalid offset, got confused, and re-replicated the > entire topic from scratch to recover (this matches our network graphs, which > show 3 sending a bunch of data to 1 and 4). > Taking a stab in the dark at the cause, there appears to be a race condition > where replicas can receive a new offset before the leader has committed it > and is ready to replicate? -- This message was sent by Atlassian JIRA (v6.3.4#6332)