kirktrue commented on code in PR #12590: URL: https://github.com/apache/kafka/pull/12590#discussion_r999989762
########## clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java: ########## @@ -2403,17 +2404,44 @@ private ClusterResourceListeners configureClusterResourceListeners(Deserializer< return clusterResourceListeners; } - private void close(long timeoutMs, boolean swallowException) { + private Timer createTimerForRequest(final Duration timeout) { + final Time localTime = (time == null) ? new SystemTime() : time; + return localTime.timer(Math.min(timeout.toMillis(), requestTimeoutMs)); + } + + private void close(Duration timeout, boolean swallowException) { log.trace("Closing the Kafka consumer"); AtomicReference<Throwable> firstException = new AtomicReference<>(); - try { - if (coordinator != null) - coordinator.close(time.timer(Math.min(timeoutMs, requestTimeoutMs))); - } catch (Throwable t) { - firstException.compareAndSet(null, t); - log.error("Failed to close coordinator", t); + + final Timer closeTimer = createTimerForRequest(timeout); + // Close objects with a timeout. The timeout is required because coordinator & fetcher send requests to the + // server in the process of closing which may not respect the overall timeout defined for closing the consumer. + if (coordinator != null) { + try { + coordinator.close(closeTimer); + } catch (Throwable t) { + firstException.compareAndSet(null, t); + log.error("Failed to close consumer coordinator", t); + } } - Utils.closeQuietly(fetcher, "fetcher", firstException); + + if (fetcher != null) { + // the timeout for the session close is at-most the requestTimeoutMs + long remainingDurationInTimeout = Math.max(0, timeout.toMillis() - closeTimer.elapsedMs()); + if (remainingDurationInTimeout > 0) { + remainingDurationInTimeout = Math.min(requestTimeoutMs, remainingDurationInTimeout); + } + + closeTimer.reset(remainingDurationInTimeout); + + try { + fetcher.close(closeTimer); Review Comment: Does this call to `Fetcher.close()` become blocking as well? It calls into `maybeCloseFetchSessions()` which ends up calling `ConsumerNetworkClient.poll()`. ########## clients/src/main/java/org/apache/kafka/clients/FetchSessionHandler.java: ########## @@ -590,6 +595,14 @@ public boolean handleResponse(FetchResponse response, short version) { } } + /** + * The client will initiate the session close on next fetch request. + */ + public void notifyClose() { + log.info("Set the metadata for next fetch request to close the existing session ID={} ", nextMetadata.sessionId()); Review Comment: The trailing space in the message format is intentional or no? Sorry -- I know that's really nitpick-y. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java: ########## @@ -1933,11 +1941,79 @@ private Map<String, String> topicPartitionTags(TopicPartition tp) { } } + // Visible for testing + void maybeCloseFetchSessions(final Timer timer) { + final Cluster cluster = metadata.fetch(); + final List<RequestFuture<ClientResponse>> requestFutures = new ArrayList<>(); + for (final Map.Entry<Integer, FetchSessionHandler> entry : sessionHandlers.entrySet()) { + final FetchSessionHandler sessionHandler = entry.getValue(); + // set the session handler to notify close. This will set the next metadata request to send close message. + sessionHandler.notifyClose(); + + final int sessionId = sessionHandler.sessionId(); + final Integer fetchTargetNodeId = entry.getKey(); + // FetchTargetNode may not be available as it may have disconnected the connection. In such cases, we will + // skip sending the close request. + final Node fetchTarget = cluster.nodeById(fetchTargetNodeId); + if (fetchTarget == null || client.isUnavailable(fetchTarget)) { + log.debug("Skip sending close session request to broker {} since it is not reachable", fetchTarget); + continue; + } + + final RequestFuture<ClientResponse> responseFuture = sendFetchRequestToNode(sessionHandler.newBuilder().build(), fetchTarget); + responseFuture.addListener(new RequestFutureListener<ClientResponse>() { Review Comment: Is it possible for the listener to not have been added before the response is received? I see we don't call `poll()` until after we've added all the requests (and their listeners), but is there another thread that can call `poll()` without us knowing? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java: ########## @@ -446,6 +429,31 @@ private RequestFuture<ClientResponse> sendMetadataRequest(MetadataRequest.Builde return client.send(node, request); } + /** + * Send Fetch Request to Kafka cluster asynchronously. + * + * This method is visible for testing. + * + * @return A future that indicates result of sent Fetch request + */ + RequestFuture<ClientResponse> sendFetchRequestToNode(final FetchSessionHandler.FetchRequestData requestData, + final Node fetchTarget) { + final short maxVersion = requestData.canUseTopicIds() ? ApiKeys.FETCH.latestVersion() : (short) 12; Review Comment: Newbie question: where is this `12` documented? Is it referring to `FetchRequestData.SCHEMA_12`/`FetchTopic.SCHEMA_12`? Is there a way to refer to the schema number programmatically instead of via the magic number? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java: ########## @@ -446,6 +429,31 @@ private RequestFuture<ClientResponse> sendMetadataRequest(MetadataRequest.Builde return client.send(node, request); } + /** + * Send Fetch Request to Kafka cluster asynchronously. + * + * This method is visible for testing. + * + * @return A future that indicates result of sent Fetch request + */ + RequestFuture<ClientResponse> sendFetchRequestToNode(final FetchSessionHandler.FetchRequestData requestData, + final Node fetchTarget) { + final short maxVersion = requestData.canUseTopicIds() ? ApiKeys.FETCH.latestVersion() : (short) 12; Review Comment: Just to be clear: I know that's unrelated to your change. I'm just curious as I wasn't sure what `12` referred to, exactly 😄 ########## clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java: ########## @@ -2403,17 +2404,40 @@ private ClusterResourceListeners configureClusterResourceListeners(Deserializer< return clusterResourceListeners; } - private void close(long timeoutMs, boolean swallowException) { + private void close(Duration timeout, boolean swallowException) { log.trace("Closing the Kafka consumer"); AtomicReference<Throwable> firstException = new AtomicReference<>(); - try { - if (coordinator != null) - coordinator.close(time.timer(Math.min(timeoutMs, requestTimeoutMs))); - } catch (Throwable t) { - firstException.compareAndSet(null, t); - log.error("Failed to close coordinator", t); + + final Timer closeTimer = (time == null) ? new SystemTime().timer(Math.min(timeout.toMillis(), requestTimeoutMs)) : time.timer(Math.min(timeout.toMillis(), requestTimeoutMs)); + // Close objects with a timeout. The timeout is required because fetcher makes request to the server in the + // process of closing which may not respect the overall timeout defined for closing the consumer. + if (coordinator != null) { + try { + coordinator.close(closeTimer); Review Comment: Would you mind adding that comment about the blocking nature of the `close` call to the source for posterity? -- 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