-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-10167: use the admin client to read end-offset #8876
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
guozhangwang
merged 11 commits into
apache:trunk
from
guozhangwang:KXXX-end-offsets-uncommitted
Jun 18, 2020
Merged
Changes from all commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
b9a27a8
check
guozhangwang a322765
Merge branch 'trunk' of https://github.com/apache/kafka into KXXX-end…
guozhangwang b4589e6
continue revamping
guozhangwang a1a4bb6
Merge branch 'trunk' of https://github.com/apache/kafka into KXXX-end…
guozhangwang 7f81699
github comments
guozhangwang 2fd0ea5
add the mock-admin client to ttd
guozhangwang 67bf693
copy past the mock admin client
guozhangwang 6934cd6
github comments
guozhangwang ebe86f9
not use changelog at all
guozhangwang 097dbca
Merge branch 'trunk' of https://github.com/apache/kafka into KXXX-end…
guozhangwang 8d80cce
github comments
guozhangwang 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
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
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
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,10 +16,15 @@ | |
| */ | ||
| package org.apache.kafka.streams.processor.internals; | ||
|
|
||
| import org.apache.kafka.clients.admin.Admin; | ||
| import org.apache.kafka.clients.admin.ListOffsetsOptions; | ||
| import org.apache.kafka.clients.admin.ListOffsetsResult; | ||
| import org.apache.kafka.clients.admin.OffsetSpec; | ||
| import org.apache.kafka.clients.consumer.Consumer; | ||
| import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
| import org.apache.kafka.clients.consumer.ConsumerRecords; | ||
| import org.apache.kafka.clients.consumer.InvalidOffsetException; | ||
| import org.apache.kafka.common.IsolationLevel; | ||
| import org.apache.kafka.common.KafkaException; | ||
| import org.apache.kafka.common.TopicPartition; | ||
| import org.apache.kafka.common.errors.TimeoutException; | ||
|
|
@@ -43,6 +48,8 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.function.Function; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchCommittedOffsets; | ||
|
|
@@ -199,6 +206,9 @@ int bufferedLimitIndex() { | |
| // to update offset limit for standby tasks; | ||
| private Consumer<byte[], byte[]> mainConsumer; | ||
|
|
||
| // the changelog reader needs the admin client to list end offsets | ||
| private final Admin adminClient; | ||
|
|
||
| private long lastUpdateOffsetTime; | ||
|
|
||
| void setMainConsumer(final Consumer<byte[], byte[]> consumer) { | ||
|
|
@@ -208,11 +218,13 @@ void setMainConsumer(final Consumer<byte[], byte[]> consumer) { | |
| public StoreChangelogReader(final Time time, | ||
| final StreamsConfig config, | ||
| final LogContext logContext, | ||
| final Admin adminClient, | ||
| final Consumer<byte[], byte[]> restoreConsumer, | ||
| final StateRestoreListener stateRestoreListener) { | ||
| this.time = time; | ||
| this.log = logContext.logger(StoreChangelogReader.class); | ||
| this.state = ChangelogReaderState.ACTIVE_RESTORING; | ||
| this.adminClient = adminClient; | ||
| this.restoreConsumer = restoreConsumer; | ||
| this.stateRestoreListener = stateRestoreListener; | ||
|
|
||
|
|
@@ -564,8 +576,13 @@ private Map<TopicPartition, Long> endOffsetForChangelogs(final Set<TopicPartitio | |
| return Collections.emptyMap(); | ||
|
|
||
| try { | ||
| return restoreConsumer.endOffsets(partitions); | ||
| } catch (final TimeoutException e) { | ||
| final ListOffsetsResult result = adminClient.listOffsets( | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Did you check out the new methods in |
||
| partitions.stream().collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest())), | ||
| new ListOffsetsOptions(IsolationLevel.READ_UNCOMMITTED) | ||
| ); | ||
| return result.all().get().entrySet().stream().collect( | ||
| Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().offset())); | ||
| } catch (final TimeoutException | InterruptedException | ExecutionException e) { | ||
| // if timeout exception gets thrown we just give up this time and retry in the next run loop | ||
| log.debug("Could not fetch all end offsets for {}, will retry in the next run loop", partitions); | ||
| return Collections.emptyMap(); | ||
|
|
||
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.
Uh oh!
There was an error while loading. Please reload this page.