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

Dong Lin commented on KAFKA-7040:
---------------------------------

[~luwang] Nice catch. It is indeed problematic (with some probability of losing 
data) that a replica fetcher thread can truncate to an offset smaller than the 
high watermark due to receiving an "outdated" OffsetsForLeaderEpochResponse.

One possible solution is to add one more field, say `initialized`, in 
AbstractFetcherThread.partitionStates. The value is set to false after the 
partition is added to partitionStates. ReplicaFetcherThread will update 
`initialized` to true the first time it reads this partition. Later, after 
ReplicaFetcherThread receives either FetchResponse or 
OffsetsForLeaderEpochResponse, it should discard this partitions in the 
response whose `initialized` is false. This seems to fix the issue here.

 

 

> The replica fetcher thread may truncate accepted messages during multiple 
> fast leadership transitions
> -----------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-7040
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7040
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Lucas Wang
>            Priority: Minor
>
> Problem Statement:
> Consider the scenario where there are two brokers, broker0, and broker1, and 
> there are two partitions "t1p0", and "t1p1"[1], both of which have broker1 as 
> the leader and broker0 as the follower. The following sequence of events 
> happened on broker0
> 1. The replica fetcher thread on a broker0 issues a LeaderEpoch request to 
> broker1, and awaits to get the response
> 2. A LeaderAndISR request causes broker0 to become the leader for one 
> partition t1p0, which in turn will remove the partition t1p0 from the replica 
> fetcher thread
> 3. Broker0 accepts some messages from a producer
> 4. A 2nd LeaderAndISR request causes broker1 to become the leader, and 
> broker0 to become the follower for partition t1p0. This will cause the 
> partition t1p0 to be added back to the replica fetcher thread on broker0.
> 5. The replica fetcher thread on broker0 receives a response for the 
> LeaderEpoch request issued in step 1, and truncates the accepted messages in 
> step3.
> The issue can be reproduced with the test from 
> https://github.com/gitlw/kafka/commit/8956e743f0e432cc05648da08c81fc1167b31bea
> [1] Initially we set up broker0 to be the follower of two partitions instead 
> of just one, to avoid the shutting down of the replica fetcher thread when it 
> becomes idle.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to