Skip to content

KAFKA-5758: Don't fail fetch request if replica is no longer a follower for a partition#3954

Closed
ijuma wants to merge 9 commits intoapache:trunkfrom
ijuma:kafka-5758-dont-fail-fetch-request-if-replica-is-not-follower
Closed

KAFKA-5758: Don't fail fetch request if replica is no longer a follower for a partition#3954
ijuma wants to merge 9 commits intoapache:trunkfrom
ijuma:kafka-5758-dont-fail-fetch-request-if-replica-is-not-follower

Conversation

@ijuma
Copy link
Copy Markdown
Member

@ijuma ijuma commented Sep 25, 2017

We log a warning instead, which is what we also do if the partition
hasn't been created yet.

A few other improvements:

  • Return updated high watermark if fetch is returned immediately.
    This seems to be more intuitive and is consistent with the case
    where the fetch request is served from the purgatory.
  • Centralise offline partition handling
  • Remove unnecessary tryCompleteDelayedProduce that would
    already have been done by the called method
  • A few other minor clean-ups

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Sep 25, 2017

@junrao, does this look reasonable?

@ijuma ijuma force-pushed the kafka-5758-dont-fail-fetch-request-if-replica-is-not-follower branch 2 times, most recently from 909c689 to 6c73665 Compare September 25, 2017 13:30
@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Sep 29, 2017

retest this please

Copy link
Copy Markdown
Contributor

@junrao junrao left a comment

Choose a reason for hiding this comment

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

@ijuma : Thanks for the patch. Great cleanup as always. Just a few minor comments below.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

The line probably should be above line 792 now?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Good catch that this comment is in the wrong place. I updated the code a little and I think the comment doesn't add much after that, so I removed it. Let me know if you disagree.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This seems to be an existing problem, but it seems that we only need to call tryCompleteDelayedRequests() when the HW increments.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

I think tryCompleteDelayedDeleteRecords is affected by leaderLWIncremented.

private def tryCompleteDelayedRequests() {
  val requestKey = new TopicPartitionOperationKey(topicPartition)
  replicaManager.tryCompleteDelayedFetch(requestKey)
  replicaManager.tryCompleteDelayedProduce(requestKey)
  replicaManager.tryCompleteDelayedDeleteRecords(requestKey)
}

We could make the tryComplete calls more specific depending on what changed, if you think that's worth it.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Shouldn't we set the broker.id property since ReplicaManager needs config.brokerId?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Yes, createBrokerConfig does it:

if (nodeId >= 0) props.put(KafkaConfig.BrokerIdProp, nodeId.toString)

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

If the replica is not in the assigned replica list, I am wondering if we should just send an empty fetched data back.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Yes, I agree that it would be better and I was a bit unhappy about the wastefulness of sending a non empty fetch response in that case. Will fix.

@ijuma ijuma force-pushed the kafka-5758-dont-fail-fetch-request-if-replica-is-not-follower branch from 84ffc4b to 45f52b3 Compare October 1, 2017 01:14
@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Oct 1, 2017

retest this please

@ijuma
Copy link
Copy Markdown
Member Author

ijuma commented Oct 2, 2017

@junrao, I've addressed your comments.

@junrao
Copy link
Copy Markdown
Contributor

junrao commented Oct 2, 2017

@ijuma : Thanks for the updated patch. LGTM. I will let you merge it.

@asfgit asfgit closed this in e110e1c Oct 2, 2017
val logStartOffsets = assignedReplicas.collect {
case replica if replicaManager.metadataCache.isBrokerAlive(replica.brokerId) => replica.logStartOffset
}
CoreUtils.min(logStartOffsets, 0L)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Wouldn't this return 0 in most cases ?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

No, it only returns 0 if the collection is empty.

@ijuma ijuma deleted the kafka-5758-dont-fail-fetch-request-if-replica-is-not-follower branch December 22, 2017 18:24
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 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>
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>
isolis pushed a commit to linkedin/kafka that referenced this pull request Sep 13, 2018
KAFKA-5758; Don't fail fetch request if replica is no longer a follower for a partition

We log a warning instead, which is what we also do if the partition
hasn't been created yet.

A few other improvements:
- Return updated high watermark if fetch is returned immediately.
This seems to be more intuitive and is consistent with the case
where the fetch request is served from the purgatory.
- Centralise offline partition handling
- Remove unnecessary `tryCompleteDelayedProduce` that would
already have been done by the called method
- A few other minor clean-ups

Author: Ismael Juma <ismael@juma.me.uk>

Reviewers: Jun Rao <junrao@gmail.com>

Closes apache#3954 from ijuma/kafka-5758-dont-fail-fetch-request-if-replica-is-not-follower
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.

3 participants