Skip to content

KAFKA-19171: Kafka Streams crashes with UnsupportedOperationException #19507

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 3 commits into
base: trunk
Choose a base branch
from

Conversation

mjsax
Copy link
Member

@mjsax mjsax commented Apr 17, 2025

This PR fixes a regression bug introduced with KAFKA-17203. We need to
pass in mutable collections into closeTaskClean(...).

This PR fixes a regression bug introduced with KAFKA-17203.
We need to pass in mutable collections into `closeTaskClean(...)`.
@mjsax mjsax added streams small Small PRs labels Apr 17, 2025
@mjsax
Copy link
Member Author

mjsax commented Apr 17, 2025

Need to be cherry-picked to 4.0 branch.

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

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

@mjsax thank you for finding and fixing this.

@@ -553,7 +553,7 @@ private void handleTasksWithStateUpdater(final Map<TaskId, Set<TopicPartition>>
private void handleTasksPendingInitialization() {
// All tasks pending initialization are not part of the usual bookkeeping
for (final Task task : tasks.drainPendingTasksToInit()) {
closeTaskClean(task, Collections.emptySet(), Collections.emptyMap());
closeTaskClean(task, new HashSet<>(), new HashMap<>());
Copy link
Member

Choose a reason for hiding this comment

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

Is it necessary to call closeDirty for those failed tasks?

Copy link
Member Author

@mjsax mjsax Apr 17, 2025

Choose a reason for hiding this comment

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

Good point. I guess yes.

Originally we did not use closetTaskClean() but the PR introducing this change started to use it: #16730

I did not dig into the original PR, but it seems there was some issue with leaking resources in tests? So maybe it's just a test issue?

Copy link
Member Author

@mjsax mjsax Apr 17, 2025

Choose a reason for hiding this comment

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

I am frankly not sure, why #16730 would actually address any resource leak, as the old and new code would just call

            task.suspend();
            task.closeClean();

The difference is just, that the new code would catch and swallow potential exceptions.

So maybe if we throw some exception, other cleanup code did not run leaking the producer? But with the new code, we would swallow the exception ensuring that other cleanup code does run?

Would be good to understand better. Maybe @FrankYang0529 can shed some light?

I am not 100% sure right now, if #16730 is the correct fix for the leak or not.

Copy link
Member Author

Choose a reason for hiding this comment

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

It's actually not even clear to me, how a "pending task to init" could fail to close clean to begin with... \cc @cadonna @lucasbru @ableegoldman and ideas?

Copy link
Member

@FrankYang0529 FrankYang0529 Apr 18, 2025

Choose a reason for hiding this comment

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

The leak resource was found by #14783. On that branch, the leak resource only happened for the case EosIntegrationTest#shouldCheckpointRestoredOffsetsWhenClosingCleanDuringRestoringStateUpdaterEnabled(eosConfig=StreamsConfig.EXACTLY_ONCE, processingThreadsEnabled="false"). I think it's safe to revert the change cause of the EOSv1 was removed in 4.0.

Copy link
Member Author

@mjsax mjsax Apr 18, 2025

Choose a reason for hiding this comment

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

Thanks @FrankYang0529 -- still not sure right now if just reverting is the right thing:

that is a good question ...

This is the thing that worries me -- we did see it crash with EOSv2, so some pending tasks did get revoked and there was an error during revocation. So maybe we did expose some different bug.

Copy link
Member Author

@mjsax mjsax Apr 19, 2025

Choose a reason for hiding this comment

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

I was looking into the logs we collected more plus the code, and it seems possible that we get an error on close with EOSv2.

When we close the unitized tasks, we close the RecordCollector and we check for pending errors: https://github.com/apache/kafka/blob/4.0/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java#L568

Because we shared a single producer across multiple tasks, the initialized task could observe an error from some other task. -- So it seems to be actually safe to swallow this exception when closing a pending task. It's just not totally clear to me, how/where to implement this swallowing correctly? The simples way might be to close pending task dirty, but not sure if that's really the right way to do it (would also result in confusing logging...) -- or we really just try to close clean, and for any task with error close dirty afterwards?

Copy link
Member

@cadonna cadonna Apr 23, 2025

Choose a reason for hiding this comment

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

I believe something like in the following code, will avoid the leak:


Basically, we need to collect the failed tasks in tasksToCloseDirty and close those tasks dirty. The thrown exception is swallowed.

Copy link
Member

Choose a reason for hiding this comment

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

@cadonna 's suggestion makes sense to me

Copy link
Member Author

Choose a reason for hiding this comment

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

Thanks, agree. It was also what @chia7712 original comment was about:

Is it necessary to call closeDirty for those failed tasks?

It was originally just not clear to me, why a uninitilzied task could fail.

@@ -1312,7 +1317,6 @@ private void closeRunningTasksDirty() {
private void removeLostActiveTasksFromStateUpdaterAndPendingTasksToInit() {
if (stateUpdater != null) {
final Map<TaskId, CompletableFuture<StateUpdater.RemovedTaskResult>> futures = new LinkedHashMap<>();
final Map<TaskId, RuntimeException> failedTasksDuringCleanClose = new HashMap<>();
Copy link
Member Author

Choose a reason for hiding this comment

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

Side cleanup -- unused.

closeTaskClean(task, tasksToCloseDirty, new HashMap<>());
}
for (final Task task : tasksToCloseDirty) {
closeTaskDirty(task, false);
Copy link
Member Author

Choose a reason for hiding this comment

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

Not 100% sure if we need to pass true or false here?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants