divijvaidya commented on code in PR #12590:
URL: https://github.com/apache/kafka/pull/12590#discussion_r969341737
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java:
##########
@@ -1933,11 +1941,81 @@ 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;
+ }
+
+ log.debug("Sending close request for fetch session: {} to node:
{}", sessionId, fetchTarget);
+ final RequestFuture<ClientResponse> responseFuture =
sendFetchRequestToNode(sessionHandler.newBuilder().build(), fetchTarget);
+ responseFuture.addListener(new
RequestFutureListener<ClientResponse>() {
+ @Override
+ public void onSuccess(ClientResponse value) {
+ log.info("Successfully sent a close message for fetch
session: {} to node: {}", sessionId, fetchTarget);
+ }
+
+ @Override
+ public void onFailure(RuntimeException e) {
+ log.info("Unable to a close message for fetch session: {}
to node: {}. " +
+ "This may result in unnecessary fetch sessions at the
broker.", sessionId, fetchTarget, e);
+ }
+ });
+
+ requestFutures.add(responseFuture);
+ }
+
+ // Poll to ensure that request has been written to the socket. Wait
until either the timer has expired or until
+ // all requests have received a response.
+ do {
+ client.poll(timer, null, true);
+ } while (timer.notExpired() &&
!requestFutures.stream().allMatch(RequestFuture::isDone));
+
+ if (!requestFutures.stream().allMatch(RequestFuture::isDone)) {
+ // we ran out of time before completing all futures. It is ok
since we don't want to block the shutdown
+ // here.
+ log.warn("All requests couldn't be sent in the specific timeout
period {}ms. " +
+ "This may result in unnecessary fetch sessions at the broker.
Consider increasing the timeout passed for " +
+ "KafkaConsumer.close(Duration timeout)", timer.timeoutMs());
+ }
+ }
+
+ public void close(final Timer timer) {
+ if (isClosed.get()) {
+ log.info("Fetcher {} is already closed.", this);
+ return;
+ }
+
+ // Shared states (e.g. sessionHandlers) could be accessed by multiple
threads (such as heartbeat thread), hence,
+ // it is necessary to acquire a lock on the fetcher instance before
modifying the states.
+ synchronized (Fetcher.this) {
+ // we do not need to re-enable wakeups since we are closing already
+ client.disableWakeups();
+ if (nextInLineFetch != null)
+ nextInLineFetch.drain();
+ maybeCloseFetchSessions(timer);
+ Utils.closeQuietly(decompressionBufferSupplier,
"decompressionBufferSupplier");
+ sessionHandlers.clear();
+ }
+ this.isClosed.compareAndSet(false, true);
Review Comment:
Good idea. Changed in latest commit.
--
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]