-
Notifications
You must be signed in to change notification settings - Fork 15k
KAFKA-20131: ClassicKafkaConsumer does not clear endOffsetRequested flag on failed LIST_OFFSETS calls #21457
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
Merged
AndrewJSchofield
merged 26 commits into
apache:trunk
from
kirktrue:KAFKA-20131-clear-endOffsetRequested-on-LIST_OFFSETS-failures
Feb 20, 2026
Merged
Changes from 8 commits
Commits
Show all changes
26 commits
Select commit
Hold shift + click to select a range
8b20d7d
KAFKA-20131: SubscriptionState endOffsetRequested is left set when LI…
kirktrue af7def2
WIP
kirktrue e91784c
Updates
kirktrue 39bb381
Clearing partition end offset request flag for new consumer
kirktrue 49b51b3
Comments and minor formatting
kirktrue 5cda4f8
More comments
kirktrue 047d3cf
Catching more cases
kirktrue d56e17e
Merge branch 'trunk' into KAFKA-20131-clear-endOffsetRequested-on-LIS…
kirktrue 4d7073c
Merge branch 'trunk' into KAFKA-20131-clear-endOffsetRequested-on-LIS…
kirktrue 15dbff5
Fixed typo in log message and verifying requestPartitionEndOffset inv…
kirktrue a1a71f5
Updates to remove changes to AsyncKafkaConsumer
kirktrue d393a59
Updates to remove changes to AsyncKafkaConsumer
kirktrue 820f91e
More clean up for PR
kirktrue 29e8810
Removed unnecessary changes to OffsetFetcher
kirktrue aedb1b4
Another minor refactor
kirktrue 9ba4c96
Removed checking for log message output
kirktrue 90aa258
Merge branch 'trunk' into KAFKA-20131-clear-endOffsetRequested-on-LIS…
kirktrue c5d5317
Revising the logic around currentLag() in ClassicKafkaConsumer and Of…
kirktrue 6c72eb8
Reverting changes in ApplicationEventProcessor
kirktrue 4d0727f
Reverted several changes to SubscriptionState
kirktrue ee3399c
Moved both setting and clearing of partitionEndOffsets flag to fetchO…
kirktrue ba79cf1
Minor clean up to reduce diff noise
kirktrue 9fd51f8
Added setPartitionEndOffsetRequests
kirktrue 5b13372
Renamed shouldUpdatePartitionEndOffsets → updatePartitionEndOffsetsFl…
kirktrue f3ead35
Adding testCurrentLagPreventsMultipleInFlightRequests test and minor …
kirktrue 6a0613c
Minor refactoring of request count check to reformat and include message
kirktrue File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Some comments aren't visible on the classic Files Changed page.
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -141,6 +141,15 @@ private ListOffsetResult fetchOffsetsByTimes(Map<TopicPartition, Long> timestamp | |
| if (timestampsToSearch.isEmpty()) | ||
| return result; | ||
|
|
||
| // In the case that the user supplied a zero timeout to this method, only a single pass of the loop below will | ||
| // be performed before exiting. No TimeoutException will be thrown in that case. In the case that the single | ||
| // pass did not yield a response (either transient or fatal error), make sure to clear the relevant partitions' | ||
| // respective 'end offset requested' flags so that another attempt can be made by the user. | ||
| // | ||
| // If the timeout is not zero, the loop will be executed at least once. In the case that not all the partitions | ||
| // were found, a TimeoutException will be thrown. | ||
| boolean isZeroTimestamp = timer.timeoutMs() == 0L; | ||
|
|
||
| Map<TopicPartition, Long> remainingToSearch = new HashMap<>(timestampsToSearch); | ||
| do { | ||
| RequestFuture<ListOffsetResult> future = sendListOffsetsRequests(remainingToSearch, requireTimestamps); | ||
|
|
@@ -153,11 +162,14 @@ public void onSuccess(ListOffsetResult value) { | |
| remainingToSearch.keySet().retainAll(value.partitionsToRetry); | ||
|
|
||
| offsetFetcherUtils.updateSubscriptionState(value.fetchedOffsets, isolationLevel); | ||
| offsetFetcherUtils.clearPartitionEndOffsetRequests(remainingToSearch.keySet()); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void onFailure(RuntimeException e) { | ||
| offsetFetcherUtils.clearPartitionEndOffsetRequests(remainingToSearch.keySet()); | ||
|
||
|
|
||
| if (!(e instanceof RetriableException)) { | ||
| throw future.exception(); | ||
| } | ||
|
|
@@ -167,7 +179,7 @@ public void onFailure(RuntimeException e) { | |
| // if timeout is set to zero, do not try to poll the network client at all | ||
| // and return empty immediately; otherwise try to get the results synchronously | ||
| // and throw timeout exception if it cannot complete in time | ||
| if (timer.timeoutMs() == 0L) | ||
| if (isZeroTimestamp) | ||
| return result; | ||
|
|
||
| client.poll(future, timer); | ||
|
|
@@ -181,6 +193,10 @@ public void onFailure(RuntimeException e) { | |
| } | ||
| } while (timer.notExpired()); | ||
|
|
||
| // If there are any remaining partitions that have not received responses, clear their respective | ||
| // 'end offset requested' flags so that another attempt can be made. | ||
| offsetFetcherUtils.clearPartitionEndOffsetRequests(remainingToSearch.keySet()); | ||
|
|
||
| throw new TimeoutException("Failed to get offsets by times in " + timer.elapsedMs() + "ms"); | ||
| } | ||
|
|
||
|
|
||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
here we're clearing the flag for the partitions that didn't get offsets yet. I agree we need this if we don't have any time left to retry. But if there's still time, the do-while will try again. In that case, do we want to clear the flag here?
I would imagine we don't, because we'll continue retrying while there is time. It could be the case of missing leader info for instance: we want to keep the flag on for those partitions, hit the
client.awaitMetadataUpdate(timer)below, and try again in the next iteration of the do-while, right?If so, I imagine we could take the timer into consideration here? (clear the flag for the failed partitions only if timer expired?). Thoughts?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's precisely what happens in the
currentLag()case, though. It's always using a timeout of0, so there's never a second pass in that loop.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok, we both agree we need it for currentLag/timerExpired. But in the way it's called now it applies to all cases, that's my concern. Isn't this going to clear the flag also in the case where there is time left to retry, and there is a partition that didn't have a known leader?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I've added an explicit parameter to 'clear end offsets requests' that only the
ClassicKafkaConsumer.currentLag()sets to true. This should prevent other callers from clearing the flag, regardless of the timeout setting.