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

Jason Gustafson commented on KAFKA-9491:
----------------------------------------

After reviewing the previous logic, it looks like this was a regression in 
2.4.0. It came from the new logic in KIP-392, which required followers to 
maintain the high watermark more strictly than before.

> Fast election during reassignment can lead to replica fetcher failures
> ----------------------------------------------------------------------
>
>                 Key: KAFKA-9491
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9491
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.4.0
>            Reporter: Jason Gustafson
>            Assignee: Jason Gustafson
>            Priority: Major
>             Fix For: 2.5.0, 2.4.1
>
>
> We have observed an unusual case in which a new replica became leader before 
> it had received an initial high watermark from the previous leader. This 
> resulted in an OffsetOutOfRangeException being raised while looking up the 
> segment position of the uninitialized high watermark, since it was lower than 
> the log start offset. The error was raised while handle the fetch request 
> from one of the followers and prevented it from making progress.
> {code}
> org.apache.kafka.common.errors.OffsetOutOfRangeException: Received request 
> for offset 0 for partition foo-0, but we only have log segments in the range 
> 20 to 20.
> {code}
> Here is what we have observed from the logs. The initial state of the 
> partition for the relevant sequence of events is the following:
> Initial state: replicas=[4,1,2,3], leader=1, isr=[1,2,3], adding=[4], 
> removing=[1], epoch=5, logStartOffset=20, logEndOffset=20
> We see the following events:
> t0: Replica 4 becomes follower and initializes log with hw=0, logStartOffset=0
> t1: Replica 4 begins fetching from offset 0 and receives an out of range error
> t2: After a ListOffset request to the leader, replica 4 initializes 
> logStartOffset to 20.
> t3: Replica 4 sends fetch request to the leader at start offset 20
> t4: Upon receiving the fetch request, the leader adds 4 to the ISR (i.e. 
> isr=[1,2,3,4])
> t5: The controller notices the ISR addition and makes 4 the leader since 1 is 
> to be removed and 4 is the new preferred leader
> t6: Replica 4 stops fetchers and becomes leader
> t7: We begin seeing the out of range errors as the other replicas begin 
> fetching from 4.
> We know from analysis of a heap dump from broker 4, that the high watermark 
> was still set to 0 some time after it had become leader. We also know that 
> broker 1 was under significant load. The time between events t4 and t6 was 
> less than 10ms. We don't know when the fetch response sent at t3 returned to 
> broker 4, but we speculate that it happened after t6 due to the heavy load on 
> the leader, which is why broker 4 had an uninitialized high watermark.
> A more mundane possibility is that there is a bug in the fetch session logic 
> and the partition was simply not included in the fetch response. However, the 
> code appears to anticipate this case. When a partition has an error, we set 
> the cached high watermark to -1 to ensure that it gets updated as soon as the 
> error clears.
> Regardless how we got there, the fix should be straightforward. When a broker 
> becomes leader, it should ensure its high watermark is at least as large as 
> the log start offset.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to