KAFKA-19572: Added check to prevent NPE logs during ShareConsumer::close#20290
Conversation
| swallow(log, Level.ERROR, "Failed to stop finding coordinator", | ||
| this::stopFindCoordinatorOnClose, firstException); |
There was a problem hiding this comment.
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.
There was a problem hiding this comment.
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)
There was a problem hiding this comment.
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
left a comment
There was a problem hiding this comment.
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)
| swallow(log, Level.ERROR, "Failed to stop finding coordinator", | ||
| this::stopFindCoordinatorOnClose, firstException); |
There was a problem hiding this comment.
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)
…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>
…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>
…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>
What
https://issues.apache.org/jira/browse/KAFKA-19572
If a
ShareConsumerconstructor failed due to any exception, then wecall
close()in the catch block.If there were uninitialized members accessed during
close(), then itwould 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 weaccess fields which possibly could be null.
Reviewers: Apoorv Mittal amittal@confluent.io, Lianet Magrans
lmagrans@confluent.io