Skip to content

KAFKA-19572: Added check to prevent NPE logs during ShareConsumer::close#20290

Merged
lianetm merged 4 commits into
apache:trunkfrom
ShivsundarR:KAFKA-19572
Aug 7, 2025
Merged

KAFKA-19572: Added check to prevent NPE logs during ShareConsumer::close#20290
lianetm merged 4 commits into
apache:trunkfrom
ShivsundarR:KAFKA-19572

Conversation

@ShivsundarR

@ShivsundarR ShivsundarR commented Aug 1, 2025

Copy link
Copy Markdown
Contributor

What
https://issues.apache.org/jira/browse/KAFKA-19572

  • If a ShareConsumer constructor failed due to any exception, then we
    call close() in the catch block.

  • If there were uninitialized members accessed during close(), then it
    would throw a NPE. Currently there are no null checks, hence we were
    attempting to use these fields during close() execution.

  • To avoid this, PR adds null checks in the close() function before we
    access fields which possibly could be null.

Reviewers: Apoorv Mittal amittal@confluent.io, Lianet Magrans
lmagrans@confluent.io

@github-actions github-actions Bot added triage PRs from the community consumer clients small Small PRs labels Aug 1, 2025
@ShivsundarR ShivsundarR added KIP-932 Queues for Kafka ci-approved and removed small Small PRs triage PRs from the community labels Aug 1, 2025
@ShivsundarR ShivsundarR requested a review from lianetm August 4, 2025 11:52

@apoorvmittal10 apoorvmittal10 left a comment

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Minor comment, else looks good. If someone from clients can also review then it will be helpful, @kirktrue @lianetm.

Comment on lines +901 to +902
swallow(log, Level.ERROR, "Failed to stop finding coordinator",
this::stopFindCoordinatorOnClose, firstException);

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This close is only dependent on applicationEventHandler hence do not require all 3 to be non-null i.e. applicationEventHandler != null && backgroundEventReaper != null && backgroundEventQueue != null. Please simplify for stopFindCoordinatorOnClose.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

agree, also the handleCompletedAcknowledgements seems to only need the background components?

Wonder if it would be clearer to do the null checks inside each func (where we can easily see the component they need, and will avoid facing this same issue again if they end up being reused)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Thanks @apoorvmittal10 and @lianetm for the review.
I have added the null checks inside each function now to make it clearer and ensure we do not face this issue again if we re-use the function later.

@lianetm lianetm left a comment

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Thanks @ShivsundarR ! Just minor comment (and I created https://issues.apache.org/jira/browse/KAFKA-19585 to review this same situation in the async consumer)

Comment on lines +901 to +902
swallow(log, Level.ERROR, "Failed to stop finding coordinator",
this::stopFindCoordinatorOnClose, firstException);

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

agree, also the handleCompletedAcknowledgements seems to only need the background components?

Wonder if it would be clearer to do the null checks inside each func (where we can easily see the component they need, and will avoid facing this same issue again if they end up being reused)

@github-actions github-actions Bot added the small Small PRs label Aug 7, 2025

@lianetm lianetm left a comment

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Thanks! LGTM.

@lianetm lianetm merged commit bf42924 into apache:trunk Aug 7, 2025
32 of 34 checks passed
lianetm pushed a commit that referenced this pull request Sep 8, 2025
…ctor failures (#20491)

If there's a failure in the kafka consumer constructor, we attempt to
close it

https://github.com/lianetm/kafka/blob/2329def2ff9ca4f7b9426af159b6fa19a839dc4d/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java#L540
In that case, it could be the case that some components may have not
been created, so we should consider some null checks to avoid noisy logs
about NPE.

This noisy logs have been reported with the console share consumer in a
similar scenario, so this task is to review and do a similar fix for the
Async if needed.

The fix is to check if handlers/invokers are null before trying to close
them. Similar to what was done here
#20290

Reviewers: TengYao Chi <frankvicky@apache.org>, Lianet Magrans
 <lmagrans@confluent.io>
eduwercamacaro pushed a commit to littlehorse-enterprises/kafka that referenced this pull request Nov 12, 2025
…ctor failures (apache#20491)

If there's a failure in the kafka consumer constructor, we attempt to
close it

https://github.com/lianetm/kafka/blob/2329def2ff9ca4f7b9426af159b6fa19a839dc4d/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java#L540
In that case, it could be the case that some components may have not
been created, so we should consider some null checks to avoid noisy logs
about NPE.

This noisy logs have been reported with the console share consumer in a
similar scenario, so this task is to review and do a similar fix for the
Async if needed.

The fix is to check if handlers/invokers are null before trying to close
them. Similar to what was done here
apache#20290

Reviewers: TengYao Chi <frankvicky@apache.org>, Lianet Magrans
 <lmagrans@confluent.io>
shashankhs11 pushed a commit to shashankhs11/kafka that referenced this pull request Dec 15, 2025
…ctor failures (apache#20491)

If there's a failure in the kafka consumer constructor, we attempt to
close it

https://github.com/lianetm/kafka/blob/2329def2ff9ca4f7b9426af159b6fa19a839dc4d/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java#L540
In that case, it could be the case that some components may have not
been created, so we should consider some null checks to avoid noisy logs
about NPE.

This noisy logs have been reported with the console share consumer in a
similar scenario, so this task is to review and do a similar fix for the
Async if needed.

The fix is to check if handlers/invokers are null before trying to close
them. Similar to what was done here
apache#20290

Reviewers: TengYao Chi <frankvicky@apache.org>, Lianet Magrans
 <lmagrans@confluent.io>
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