Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-7104: Consistent leader's state in fetch response #5305

Merged
merged 2 commits into from
Jun 28, 2018

Conversation

apovzner
Copy link
Contributor

@apovzner apovzner commented Jun 28, 2018

Do not update LogReadResult after it is initially populated when returning fetches immediately (i.e. without hitting the purgatory). This was done in #3954 as optimization so that followers get most recent high watermark and log start offset. However, since many things can happen (like deleting old segments and advancing log start offset) between initial creation of LogReadResult and the update, we can hit issues like log start offset in fetch response being higher than the last offset in fetched records.

Committer Checklist (excluded from commit message)

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

@@ -1340,7 +1335,12 @@ class ReplicaManager(val config: KafkaConfig,

/**
* Update the follower's fetch state in the leader based on the last fetch request and update `readResult`,
* if necessary.
* if the follower replica is not recognized to be one of the assigned replicas. Do not update
* `readResult` otherwise, so that log start offset and high watermark is consistent with
Copy link
Contributor

Choose a reason for hiding this comment

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

Should it be log start/end here and in other places?

Copy link
Contributor

@ijuma ijuma left a comment

Choose a reason for hiding this comment

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

LGTM. @hachikuji, does it look good to you?

Copy link

@hachikuji hachikuji left a comment

Choose a reason for hiding this comment

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

LGTM. Thanks for the fix!

@ijuma ijuma changed the title KAFKA-7104: More consistent leader's state in fetch response KAFKA-7104: Consistent leader's state in fetch response Jun 28, 2018
@ijuma ijuma merged commit 10b84a3 into apache:trunk Jun 28, 2018
ijuma pushed a commit that referenced this pull request Jun 28, 2018
Do not update LogReadResult after it is initially populated when returning fetches immediately (i.e. without hitting the purgatory). This was done in #3954 as an optimization so that the followers get the potentially updated high watermark. However, since many things can happen (like deleting old segments and advancing log start offset) between initial creation of LogReadResult and the update, we can hit issues like log start offset in fetch response being higher than the last offset in fetched records.

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
ijuma pushed a commit that referenced this pull request Jun 28, 2018
Do not update LogReadResult after it is initially populated when returning fetches immediately (i.e. without hitting the purgatory). This was done in #3954 as an optimization so that the followers get the potentially updated high watermark. However, since many things can happen (like deleting old segments and advancing log start offset) between initial creation of LogReadResult and the update, we can hit issues like log start offset in fetch response being higher than the last offset in fetched records.

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
@ijuma
Copy link
Contributor

ijuma commented Jun 28, 2018

Merging to trunk, 2.0, 1.1 and 1.0 branches.

ijuma pushed a commit that referenced this pull request Jun 28, 2018
Do not update LogReadResult after it is initially populated when returning fetches immediately (i.e. without hitting the purgatory). This was done in #3954 as an optimization so that the followers get the potentially updated high watermark. However, since many things can happen (like deleting old segments and advancing log start offset) between initial creation of LogReadResult and the update, we can hit issues like log start offset in fetch response being higher than the last offset in fetched records.

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
@junrao
Copy link
Contributor

junrao commented Jun 28, 2018

@apovzner : Thanks for the patch. The patch looks good. It's probably useful to also tighten up the follower side to bound log start offset by log end offset too so that it's consistent with HW.

@junrao
Copy link
Contributor

junrao commented Jun 28, 2018

Also, while it's reasonable not to include the latest log start offset in the fetch response, it's probably useful to propagate the latest HW (after artition.updateReplicaLogReadResult is called) to the client. This allows the client to calculate more accurate lags.

@ijuma
Copy link
Contributor

ijuma commented Jun 28, 2018

@junrao, I agree that it would be nice to propagate the hw change as soon as we know it. But we went with the safe and simple option for this PR that was backported to all the branches. We can consider whether that optimisation is worth it or if we should do it as part of the read from follower work. @hachikuji preferred the latter.

ying-zheng pushed a commit to ying-zheng/kafka that referenced this pull request Jul 6, 2018
…5305)

Do not update LogReadResult after it is initially populated when returning fetches immediately (i.e. without hitting the purgatory). This was done in apache#3954 as an optimization so that the followers get the potentially updated high watermark. However, since many things can happen (like deleting old segments and advancing log start offset) between initial creation of LogReadResult and the update, we can hit issues like log start offset in fetch response being higher than the last offset in fetched records.

Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
allenxwang pushed a commit to allenxwang/kafka that referenced this pull request Aug 24, 2018
…:1.1.1-sync to 1.1-nflx

* commit '9611672e287c1a7933a78590e3f381da2ae7d136': (57 commits)
  MINOR: increase dev version from 1.1.1-SNAPSHOT to 1.1.2-SNAPSHOT (apache#5409)
  MINOR: Add thread dumps if broker node cannot be stopped (apache#5373)
  MINOR: update release.py
  MINOR: fix upgrade docs for Streams (apache#5392)
  MINOR: improve docs version numbers (apache#5372)
  Update version on the branch to 1.1.2-SNAPSHOT
  KAFKA-6292; Improve FileLogInputStream batch position checks to avoid type overflow (apache#4928)
  HOTFIX: Fix checkstyle errors in MetricsTest (apache#5345)
  KAFKA-7136: Avoid deadlocks in synchronized metrics reporters (apache#5341)
  MINOR: Close timing window in SimpleAclAuthorizer startup (apache#5318)
  MINOR: Use kill_java_processes when killing ConsoleConsumer in system tests (apache#5297)
  KAFKA-7104: More consistent leader's state in fetch response (apache#5305)
  Revert "MINOR: Avoid coarse lock in Pool#getAndMaybePut (apache#5258)"
  MINOR: Avoid coarse lock in Pool#getAndMaybePut (apache#5258)
  MINOR: bugfix streams total metrics (apache#5277)
  KAFKA-7082: Concurrent create topics may throw NodeExistsException (apache#5259)
  MINOR: Upgrade to Gradle 4.8.1
  KAFKA-7012: Don't process SSL channels without data to process (apache#5237)
  KAFKA-7058: Comparing schema default values using Objects#deepEquals()
  KAFKA-7047: Added SimpleHeaderConverter to plugin isolation whitelist
  ...
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