Skip to content

Conversation

@AntonVasant
Copy link
Contributor

@AntonVasant AntonVasant commented Nov 22, 2025

This PR improves the logging behavior in AbstractHeartbeatRequestManager
when the heartbeat response is null due to an exception.

Currently, TimeoutException / DisconnectException are logged at ERROR
level with full stacktrace, even though these are normal operational
conditions (e.g., broker down, network turbulence).

The classic KafkaConsumer logs such cases at WARN, and the Jira ticket
KAFKA-19891 confirms that the async consumer should behave
consistently.

This PR:

  • Logs TimeoutException and DisconnectException at WARN (without
    stacktrace)
  • Logs all other exceptions at ERROR (with stacktrace)

No functional behaviour has been changed.

@github-actions github-actions bot added triage PRs from the community consumer clients small Small PRs labels Nov 22, 2025
Copy link
Contributor

@kirktrue kirktrue left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this a fix for KAFKA-19891 or no? If it is, the PR subject line should start with "KAFKA-19891: AsyncKafkaConsumer..."

In the case described in the bug, the log message is emitted on close(). One option is to update AbstractHeartbeatRequestManager to implement the signalClose() API (part of the RequestManager interface) to set a flag when it's closing. Then that flag can be checked before logging, e.g.:

             });
     }
 
+    private boolean isClosed;
+
+    @Override
+    public void signalClose() {
+        isClosed = true;
+    }
+
     @SuppressWarnings("unchecked")
     private NetworkClientDelegate.UnsentRequest logResponse(final NetworkClientDelegate.UnsentRequest request) {
         return request.whenComplete((response, exception) -> {
@@ -315,7 +322,10 @@ public abstract class AbstractHeartbeatRequestManager<R extends AbstractResponse
                 else
                     logger.error("{} failed because of {}: {}", heartbeatRequestName(), error, response);
             } else {
-                logger.error("{} failed because of unexpected exception.", heartbeatRequestName(), exception);
+                if (isClosed)
+                    logger.debug("{} failed because of exception during close.", heartbeatRequestName(), exception);
+                else
+                    logger.error("{} failed because of unexpected exception.", heartbeatRequestName(), exception);
             }
         });
     }

@AntonVasant
Copy link
Contributor Author

sure will make the changes

@AntonVasant AntonVasant changed the title MINOR: Improve heartbeat error logging by downgrading transient Timeo… KAFKA-19891: Suppress error-level heartbeat logs during consumer close Nov 25, 2025
@github-actions github-actions bot removed the triage PRs from the community label Nov 25, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants