Skip to content
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,136 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.streams.integration;

import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.serialization.Serdes;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.TopologyWrapper;
import org.apache.kafka.streams.errors.ProcessorStateException;
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
import org.apache.kafka.streams.processor.StateStore;
import org.apache.kafka.streams.processor.StateStoreContext;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.internals.AbstractStoreBuilder;
import org.apache.kafka.test.MockApiProcessorSupplier;
import org.apache.kafka.test.MockKeyValueStore;
import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;

import java.io.IOException;
import java.time.Duration;
import java.util.Properties;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;

import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName;
import static org.junit.jupiter.api.Assertions.assertTrue;

public class StateUpdaterFailureIntegrationTest {

private static final int NUM_BROKERS = 1;
protected static final String INPUT_TOPIC_NAME = "input-topic";
private static final int NUM_PARTITIONS = 6;

private final EmbeddedKafkaCluster cluster = new EmbeddedKafkaCluster(NUM_BROKERS);

private Properties streamsConfiguration;
private final MockTime mockTime = cluster.time;
private KafkaStreams streams;

@BeforeEach
public void before(final TestInfo testInfo) throws InterruptedException, IOException {
cluster.start();
cluster.createTopic(INPUT_TOPIC_NAME, NUM_PARTITIONS, 1);
streamsConfiguration = new Properties();
final String safeTestName = safeUniqueTestName(testInfo);
streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName);
streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers());
streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath());
streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0);
streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L);
streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class);
streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class);
streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2);

}

@AfterEach
public void after() {
cluster.stop();
if (streams != null) {
streams.close(Duration.ofSeconds(30));
}
}

@Test
public void correctlyHandleFlushErrorsDuringRebalance() throws InterruptedException {
Copy link
Member

Choose a reason for hiding this comment

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

This test is somewhat complex. Might be good to add a longer comment to explain what it does.

If I read it correctly, we start up two thread, and each thread get 3 tasks each (and we init 6 stores). We stop on thread, re-trigger the init of 3 stores, and let it crash when we hit "store init no.9".

final AtomicInteger numberOfStoreInits = new AtomicInteger();
final AtomicReference<KafkaStreams.State> currentState = new AtomicReference<>();

final StoreBuilder<KeyValueStore<Object, Object>> storeBuilder = new AbstractStoreBuilder<>("testStateStore", Serdes.Integer(), Serdes.ByteArray(), new MockTime()) {

@Override
public KeyValueStore<Object, Object> build() {
return new MockKeyValueStore(name, false) {

@Override
public void init(final StateStoreContext stateStoreContext, final StateStore root) {
super.init(stateStoreContext, root);
numberOfStoreInits.incrementAndGet();
}

@Override
public void flush() {
if (numberOfStoreInits.get() == NUM_PARTITIONS * 1.5) {
try {
TestUtils.waitForCondition(() -> currentState.get() == KafkaStreams.State.PENDING_SHUTDOWN, "Streams never reached PENDING_SHUTDOWN state");
Copy link
Member

@mjsax mjsax Nov 6, 2025

Choose a reason for hiding this comment

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

Not sure if I understand this condition. Can you elaborate?

Also: if we fail here, how do we ensure that the test fails? This would be executed on the background StreamsThread not the actual "main" thread running the test.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

to reproduce the issue, we need to fail during a shutdown.

so what the test does:

  • it creates a KS app with 2 threads.
  • then kills one thread which causes a rebalance
  • then the rebalance hangs here, because we can't flush a task.
  • we call streams close
  • this call gets unblocked and throws an exception
  • that's where we reproduce the issue

in other words: the problem I am trying to fix here reproduces if: we are in rebalance, we have some unhandled tasks in the task updater which can throw an exception, which can kill the thread, and we are trying to close the app.
in this case the stream thread will not do its thing, which in turn will not call TaskManager, which will not call stateUpdater.drainExceptionsAndFailedTasks, which will not remove all failed tasks from the lists. which in turn will cause stateUpdater.tasks to return the failed tasks. and TaskManager will try to delete them.
but because the thread is dead, the delete will never succeed, as the dead thread will never complete the future

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I will add a comment to the test to explain what exactly we are trying to achieve here

Copy link
Member

Choose a reason for hiding this comment

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

I understand that we do want to throw here to trigger an error on the state-updater thread. But it's not clear to me, why we would need to block/wait here until we go to "pending shutdown"?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

what needs to happen to reproduce the issue:
the stream thread is already down(so, we want this condition: https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java#L933 to be false), we also want to have a task in the state updater. and this task needs to throw an exception.

maybe there is an easier way to achieve these conditions, but that's what I was able to come up with

Copy link
Member

Choose a reason for hiding this comment

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

the stream thread is already down

You mean before the state-updater crashes? Interesting. Not clear to me why though? Can you explain further?

Copy link
Contributor Author

@Nikita-Shupletsov Nikita-Shupletsov Nov 8, 2025

Choose a reason for hiding this comment

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

before the state-updater crashes

yes. when the stream thread is running it calls StreamThread#checkStateUpdater, which, in turn, calls TaskManager, which calls StateUpdater#drainExceptionsAndFailedTasks.
so it will delete the failed task from the list of tasks(hence StateUpdater#tasks will not return them anymore) and trigger shutdown.
and when we get to the shutdown, we will not have anything in the state updater, so we will not try to remove anything from in the shutdownStateUpdater method, hence we will not get stuck.

it technically can be slightly different: we have some tasks in the state updater, one of them fails and kills the thread, which will be picked up by the stream thread. but if after that we some how add more tasks to the state updater, we will get the same issue.
But I couldn't figure out if it's even possible to add new tasks to the state updater when the thread is shitting down, as it happens only during rebalance(but here I may be wrong)

} catch (final InterruptedException e) {
throw new RuntimeException(e);
}
throw new ProcessorStateException("flush");
}
}
};
}
};

final TopologyWrapper topology = new TopologyWrapper();
topology.addSource("ingest", INPUT_TOPIC_NAME);
topology.addProcessor("my-processor", new MockApiProcessorSupplier<>(), "ingest");
topology.addStateStore(storeBuilder, "my-processor");

streams = new KafkaStreams(topology, streamsConfiguration);
streams.setStateListener((newState, oldState) -> currentState.set(newState));
streams.start();

TestUtils.waitForCondition(() -> currentState.get() == KafkaStreams.State.RUNNING, "Streams never reached RUNNING state");

streams.removeStreamThread();

TestUtils.waitForCondition(() -> numberOfStoreInits.get() == NUM_PARTITIONS * 1.5, "Streams never reinitialized the store enough times");

assertTrue(streams.close(Duration.ofSeconds(60)));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -208,11 +208,7 @@ private void performActionsOnTasks() {
addTask(taskAndAction.task());
break;
case REMOVE:
if (taskAndAction.futureForRemove() == null) {
removeTask(taskAndAction.taskId());
} else {
removeTask(taskAndAction.taskId(), taskAndAction.futureForRemove());
}
removeTask(taskAndAction.taskId(), taskAndAction.futureForRemove());
break;
default:
throw new IllegalStateException("Unknown action type " + action);
Expand Down Expand Up @@ -349,23 +345,26 @@ private void handleTaskCorruptedException(final TaskCorruptedException taskCorru
// TODO: we can let the exception encode the actual corrupted changelog partitions and only
// mark those instead of marking all changelogs
private void removeCheckpointForCorruptedTask(final Task task) {
task.markChangelogAsCorrupted(task.changelogPartitions());
try {
task.markChangelogAsCorrupted(task.changelogPartitions());

// we need to enforce a checkpoint that removes the corrupted partitions
measureCheckpointLatency(() -> task.maybeCheckpoint(true));
// we need to enforce a checkpoint that removes the corrupted partitions
measureCheckpointLatency(() -> task.maybeCheckpoint(true));
} catch (final StreamsException e) {
Copy link
Member

@mjsax mjsax Nov 6, 2025

Choose a reason for hiding this comment

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

In general, we treat StreamsException as fatal, and should never swallow it. What is the reason for this change?

Also: maybe rename e -> swallow to make clear that we swallow intentionally, and not by mistake.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

because it's for a corrupted task. corrupted tasks already have an exception, so to not overwrite it, I just log it here.
renaming e to swallow is a good idea, will do

log.warn("Checkpoint failed for corrupted task {}", task.id(), e);
}
}

private void handleStreamsException(final StreamsException streamsException) {
log.info("Encountered streams exception: ", streamsException);
if (streamsException.taskId().isPresent()) {
handleStreamsExceptionWithTask(streamsException);
handleStreamsExceptionWithTask(streamsException, streamsException.taskId().get());
} else {
handleStreamsExceptionWithoutTask(streamsException);
}
}

private void handleStreamsExceptionWithTask(final StreamsException streamsException) {
final TaskId failedTaskId = streamsException.taskId().get();
private void handleStreamsExceptionWithTask(final StreamsException streamsException, final TaskId failedTaskId) {
if (updatingTasks.containsKey(failedTaskId)) {
addToExceptionsAndFailedTasksThenRemoveFromUpdatingTasks(
new ExceptionAndTask(streamsException, updatingTasks.get(failedTaskId))
Expand Down Expand Up @@ -518,7 +517,7 @@ private void removeTask(final TaskId taskId, final CompletableFuture<RemovedTask
+ " own this task.", taskId);
}
} catch (final StreamsException streamsException) {
handleStreamsException(streamsException);
handleStreamsExceptionWithTask(streamsException, taskId);
future.completeExceptionally(streamsException);
} catch (final RuntimeException runtimeException) {
handleRuntimeException(runtimeException);
Expand Down Expand Up @@ -607,44 +606,22 @@ private boolean removeFailedTask(final TaskId taskId, final CompletableFuture<Re
}
}

private void removeTask(final TaskId taskId) {
final Task task;
if (updatingTasks.containsKey(taskId)) {
task = updatingTasks.get(taskId);
private void pauseTask(final Task task) {
final TaskId taskId = task.id();
// do not need to unregister changelog partitions for paused tasks
try {
measureCheckpointLatency(() -> task.maybeCheckpoint(true));
final Collection<TopicPartition> changelogPartitions = task.changelogPartitions();
changelogReader.unregister(changelogPartitions);
removedTasks.add(task);
pausedTasks.put(taskId, task);
updatingTasks.remove(taskId);
if (task.isActive()) {
transitToUpdateStandbysIfOnlyStandbysLeft();
}
log.info((task.isActive() ? "Active" : "Standby")
+ " task " + task.id() + " was removed from the updating tasks and added to the removed tasks.");
} else if (pausedTasks.containsKey(taskId)) {
task = pausedTasks.get(taskId);
final Collection<TopicPartition> changelogPartitions = task.changelogPartitions();
changelogReader.unregister(changelogPartitions);
removedTasks.add(task);
pausedTasks.remove(taskId);
log.info((task.isActive() ? "Active" : "Standby")
+ " task " + task.id() + " was removed from the paused tasks and added to the removed tasks.");
} else {
log.info("Task " + taskId + " was not removed since it is not updating or paused.");
}
}
+ " task " + task.id() + " was paused from the updating tasks and added to the paused tasks.");

private void pauseTask(final Task task) {
final TaskId taskId = task.id();
// do not need to unregister changelog partitions for paused tasks
measureCheckpointLatency(() -> task.maybeCheckpoint(true));
pausedTasks.put(taskId, task);
updatingTasks.remove(taskId);
if (task.isActive()) {
transitToUpdateStandbysIfOnlyStandbysLeft();
} catch (final StreamsException streamsException) {
Copy link
Member

Choose a reason for hiding this comment

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

It seems we are adding a similar try-catch on multiple places. Should we instead add a new catch block to run() method to have it in a single place (and also avoid that this PR might miss some place where it should get added otherwise, too)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

the idea was to handle one task at a time instead of failing all of them together. so if we have 3 tasks, one fails but the other 2 succeed, we will have only one failed task instead of all 3.
similarly to https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java#L363 and https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/processor/internals/DefaultStateUpdater.java#L516

Copy link
Member

Choose a reason for hiding this comment

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

I see. Hard to read on the PR, as there is no loop here...

But I am not even sure why we would want to fail a task if we cannot checkpoint for this case? We only pause a task, and it seems to be non-fatal if we fail to checkpoint? Wondering if we should swallow the exception entirely and just keep the task w/o the need to move it into error state and hand back to StreamsThreads?

\cc @lucasbru

handleStreamsExceptionWithTask(streamsException, taskId);
}
log.info((task.isActive() ? "Active" : "Standby")
+ " task " + task.id() + " was paused from the updating tasks and added to the paused tasks.");
}

private void resumeTask(final Task task) {
Expand All @@ -671,11 +648,15 @@ private void maybeCompleteRestoration(final StreamTask task,
final Set<TopicPartition> restoredChangelogs) {
final Collection<TopicPartition> changelogPartitions = task.changelogPartitions();
if (restoredChangelogs.containsAll(changelogPartitions)) {
measureCheckpointLatency(() -> task.maybeCheckpoint(true));
changelogReader.unregister(changelogPartitions);
addToRestoredTasks(task);
log.info("Stateful active task " + task.id() + " completed restoration");
transitToUpdateStandbysIfOnlyStandbysLeft();
try {
measureCheckpointLatency(() -> task.maybeCheckpoint(true));
changelogReader.unregister(changelogPartitions);
addToRestoredTasks(task);
log.info("Stateful active task " + task.id() + " completed restoration");
transitToUpdateStandbysIfOnlyStandbysLeft();
} catch (final StreamsException streamsException) {
handleStreamsExceptionWithTask(streamsException, task.id());
}
}
}

Expand Down Expand Up @@ -707,8 +688,12 @@ private void maybeCheckpointTasks(final long now) {

measureCheckpointLatency(() -> {
for (final Task task : updatingTasks.values()) {
// do not enforce checkpointing during restoration if its position has not advanced much
task.maybeCheckpoint(false);
try {
// do not enforce checkpointing during restoration if its position has not advanced much
task.maybeCheckpoint(false);
} catch (final StreamsException streamsException) {
handleStreamsExceptionWithTask(streamsException, task.id());
}
}
});

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,11 +55,6 @@ public static TaskAndAction createRemoveTask(final TaskId taskId,
return new TaskAndAction(null, taskId, Action.REMOVE, future);
}

public static TaskAndAction createRemoveTask(final TaskId taskId) {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

the non future remove task isn't used anywhere anymore, so I deleted the code that handled it.
also it made the proposed solution a bit tidier

Objects.requireNonNull(taskId, "Task ID of task to remove is null!");
return new TaskAndAction(null, taskId, Action.REMOVE, null);
}

public Task task() {
if (action != Action.ADD) {
throw new IllegalStateException("Action type " + action + " cannot have a task!");
Expand All @@ -84,4 +79,4 @@ public CompletableFuture<StateUpdater.RemovedTaskResult> futureForRemove() {
public Action action() {
return action;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@
import java.util.TreeSet;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import java.util.stream.Stream;
Expand Down Expand Up @@ -772,7 +773,7 @@ private StateUpdater.RemovedTaskResult waitForFuture(final TaskId taskId,
final CompletableFuture<StateUpdater.RemovedTaskResult> future) {
final StateUpdater.RemovedTaskResult removedTaskResult;
try {
removedTaskResult = future.get();
removedTaskResult = future.get(1, TimeUnit.MINUTES);
if (removedTaskResult == null) {
throw new IllegalStateException("Task " + taskId + " was not found in the state updater. "
+ BUG_ERROR_MESSAGE);
Expand All @@ -787,6 +788,10 @@ private StateUpdater.RemovedTaskResult waitForFuture(final TaskId taskId,
Thread.currentThread().interrupt();
log.error(INTERRUPTED_ERROR_MESSAGE, shouldNotHappen);
throw new IllegalStateException(INTERRUPTED_ERROR_MESSAGE, shouldNotHappen);
} catch (final java.util.concurrent.TimeoutException timeoutException) {
log.warn("The state updater wasn't able to remove task {} in time. The state updater thread may be dead. "
+ BUG_ERROR_MESSAGE, taskId, timeoutException);
return null;
}
}

Expand Down Expand Up @@ -1567,6 +1572,12 @@ void shutdown(final boolean clean) {

private void shutdownStateUpdater() {
if (stateUpdater != null) {
// If there are failed tasks handling them first
for (final StateUpdater.ExceptionAndTask exceptionAndTask : stateUpdater.drainExceptionsAndFailedTasks()) {
final Task failedTask = exceptionAndTask.task();
closeTaskDirty(failedTask, false);
}

Copy link
Member

Choose a reason for hiding this comment

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

Below, there is stateUpdater.shutdown(Duration.ofMillis(Long.MAX_VALUE)) -- should we set this to some smaller value, too?

Copy link
Member

Choose a reason for hiding this comment

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

I also just found (inside TaskManager.java):

    private void shutdownSchedulingTaskManager() {
        if (schedulingTaskManager != null) {
            schedulingTaskManager.shutdown(Duration.ofMillis(Long.MAX_VALUE));
        }
    }

This does also "smell"...

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I was thinking about that, but then came across this change: #17018 and decided to leave it as is.
if you think it's still worth it, I can do that

Copy link
Member

Choose a reason for hiding this comment

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

Seem the other PR wants to make use of the timeout. But if we pass in MAX_VALUE this does not make much sense? As it effectively blocks forever...

If we really want to block forever when joining the thread (ie, taskExecutorThread.join(timeout.toMillis());) we should just use MAX_VALUE there, and make schedulingTaskManager.shutdown() parameterless. But not sure if we would want this?

This is all very confusing to me.

\cc @lucasbru can you chime in?

final Map<TaskId, CompletableFuture<StateUpdater.RemovedTaskResult>> futures = new LinkedHashMap<>();
for (final Task task : stateUpdater.tasks()) {
final CompletableFuture<StateUpdater.RemovedTaskResult> future = stateUpdater.remove(task.id());
Expand All @@ -1583,10 +1594,16 @@ private void shutdownStateUpdater() {
for (final Task task : tasksToCloseDirty) {
closeTaskDirty(task, false);
}
// Handling all failures that occurred during the remove process
for (final StateUpdater.ExceptionAndTask exceptionAndTask : stateUpdater.drainExceptionsAndFailedTasks()) {
final Task failedTask = exceptionAndTask.task();
closeTaskDirty(failedTask, false);
}

// If there is anything left unhandled due to timeouts, handling now
for (final Task task : stateUpdater.tasks()) {
closeTaskDirty(task, false);
}
}
}

Expand Down
Loading