KAFKA-9491; Increment high watermark after full log truncation#8037
KAFKA-9491; Increment high watermark after full log truncation#8037ijuma merged 1 commit intoapache:trunkfrom
Conversation
| updateHighWatermark(offset) | ||
| } | ||
|
|
||
| if (this.recoveryPoint < offset) { |
There was a problem hiding this comment.
Is there a case that recoveryPoint gets smaller than logStartOffset? If yes, is it ok to just move the recoveryPoint without flush? If not, is it worthy to throw exception or log this weird case?
There was a problem hiding this comment.
Is a recovery point lower than the log start offset useful? All data below the log start offset is subject to deletion.
rajinisivaram
left a comment
There was a problem hiding this comment.
@hachikuji Thanks for the PR, LGTM
| remoteReplicas.foreach { replica => | ||
| val lastCaughtUpTimeMs = if (inSyncReplicaIds.contains(replica.brokerId)) curTimeMs else 0L | ||
| replica.resetLastCaughtUpTime(curLeaderLogEndOffset, curTimeMs, lastCaughtUpTimeMs) | ||
| replica.resetLastCaughtUpTime(leaderEpochStartOffset, curTimeMs, lastCaughtUpTimeMs) |
There was a problem hiding this comment.
resetLastCaughtUpTime takes curLeaderLogEndOffset, but we are now passing leaderEpochStartOffset. Do we need to update the parameter name of that method?
There was a problem hiding this comment.
They are the same thing. I just got rid of a redundant variable.
|
|
||
| this.recoveryPoint = math.min(newOffset, this.recoveryPoint) | ||
| this.logStartOffset = newOffset | ||
| updateLogStartOffset(newOffset) |
There was a problem hiding this comment.
Previous behavior is recoveryPoint = math.min(newOffset, recoveryPoint)) but this patch changes it to
if (this.recoveryPoint < offset) {
this.recoveryPoint = offset
}
which is equal to recoveryPoint = math.max(newOffset, recoveryPoint)). Is it a bug?
There was a problem hiding this comment.
I have also updated updateLogEndOffset to set the recovery point. In truncateFully where we delete all segments and set the log start offset to be equal to the log end offset, this ensures recovery point is also set consistently.
|
1 job passed, 1 had 3 flaky failures:
|
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. The previous code did the former, but not the latter. Failure to update the high watermark in this case 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. Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
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. The previous code did the former, but not the latter. Failure to update the high watermark in this case 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. Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
|
Merged to trunk, 2.5 and 2.4. There were some conflicts when I tried to cherry-pick to 2.3, so leaving that to @hachikuji. |
Conflicts: * build.gradle: moved avro plugin definition below newly added test retry plugin. * apache-github/trunk: MINOR: further InternalTopologyBuilder cleanup (apache#8046) MINOR: Add timer for update limit offsets (apache#8047) HOTFIX: Fix spotsbug failure in Kafka examples (apache#8051) KAFKA-9447: Add new customized EOS model example (apache#8031) KAFKA-8164: Add support for retrying failed (apache#8019) HOTFIX: checkstyle for newly added unit test KAFKA-9261; Client should handle unavailable leader metadata (apache#7770) MINOR: Fix typos introduced in KIP-559 (apache#8042) MINOR: Fixing null handilg in ValueAndTimestampSerializer (apache#7679) KAFKA-9113: Clean up task management and state management (apache#7997) MINOR: fix checkstyle issue in ConsumerConfig.java (apache#8038) KAFKA-9491; Increment high watermark after full log truncation (apache#8037) KAFKA-9477 Document RoundRobinAssignor as an option for partition.assignment.strategy (apache#8007) KAFKA-9074: Correct Connect’s `Values.parseString` to properly parse a time and timestamp literal (apache#7568) KAFKA-9492; Ignore record errors in ProduceResponse for older versions (apache#8030)
…e#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. The previous code did the former, but not the latter. Failure to update the high watermark in this case 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. Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
…t-for-generated-requests * apache-github/trunk: (410 commits) KAFKA-8843: KIP-515: Zookeeper TLS support MINOR: Add missing quote for malformed line content (apache#8070) MINOR: Simplify KafkaProducerTest (apache#8044) KAFKA-9507; AdminClient should check for missing committed offsets (apache#8057) KAFKA-9519: Deprecate the --zookeeper flag in ConfigCommand (apache#8056) KAFKA-9509; Fixing flakiness of MirrorConnectorsIntegrationTest.testReplication (apache#8048) HOTFIX: Fix two test failures in JDK11 (apache#8063) DOCS - clarify transactionalID and idempotent behavior (apache#7821) MINOR: further InternalTopologyBuilder cleanup (apache#8046) MINOR: Add timer for update limit offsets (apache#8047) HOTFIX: Fix spotsbug failure in Kafka examples (apache#8051) KAFKA-9447: Add new customized EOS model example (apache#8031) KAFKA-8164: Add support for retrying failed (apache#8019) HOTFIX: checkstyle for newly added unit test KAFKA-9261; Client should handle unavailable leader metadata (apache#7770) MINOR: Fix typos introduced in KIP-559 (apache#8042) MINOR: Fixing null handilg in ValueAndTimestampSerializer (apache#7679) KAFKA-9113: Clean up task management and state management (apache#7997) MINOR: fix checkstyle issue in ConsumerConfig.java (apache#8038) KAFKA-9491; Increment high watermark after full log truncation (apache#8037) ...
…e#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. The previous code did the former, but not the latter. Failure to update the high watermark in this case 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. Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
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. The previous code did the former, but not the latter. Failure to update the high watermark in this case 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
DelayedFetchin order to determine if a fetch is satisfied.Committer Checklist (excluded from commit message)