You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2020/02/04 06:20:00 UTC

[jira] [Commented] (KAFKA-9491) Fast election during reassignment can lead to replica fetcher failures

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

ASF GitHub Bot commented on KAFKA-9491:
---------------------------------------

hachikuji commented on pull request #8037: KAFKA-9491; Increment high watermark after full log truncation
URL: https://github.com/apache/kafka/pull/8037
 
 
   When a follower's fetch offset is behind the leader's log start offset, the follower will do a full log truncation. When it does so, it must update both its log start offset and high watermark. Failure to do so can lead to out of range errors if the follower becomes leader before getting the latest high watermark from the previous leader. The out of range errors occur when we attempt to resolve the log position of the high watermark in `DelayedFetch` in order to determine if a fetch is satisfied.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> 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
>            Reporter: Jason Gustafson
>            Assignee: Jason Gustafson
>            Priority: Major
>
> 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)