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

Francois Visconte commented on KAFKA-17678:
-------------------------------------------

[~calvinliu] any opinion on that one (since you're working on KIP-966)? 

> Problematic new HWM increment behaviour introduced by KIP-207 and KIP-966
> -------------------------------------------------------------------------
>
>                 Key: KAFKA-17678
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17678
>             Project: Kafka
>          Issue Type: Bug
>          Components: replication
>            Reporter: Francois Visconte
>            Priority: Major
>
> We identified a bug/new behaviour that would lead to consumer lagging for a 
> long time and ListOffsets requests failing during that time frame.
> While the ListOffsets requests failure is expected and has been introduced by 
> KIP-207, the problematic behavior is more about the inability to increment 
> highWatermark and the consequence of having lagging consumers.
> Here is the situation
>  * We have a topic with min.isr=2
>  * We have a partition on broker 16, 17 and 18
>  * Leader for this partition is broker 17
>  # Broker 18 failed. Partition has 2 ISRs
>  # Broker 16 failed. Partition has 1 ISR (17)
>  # Broker 7 has LEO higher than HWM:
> {{[Broker id=17] Leader topic-86 with topic id Some(yFhPOnPsRDiYHgfF2bR2aQ) 
> starts at leader epoch 7 from offset 3067193660 with partition epoch 11, high 
> watermark 3067191497, ISR [10017], adding replicas [] and removing replicas 
> [] (under-min-isr). Previous leader Some(10017) and previous leader epoch was 
> 6.}}
> At this point producers cannot produce to topic-86 partition because there is 
> only one ISR, which is expected behavior.
> But it seems that KIP-207 prevent answering to ListOffsets requests here
>  
> {code:java}
> // Only consider throwing an error if we get a client request (isolationLevel 
> is defined) and the high watermark
> // is lagging behind the start offset
> val maybeOffsetsError: Option[ApiException] = leaderEpochStartOffsetOpt
> .filter(epochStart => isolationLevel.isDefined && epochStart > 
> localLog.highWatermark)
> .map(epochStart => Errors.OFFSET_NOT_AVAILABLE.exception(s"Failed to fetch 
> offsets for " +
> s"partition $topicPartition with leader $epochLogString as this partition's " 
> +
> s"high watermark (${localLog.highWatermark}) is lagging behind the " +
> s"start offset from the beginning of this epoch ($epochStart).")){code}
> It seems that the path to get to the HWM being stuck for so long was 
> introduced in preparation of KIP-966, see this ticket and PR.
> As a result:
>  * The stuck HWM in the above scenario can also mean that a small part of 
> messages isn't readable by consumers even though it was in the past.
>  * In case of truncation, the HWM might still go backwards. This is still 
> possible even with min.ISR, although it should be rare.



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

Reply via email to