Calvin Liu created KAFKA-19996:
----------------------------------
Summary: The conflicts between ExpandIsr and ShrinkIsr
Key: KAFKA-19996
URL: https://issues.apache.org/jira/browse/KAFKA-19996
Project: Kafka
Issue Type: Bug
Reporter: Calvin Liu
Thanks to [~mjd95] to report the following issue.
In the Partition.scala, the condition for the ISR expand and shrink does not
match, which can cause a follower flapping in and out of ISR when:
# The partition is under min-isr which means the HWM can't advance.
# The replication is slow.
# The follower is far behind the LEO, but it is at/above the HWM.
Due to the "conflict" between the ISR expand and shrink:
* Shrink ISR, attempted on a schedule, triggers if the follower's LEO has not
recently caught up with the leader's LEO which takes the last caught up time
into the consideration.
{code:java}
/**
* Returns true when the replica is considered as "caught-up". A replica is
* considered "caught-up" when its log end offset is equals to the log end
* offset of the leader OR when its last caught up time minus the current
* time is smaller than the max replica lag.
*/
public boolean isCaughtUp(
long leaderEndOffset,
long currentTimeMs,
long replicaMaxLagMs) {
return leaderEndOffset == logEndOffset() || currentTimeMs -
lastCaughtUpTimeMs <= replicaMaxLagMs;
} {code}
* Expand ISR, attempted on processing follower fetch, triggers if the
follower's LEO caught up with the leader's HWM in this fetch which only
consider whether the replica reaches the HWM.
{code:java}
private def isFollowerInSync(followerReplica: Replica): Boolean = {
leaderLogIfLocal.exists { leaderLog =>
val followerEndOffset = followerReplica.stateSnapshot.logEndOffset
followerEndOffset >= leaderLog.highWatermark &&
leaderEpochStartOffsetOpt.exists(followerEndOffset >= _)
}
} {code}
--
This message was sent by Atlassian Jira
(v8.20.10#820010)