kirktrue commented on code in PR #14937:
URL: https://github.com/apache/kafka/pull/14937#discussion_r1418183946


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkThread.java:
##########
@@ -274,79 +269,18 @@ private void closeInternal(final Duration timeout) {
     }
 
     void cleanup() {
+        log.trace("Closing the consumer network thread");
+        Timer timer = time.timer(closeTimeout);
         try {
-            log.trace("Closing the consumer network thread");
-            Timer timer = time.timer(closeTimeout);
-            maybeAutocommitOnClose(timer);
             runAtClose(requestManagers.entries(), networkClientDelegate, 
timer);
-            maybeLeaveGroup(timer);
         } catch (Exception e) {
             log.error("Unexpected error during shutdown.  Proceed with 
closing.", e);
         } finally {
+            networkClientDelegate.awaitPendingRequests(timer);

Review Comment:
   Network requests are tied to the `CompletableApplicationEvent`s, right? Can 
we just rely on the events to wait for their network I/O to complete via the 
`addAndGet()` method.?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerCloseApplicationEvent.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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 ConsumerCloseApplicationEvent extends 
CompletableApplicationEvent<Void> {

Review Comment:
   Why not just have separate event types as per the rest of the codebase?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ConsumerCloseApplicationEvent.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * 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 ConsumerCloseApplicationEvent extends 
CompletableApplicationEvent<Void> {

Review Comment:
   I'm happy to have a superclass for 'close' events, but having a type and a 
task gets a bit muddy, doesn't it?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -45,15 +46,19 @@ public class ApplicationEventProcessor extends 
EventProcessor<ApplicationEvent>
     private final Logger log;
     private final ConsumerMetadata metadata;
     private final RequestManagers requestManagers;
+    private final NetworkClientDelegate networkClientDelegate;

Review Comment:
   I'm uncomfortable with introducing the `NetworkClientDelegate` at this 
layer. It's centralized in `ConsumerNetworkThread` for the reason that we can 
reason on where the various network I/O is performed.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -113,6 +118,10 @@ public void process(ApplicationEvent event) {
                 process((UnsubscribeApplicationEvent) event);
                 return;
 
+            case PREP_CLOSING:
+                processPrepClosingEvent((ConsumerCloseApplicationEvent) event);
+                return;
+

Review Comment:
   Any reason we can't have these as separate types like the other events?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -223,23 +232,54 @@ private void process(final TopicMetadataApplicationEvent 
event) {
         event.chain(future);
     }
 
+    private void processPrepClosingEvent(ConsumerCloseApplicationEvent event) {
+        switch (event.task()) {
+            case COMMIT:
+                log.debug("Sending unsent commit before closing.");
+                sendUnsentCommit();
+                event.future().complete(null);

Review Comment:
   This is a bit of a different pattern than our other 
`CompletableApplicationEvent`s. In the other events, we completed the `Future` 
when the response was processed. In these events, we're completing them just 
after sending off the request. Is that truly what we want?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -223,23 +232,54 @@ private void process(final TopicMetadataApplicationEvent 
event) {
         event.chain(future);
     }
 
+    private void processPrepClosingEvent(ConsumerCloseApplicationEvent event) {
+        switch (event.task()) {
+            case COMMIT:
+                log.debug("Sending unsent commit before closing.");
+                sendUnsentCommit();
+                event.future().complete(null);
+                break;
+            case LEAVE_GROUP:
+                log.debug("Leaving group before closing");
+                
requestManagers.membershipManager.ifPresent(MembershipManager::leaveGroupOnClose);
+                event.future().complete(null);
+                break;
+            default:
+                log.warn("Invalid ConsumerCloseApplicationEvent task {}", 
event.task());
+                event.future().completeExceptionally(new 
KafkaException("Invalid closing task."));
+        }
+    }
+
+    private void sendUnsentCommit() {
+        if (!requestManagers.commitRequestManager.isPresent())
+            return;
+        NetworkClientDelegate.PollResult res = 
requestManagers.commitRequestManager.get().pollOnClose();
+        if (res.unsentRequests.isEmpty())
+            return;
+        // NetworkThread will continue to poll the networkClientDelegate
+        networkClientDelegate.addAll(res);
+    }
+

Review Comment:
   I'm not quite understanding why this needs to be done as a special case. Why 
can't we rely on the normal `runOnce()` invocation to `poll()` the request 
managers?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1050,6 +1062,81 @@ 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, final 
AtomicReference<Throwable> firstException) {
+        if (!groupMetadata.isPresent())
+            return;
+        maybeAutoCommitSync(timer, firstException);
+        timer.update();
+        waitOnEventCompletion(new 
ConsumerCloseApplicationEvent(ConsumerCloseApplicationEvent.Task.COMMIT, 
timer.remainingMs()), timer, firstException);
+        maybeInvokeCommitCallbacks();
+        maybeRevokePartitions(timer, firstException);
+        waitOnEventCompletion(new 
ConsumerCloseApplicationEvent(ConsumerCloseApplicationEvent.Task.LEAVE_GROUP, 
timer.remainingMs()), timer, firstException);
+    }
+
+    private void waitOnEventCompletion(final ConsumerCloseApplicationEvent 
event,
+                                       final Timer timer,
+                                       final AtomicReference<Throwable> 
firstException) {
+        try {
+            applicationEventHandler.addAndGet(event, timer);
+        } catch (TimeoutException e) {

Review Comment:
   Yes, we have issue with timeouts of 0 elsewhere. There's a Jira somewhere to 
solve it, but it's not been designed/fixed.



-- 
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

Reply via email to