divijvaidya commented on code in PR #12590:
URL: https://github.com/apache/kafka/pull/12590#discussion_r1000596477
##########
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:
No, it is not possible because multi-thread usage of `KafkaConsumer` is not
supported. We enforce this by acquiring a lock at the beginning of
`KafkaConsumer#close()`. Hence, there won't be another thread which would call
poll() while this method is executing.
Separately, even if we miss adding the listener due to race conditions, it
is ok since the log messages are best effort and the this whole close sequence
is best-effort. Worst case, either we don't log the messages or the close
request doesn't reach the server. Both these cases are recoverable and doesn't
have large impact on server.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]