KAFKA-8412: Still a nullpointer exception thrown on shutdown whi…#7207
KAFKA-8412: Still a nullpointer exception thrown on shutdown whi…#7207guozhangwang merged 3 commits intoapache:trunkfrom
Conversation
…le flushing before closing producers
Prior to this change an NPE is raised when calling AssignedTasks.close
under the following conditions:
1. EOS is enabled
2. The task was in a suspended state
The cause for the NPE is that when a clean close is requested for a
StreamTask the StreamTask tries to commit. However, in the suspended
state there is no producer so ultimately an NPE is thrown for the
contained RecordCollector in flush.
It is my opinion that in the long term, this (and probably other
surprising state interactions) could be cleaned up by consolidating
state into one place instead of spreading it across AssignedTasks,
StreamTask, and AbstractTask. However, that is a much larger, more risky
change, and this issue is currently considered minor.
The fix put forth in this commit is to have AssignedTasks call
closeSuspended when it knows the underlying StreamTask is suspended.
Currently the only externally visible way to detect this problem in test
seems to be via logging. This is because the NPE is logged but then
suppressed under the following sequence:
RecordCollectorImpl.flush:266
- throws NPE (producer is null)
StreamTask.suspend:578
- goes through the finally block and then reraises the NPE
StreamTask.close:706
- catches the NPE, calls closeSuspended with the NPE
StreamTask.closeSuspended:676
- rethrows the NPE after some cleanup
AssignedTasks.close:341
- catches and logs the exception
- tries a "dirty" close (clean = true) which succeeds
- firstException is NOT set because the test `!closeUnclean(task)`
does not hold.
It seems this is not the intended behavior? If so, I will happily
correct that and stop using logging as a way to detect failure.
Otherwise this commit does not currently pass checkstyle because I'm
using blacklisted imports: `LogCaptureAppender` and its various
dependencies from `log4j`. I would appreciate guidance as to whether we
should whitelist these or use another technique for detection.
Note also that this test is quite involved. I could have just tested
that AssignedTasks calls closeSuspended when appropriate, but that is
testing, IMO, a detail of the implementation and doesn't actually verify
we reproduced the original problem as it was described. I feel much more
confident that we are reproducing the behavior - and we can test exactly
the conditions that lead to it - when testing across AssignedTasks and
StreamTask. I believe this is an additional support for the argument of
eventually consolidating the state split across classes.
|
@mjsax @guozhangwang Please see patch notes, particularly about how the NPE is suppressed and thus difficult to test for. I offered a few ideas, but would appreciate your guidance. |
|
bump |
| final AtomicReference<RuntimeException> firstException = new AtomicReference<>(null); | ||
| for (final T task : allTasks()) { | ||
| try { | ||
| task.close(clean, false); |
There was a problem hiding this comment.
Why not leaf the code "as-is" and change this line to:
if (suspended.values().contains(task)) {
task.closeSuspended(clean, false, firstException);
} else {
task.close(clean, false);
}
There was a problem hiding this comment.
Absolutely! Will do.
|
I think the overall approach is correct. Side notice: should we revert #5993 that seems not to be necessary after this change any longer? |
|
I guess you can use |
|
I inlined closeTask per your suggestion - it's definitely an improvement. For now I whitelisted |
guozhangwang
left a comment
There was a problem hiding this comment.
LGTM. Triggering one system test before merging.
|
System test passed, merging to trunk. |
…ng producers (#7207) Prior to this change an NPE is raised when calling AssignedTasks.close under the following conditions: 1. EOS is enabled 2. The task was in a suspended state The cause for the NPE is that when a clean close is requested for a StreamTask the StreamTask tries to commit. However, in the suspended state there is no producer so ultimately an NPE is thrown for the contained RecordCollector in flush. The fix put forth in this commit is to have AssignedTasks call closeSuspended when it knows the underlying StreamTask is suspended. Note also that this test is quite involved. I could have just tested that AssignedTasks calls closeSuspended when appropriate, but that is testing, IMO, a detail of the implementation and doesn't actually verify we reproduced the original problem as it was described. I feel much more confident that we are reproducing the behavior - and we can test exactly the conditions that lead to it - when testing across AssignedTasks and StreamTask. I believe this is an additional support for the argument of eventually consolidating the state split across classes. Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
…ng producers (#7207) Prior to this change an NPE is raised when calling AssignedTasks.close under the following conditions: 1. EOS is enabled 2. The task was in a suspended state The cause for the NPE is that when a clean close is requested for a StreamTask the StreamTask tries to commit. However, in the suspended state there is no producer so ultimately an NPE is thrown for the contained RecordCollector in flush. The fix put forth in this commit is to have AssignedTasks call closeSuspended when it knows the underlying StreamTask is suspended. Note also that this test is quite involved. I could have just tested that AssignedTasks calls closeSuspended when appropriate, but that is testing, IMO, a detail of the implementation and doesn't actually verify we reproduced the original problem as it was described. I feel much more confident that we are reproducing the behavior - and we can test exactly the conditions that lead to it - when testing across AssignedTasks and StreamTask. I believe this is an additional support for the argument of eventually consolidating the state split across classes. Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
…ng producers (#7207) Prior to this change an NPE is raised when calling AssignedTasks.close under the following conditions: 1. EOS is enabled 2. The task was in a suspended state The cause for the NPE is that when a clean close is requested for a StreamTask the StreamTask tries to commit. However, in the suspended state there is no producer so ultimately an NPE is thrown for the contained RecordCollector in flush. The fix put forth in this commit is to have AssignedTasks call closeSuspended when it knows the underlying StreamTask is suspended. Note also that this test is quite involved. I could have just tested that AssignedTasks calls closeSuspended when appropriate, but that is testing, IMO, a detail of the implementation and doesn't actually verify we reproduced the original problem as it was described. I feel much more confident that we are reproducing the behavior - and we can test exactly the conditions that lead to it - when testing across AssignedTasks and StreamTask. I believe this is an additional support for the argument of eventually consolidating the state split across classes. Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
|
Cherry-picked to 2.3, 2.2, and 2.1 as well. Thanks @cpettitt-confluent for your contribution. |
…yWithSuspendedTaskAndEOS The previous approach to testing KAFKA-8412 was to look at the logs and determine if an error occurred during close. There was no direct way to detect than an exception occurred because the exception was eaten in `AssignedTasks.close`. In the PR for that ticket (apache#7207) it was acknowledged that this was a brittle way to test for the exception. We now see occasional failures because an unrelated ERROR level log entry is made while closing the task. This change eliminates the brittle log checking by rethrowing any time an exception occurs in close, even when a subsequent unclean close succeeds. This has the potential benefit of uncovering other supressed exceptions down the road. I've verified that even with us rethrowing on `closeUnclean` that all tests pass.
…yWithSuspendedTaskAndEOS (#7302) The previous approach to testing KAFKA-8412 was to look at the logs and determine if an error occurred during close. There was no direct way to detect than an exception occurred because the exception was eaten in AssignedTasks.close. In the PR for that ticket (#7207) it was acknowledged that this was a brittle way to test for the exception. We now see occasional failures because an unrelated ERROR level log entry is made while closing the task. This change eliminates the brittle log checking by rethrowing any time an exception occurs in close, even when a subsequent unclean close succeeds. This has the potential benefit of uncovering other supressed exceptions down the road. I've verified that even with us rethrowing on closeUnclean that all tests pass. Reviewers: Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bbejeck@gmail.com>
…le flushing before closing producers
Prior to this change an NPE is raised when calling AssignedTasks.close
under the following conditions:
The cause for the NPE is that when a clean close is requested for a
StreamTask the StreamTask tries to commit. However, in the suspended
state there is no producer so ultimately an NPE is thrown for the
contained RecordCollector in flush.
It is my opinion that in the long term, this (and probably other
surprising state interactions) could be cleaned up by consolidating
state into one place instead of spreading it across AssignedTasks,
StreamTask, and AbstractTask. However, that is a much larger, more risky
change, and this issue is currently considered minor.
The fix put forth in this commit is to have AssignedTasks call
closeSuspended when it knows the underlying StreamTask is suspended.
Currently the only externally visible way to detect this problem in test
seems to be via logging. This is because the NPE is logged but then
suppressed under the following sequence:
RecordCollectorImpl.flush:266
- throws NPE (producer is null)
StreamTask.suspend:578
- goes through the finally block and then reraises the NPE
StreamTask.close:706
- catches the NPE, calls closeSuspended with the NPE
StreamTask.closeSuspended:676
- rethrows the NPE after some cleanup
AssignedTasks.close:341
- catches and logs the exception
- tries a "dirty" close (clean = true) which succeeds
- firstException is NOT set because the test
!closeUnclean(task)does not hold.
It seems this is not the intended behavior? If so, I will happily
correct that and stop using logging as a way to detect failure.
Otherwise this commit does not currently pass checkstyle because I'm
using blacklisted imports:
LogCaptureAppenderand its variousdependencies from
log4j. I would appreciate guidance as to whether weshould whitelist these or use another technique for detection.
Note also that this test is quite involved. I could have just tested
that AssignedTasks calls closeSuspended when appropriate, but that is
testing, IMO, a detail of the implementation and doesn't actually verify
we reproduced the original problem as it was described. I feel much more
confident that we are reproducing the behavior - and we can test exactly
the conditions that lead to it - when testing across AssignedTasks and
StreamTask. I believe this is an additional support for the argument of
eventually consolidating the state split across classes.
More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.
Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.
Committer Checklist (excluded from commit message)