-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-15696: Refactor AsyncConsumer close procedure #14920
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 all commits
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 |
|---|---|---|
|
|
@@ -79,6 +79,7 @@ | |
| import org.apache.kafka.common.utils.Time; | ||
| import org.apache.kafka.common.utils.Timer; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.event.Level; | ||
|
|
||
| import java.net.InetSocketAddress; | ||
| import java.time.Duration; | ||
|
|
@@ -93,10 +94,13 @@ | |
| import java.util.Optional; | ||
| import java.util.OptionalLong; | ||
| import java.util.Set; | ||
| import java.util.SortedSet; | ||
| import java.util.TreeSet; | ||
| import java.util.concurrent.BlockingQueue; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.ExecutionException; | ||
| import java.util.concurrent.LinkedBlockingQueue; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
|
|
@@ -119,6 +123,7 @@ | |
| import static org.apache.kafka.common.utils.Utils.closeQuietly; | ||
| import static org.apache.kafka.common.utils.Utils.isBlank; | ||
| import static org.apache.kafka.common.utils.Utils.join; | ||
| import static org.apache.kafka.common.utils.Utils.swallow; | ||
|
|
||
| /** | ||
| * This {@link Consumer} implementation uses an {@link ApplicationEventHandler event handler} to process | ||
|
|
@@ -210,7 +215,7 @@ private void process(final GroupMetadataUpdateEvent event) { | |
|
|
||
| private final ApplicationEventHandler applicationEventHandler; | ||
| private final Time time; | ||
| private Optional<ConsumerGroupMetadata> groupMetadata; | ||
| private Optional<ConsumerGroupMetadata> groupMetadata = Optional.empty(); | ||
| private final KafkaConsumerMetrics kafkaConsumerMetrics; | ||
| private Logger log; | ||
| private final String clientId; | ||
|
|
@@ -233,6 +238,7 @@ private void process(final GroupMetadataUpdateEvent event) { | |
| private final Metrics metrics; | ||
| private final long retryBackoffMs; | ||
| private final int defaultApiTimeoutMs; | ||
| private final boolean autoCommitEnabled; | ||
| private volatile boolean closed = false; | ||
| private final List<ConsumerPartitionAssignor> assignors; | ||
| private final Optional<ClientTelemetryReporter> clientTelemetryReporter; | ||
|
|
@@ -264,6 +270,7 @@ private void process(final GroupMetadataUpdateEvent event) { | |
| GroupRebalanceConfig.ProtocolType.CONSUMER | ||
| ); | ||
| this.clientId = config.getString(CommonClientConfigs.CLIENT_ID_CONFIG); | ||
| this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); | ||
| LogContext logContext = createLogContext(config, groupRebalanceConfig); | ||
| this.log = logContext.logger(getClass()); | ||
|
|
||
|
|
@@ -378,7 +385,8 @@ private void process(final GroupMetadataUpdateEvent event) { | |
| long retryBackoffMs, | ||
| int defaultApiTimeoutMs, | ||
| List<ConsumerPartitionAssignor> assignors, | ||
| String groupId) { | ||
| String groupId, | ||
| boolean autoCommitEnabled) { | ||
| this.log = logContext.logger(getClass()); | ||
| this.subscriptions = subscriptions; | ||
| this.clientId = clientId; | ||
|
|
@@ -398,6 +406,7 @@ private void process(final GroupMetadataUpdateEvent event) { | |
| this.assignors = assignors; | ||
| this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, "consumer"); | ||
| this.clientTelemetryReporter = Optional.empty(); | ||
| this.autoCommitEnabled = autoCommitEnabled; | ||
| } | ||
|
|
||
| // Visible for testing | ||
|
|
@@ -413,6 +422,7 @@ private void process(final GroupMetadataUpdateEvent event) { | |
| this.log = logContext.logger(getClass()); | ||
| this.subscriptions = subscriptions; | ||
| this.clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG); | ||
| this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG); | ||
| this.fetchBuffer = new FetchBuffer(logContext); | ||
| this.isolationLevel = IsolationLevel.READ_UNCOMMITTED; | ||
| this.interceptors = new ConsumerInterceptors<>(Collections.emptyList()); | ||
|
|
@@ -1024,15 +1034,14 @@ private void close(Duration timeout, boolean swallowException) { | |
| final Timer closeTimer = time.timer(timeout); | ||
| clientTelemetryReporter.ifPresent(reporter -> reporter.initiateClose(timeout.toMillis())); | ||
| closeTimer.update(); | ||
|
|
||
| // Prepare shutting down the network thread | ||
| swallow(log, Level.ERROR, "Unexpected exception when preparing for shutdown", () -> prepareShutdown(closeTimer), firstException); | ||
| closeTimer.update(); | ||
| if (applicationEventHandler != null) | ||
| closeQuietly(() -> applicationEventHandler.close(Duration.ofMillis(closeTimer.remainingMs())), "Failed to close application event handler with a timeout(ms)=" + closeTimer.remainingMs(), firstException); | ||
|
|
||
| // Invoke all callbacks after the background thread exists in case if there are unsent async | ||
| // commits | ||
| maybeInvokeCommitCallbacks(); | ||
|
|
||
| closeQuietly(fetchBuffer, "Failed to close the fetch buffer", firstException); | ||
| closeQuietly(() -> applicationEventHandler.close(Duration.ofMillis(closeTimer.remainingMs())), "Failed shutting down network thread", firstException); | ||
| closeTimer.update(); | ||
| // Ensure all async commit callbacks are invoked | ||
| swallow(log, Level.ERROR, "Failed invoking asynchronous commit callback", this::maybeInvokeCommitCallbacks, firstException); | ||
| closeQuietly(interceptors, "consumer interceptors", firstException); | ||
| closeQuietly(kafkaConsumerMetrics, "kafka consumer metrics", firstException); | ||
| closeQuietly(metrics, "consumer metrics", firstException); | ||
|
|
@@ -1050,6 +1059,57 @@ private void close(Duration timeout, boolean swallowException) { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Prior to closing the network thread, we need to make sure the following operations happen in the right sequence: | ||
| * 1. autocommit offsets | ||
| * 2. revoke all partitions | ||
| */ | ||
| private void prepareShutdown(final Timer timer) { | ||
| if (!groupMetadata.isPresent()) | ||
| return; | ||
|
|
||
| maybeAutoCommitSync(timer); | ||
| timer.update(); | ||
| if (!subscriptions.hasAutoAssignedPartitions() || subscriptions.assignedPartitions().isEmpty()) | ||
| return; | ||
|
|
||
| try { | ||
| // If the consumer is in a group, we will pause and revoke all assigned partitions | ||
| onLeavePrepare().get(timer.remainingMs(), TimeUnit.MILLISECONDS); | ||
| timer.update(); | ||
| } catch (Exception e) { | ||
| Exception exception = e; | ||
| if (e instanceof ExecutionException) | ||
| exception = (Exception) e.getCause(); | ||
| throw new KafkaException("User rebalance callback throws an error", exception); | ||
| } finally { | ||
| subscriptions.assignFromSubscribed(Collections.emptySet()); | ||
| } | ||
| } | ||
|
|
||
| private void maybeAutoCommitSync(final Timer timer) { | ||
| if (autoCommitEnabled) { | ||
| Map<TopicPartition, OffsetAndMetadata> allConsumed = subscriptions.allConsumed(); | ||
| try { | ||
| log.debug("Sending synchronous auto-commit of offsets {} on closing", allConsumed); | ||
| commitSync(allConsumed, Duration.ofMillis(timer.remainingMs())); | ||
| } catch (Exception e) { | ||
| // consistent with async auto-commit failures, we do not propagate the exception | ||
| log.warn("Synchronous auto-commit of offsets {} failed: {}", allConsumed, e.getMessage()); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private CompletableFuture<Void> onLeavePrepare() { | ||
|
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. May miss a bit of context, but I'm not yet sure what this function is achieving. If this is for KAFKA-15276, maybe we can implement this as part of that ticket, because this function is mostly confusing me.
Contributor
Author
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. I need to speak to kirk about how he wants to implement the callback invocation. |
||
| SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(MembershipManagerImpl.TOPIC_PARTITION_COMPARATOR); | ||
| droppedPartitions.addAll(subscriptions.assignedPartitions()); | ||
| if (!subscriptions.hasAutoAssignedPartitions() || droppedPartitions.isEmpty()) { | ||
| return CompletableFuture.completedFuture(null); | ||
| } | ||
| // TODO: Invoke rebalanceListener via KAFKA-15276 | ||
| return CompletableFuture.completedFuture(null); | ||
| } | ||
|
|
||
|
Comment on lines
+1062
to
+1112
Contributor
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. I would imagine that the leave group process could be mostly performed using an event and a callback execution, like in #14931. We'd need to submit an event to the background thread (e.g.
Contributor
Author
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. Thanks. I mostly agree with your idea. Though - I think simply firing the callback revocation from the close() should be enough - but I think sending leave-group and closing events as you suggested is a good idea.
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. Several people seem to agree that we should solve this as much as possible via an event. Is the new draft PR going to replace this PR, or should we try to merge this one?
Contributor
Author
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. I want to be consistent with Kirk's approach for unsubscribe() in terms of callback invocation - So I took the previous comment back (sorry about the confusion) |
||
| @Override | ||
| public void wakeup() { | ||
| wakeupTrigger.wakeup(); | ||
|
|
||
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.
These should be done before we shut down the network, right?