Skip to content

KAFKA-9491; Increment high watermark after full log truncation#8037

Merged
ijuma merged 1 commit intoapache:trunkfrom
hachikuji:KAFKA-9491
Feb 4, 2020
Merged

KAFKA-9491; Increment high watermark after full log truncation#8037
ijuma merged 1 commit intoapache:trunkfrom
hachikuji:KAFKA-9491

Conversation

@hachikuji
Copy link
Copy Markdown
Contributor

@hachikuji hachikuji commented Feb 4, 2020

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.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

updateHighWatermark(offset)
}

if (this.recoveryPoint < offset) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is a recovery point lower than the log start offset useful? All data below the log start offset is subject to deletion.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

U are right :)

Copy link
Copy Markdown
Contributor

@rajinisivaram rajinisivaram left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@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)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

resetLastCaughtUpTime takes curLeaderLogEndOffset, but we are now passing leaderEpochStartOffset. Do we need to update the parameter name of that method?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

They are the same thing. I just got rid of a redundant variable.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, I see.


this.recoveryPoint = math.min(newOffset, this.recoveryPoint)
this.logStartOffset = newOffset
updateLogStartOffset(newOffset)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the explanation!

@ijuma
Copy link
Copy Markdown
Member

ijuma commented Feb 4, 2020

1 job passed, 1 had 3 flaky failures:

kafka.admin.DeleteConsumerGroupsTest.testDeleteWithMixOfSuccessAndError
kafka.admin.DescribeConsumerGroupTest.testDescribeGroupWithShortInitializationTimeout
org.apache.kafka.streams.kstream.internals.KStreamImplTest.shouldSupportTriggerMaterializedWithKTableFromKStream

@ijuma ijuma merged commit b029902 into apache:trunk Feb 4, 2020
ijuma pushed a commit that referenced this pull request Feb 4, 2020
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>
ijuma pushed a commit that referenced this pull request Feb 4, 2020
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>
@ijuma
Copy link
Copy Markdown
Member

ijuma commented Feb 4, 2020

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.

ijuma added a commit to confluentinc/kafka that referenced this pull request Feb 7, 2020
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)
stanislavkozlovski pushed a commit to stanislavkozlovski/kafka that referenced this pull request Feb 18, 2020
…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>
ijuma added a commit to ijuma/kafka that referenced this pull request Apr 28, 2020
…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)
  ...
qq619618919 pushed a commit to qq619618919/kafka that referenced this pull request May 12, 2020
…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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants