lucasbru commented on code in PR #14710: URL: https://github.com/apache/kafka/pull/14710#discussion_r1400592977
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ########## @@ -90,14 +93,13 @@ public void run() { while (running) { try { runOnce(); - } catch (final WakeupException e) { - log.debug("WakeupException caught, consumer network thread won't be interrupted"); + } catch (final Throwable e) { + log.error("Unexpected error caught in consumer network thread", e); // swallow the wakeup exception to prevent killing the thread. Review Comment: Good question. I'm not sure what will end up in the catch block here. The code around processing events seems to be wrapped by another catch-all block, so we'd expect only problems around sending requests to end up here? If you are not sure, open a ticket for investigation. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java: ########## @@ -202,6 +210,40 @@ public CompletableFuture<Void> maybeAutoCommitAllConsumed() { return maybeAutoCommit(subscriptions.allConsumed()); } + /** + * The consumer needs to send an auto commit during the shutdown if autocommit is enabled. + */ + Optional<NetworkClientDelegate.UnsentRequest> maybeCreateAutoCommitRequest() { + if (!autoCommitState.isPresent()) { + return Optional.empty(); + } + + OffsetCommitRequestState request = pendingRequests.createOffsetCommitRequest(subscriptions.allConsumed(), jitter); + request.future.whenComplete(autoCommitCallback(subscriptions.allConsumed())); + return Optional.of(request.toUnsentRequest()); + } + + private CompletableFuture<Void> sendAutoCommit(final Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets) { + log.debug("Enqueuing autocommit offsets: {}", allConsumedOffsets); + return addOffsetCommitRequest(allConsumedOffsets).whenComplete(autoCommitCallback(allConsumedOffsets)); + } + + private BiConsumer<? super Void, ? super Throwable> autoCommitCallback(final Map<TopicPartition, OffsetAndMetadata> allConsumedOffsets) { + return (response, throwable) -> { + autoCommitState.ifPresent(autoCommitState -> autoCommitState.setInflightCommitStatus(false)); Review Comment: `inFlightCommitStatus` isn't read in production code. Do we need it? If we need it, are we doing enough to avoid having two auto-commits in flight (one regular, one during close?). ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java: ########## @@ -257,10 +257,77 @@ private void closeInternal(final Duration timeout) { void cleanup() { log.trace("Closing the consumer network thread"); Timer timer = time.timer(closeTimeout); + waitForClosingTasks(timer); runAtClose(requestManagers.entries(), networkClientDelegate, timer); closeQuietly(requestManagers, "request managers"); closeQuietly(networkClientDelegate, "network client delegate"); closeQuietly(applicationEventProcessor, "application event processor"); log.debug("Closed the consumer network thread"); } + + /** + * We need to autocommit before shutting down the consumer. The method needs to first connect to the coordinator + * node to construct the closing requests. Then wait for all closing requests to finish before returning. The + * method is bounded by a closing timer. We will continue closing down the consumer if the requests cannot be + * completed in time. + */ + // Visible for testing + void waitForClosingTasks(final Timer timer) { Review Comment: Yeah, that's better ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/AutoCommitCompletionBackgroundEvent.java: ########## @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals.events; + +public class AutoCommitCompletionBackgroundEvent extends BackgroundEvent { Review Comment: I don't think we cannot not handle it at all in this PR, since `BackgroundEventProcessor` will throw an `IllegalArgumentException` when it encounters this event. If you want to move this to a separate task, we should make the handling a NOOP. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org