-
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
Changes from 23 commits
8b20d7d
af7def2
e91784c
39bb381
49b51b3
5cda4f8
047d3cf
d56e17e
4d7073c
15dbff5
a1a71f5
d393a59
820f91e
29e8810
aedb1b4
9ba4c96
90aa258
c5d5317
6c72eb8
4d0727f
ee3399c
ba79cf1
9fd51f8
5b13372
f3ead35
6a0613c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -44,6 +44,7 @@ | |
| import java.util.HashMap; | ||
| import java.util.HashSet; | ||
| import java.util.Map; | ||
| import java.util.OptionalLong; | ||
| import java.util.Set; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.function.Function; | ||
|
|
@@ -126,7 +127,7 @@ public Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(Map<TopicPartitio | |
|
|
||
| try { | ||
| Map<TopicPartition, ListOffsetData> fetchedOffsets = fetchOffsetsByTimes(timestampsToSearch, | ||
| timer, true).fetchedOffsets; | ||
| timer, true, false).fetchedOffsets; | ||
|
|
||
| return buildOffsetsForTimesResult(timestampsToSearch, fetchedOffsets); | ||
| } finally { | ||
|
|
@@ -136,11 +137,16 @@ public Map<TopicPartition, OffsetAndTimestamp> offsetsForTimes(Map<TopicPartitio | |
|
|
||
| private ListOffsetResult fetchOffsetsByTimes(Map<TopicPartition, Long> timestampsToSearch, | ||
| Timer timer, | ||
| boolean requireTimestamps) { | ||
| boolean requireTimestamps, | ||
| boolean shouldUpdatePartitionEndOffsets) { | ||
| ListOffsetResult result = new ListOffsetResult(); | ||
| if (timestampsToSearch.isEmpty()) | ||
| return result; | ||
|
|
||
| if (shouldUpdatePartitionEndOffsets) { | ||
| offsetFetcherUtils.setPartitionEndOffsetRequests(timestampsToSearch.keySet()); | ||
| } | ||
|
|
||
| Map<TopicPartition, Long> remainingToSearch = new HashMap<>(timestampsToSearch); | ||
| do { | ||
| RequestFuture<ListOffsetResult> future = sendListOffsetsRequests(remainingToSearch, requireTimestamps); | ||
|
|
@@ -153,11 +159,17 @@ public void onSuccess(ListOffsetResult value) { | |
| remainingToSearch.keySet().retainAll(value.partitionsToRetry); | ||
|
|
||
| offsetFetcherUtils.updateSubscriptionState(value.fetchedOffsets, isolationLevel); | ||
|
|
||
| if (shouldUpdatePartitionEndOffsets) | ||
| offsetFetcherUtils.clearPartitionEndOffsetRequests(remainingToSearch.keySet()); | ||
lianetm marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public void onFailure(RuntimeException e) { | ||
| if (shouldUpdatePartitionEndOffsets) | ||
| offsetFetcherUtils.clearPartitionEndOffsetRequests(remainingToSearch.keySet()); | ||
|
|
||
| if (!(e instanceof RetriableException)) { | ||
| throw future.exception(); | ||
| } | ||
|
|
@@ -185,23 +197,48 @@ public void onFailure(RuntimeException e) { | |
| } | ||
|
|
||
| public Map<TopicPartition, Long> beginningOffsets(Collection<TopicPartition> partitions, Timer timer) { | ||
| return beginningOrEndOffset(partitions, ListOffsetsRequest.EARLIEST_TIMESTAMP, timer); | ||
| return beginningOrEndOffset(partitions, ListOffsetsRequest.EARLIEST_TIMESTAMP, timer, false); | ||
| } | ||
|
|
||
| public Map<TopicPartition, Long> endOffsets(Collection<TopicPartition> partitions, Timer timer) { | ||
| return beginningOrEndOffset(partitions, ListOffsetsRequest.LATEST_TIMESTAMP, timer); | ||
| return beginningOrEndOffset(partitions, ListOffsetsRequest.LATEST_TIMESTAMP, timer, false); | ||
| } | ||
|
|
||
| public OptionalLong currentLag(TopicPartition topicPartition) { | ||
| final Long lag = subscriptions.partitionLag(topicPartition, isolationLevel); | ||
|
|
||
| // if the log end offset is not known and hence cannot return lag and there is | ||
| // no in-flight list offset requested yet, | ||
| // issue a list offset request for that partition so that next time | ||
| // we may get the answer; we do not need to wait for the return value | ||
| // since we would not try to poll the network client synchronously | ||
| if (lag == null) { | ||
| if (subscriptions.partitionEndOffset(topicPartition, isolationLevel) == null) { | ||
|
||
| beginningOrEndOffset( | ||
| Set.of(topicPartition), | ||
| ListOffsetsRequest.LATEST_TIMESTAMP, | ||
| time.timer(0L), | ||
| true | ||
| ); | ||
| } | ||
|
|
||
| return OptionalLong.empty(); | ||
| } | ||
|
|
||
| return OptionalLong.of(lag); | ||
| } | ||
|
|
||
| private Map<TopicPartition, Long> beginningOrEndOffset(Collection<TopicPartition> partitions, | ||
| long timestamp, | ||
| Timer timer) { | ||
| Timer timer, | ||
| boolean shouldUpdatePartitionEndOffsets) { | ||
| metadata.addTransientTopics(topicsForPartitions(partitions)); | ||
| try { | ||
| Map<TopicPartition, Long> timestampsToSearch = partitions.stream() | ||
| .distinct() | ||
| .collect(Collectors.toMap(Function.identity(), tp -> timestamp)); | ||
|
|
||
| ListOffsetResult result = fetchOffsetsByTimes(timestampsToSearch, timer, false); | ||
| ListOffsetResult result = fetchOffsetsByTimes(timestampsToSearch, timer, false, shouldUpdatePartitionEndOffsets); | ||
|
|
||
| return result.fetchedOffsets.entrySet().stream() | ||
| .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().offset)); | ||
|
|
||
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.
this name is very confusing because it clearly says it will update end offsets (first thing that comes to mind is an actual change to positions, not a flag).
Would it help if we rename to mention it's to update a flag (maybe updatePartitionEndOffsetsFlag), or at least a description of the param?
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 changed the variable name to
updatePartitionEndOffsetsFlag.