From ecbe3389032fdd6b6c03440c4f77e1a307608901 Mon Sep 17 00:00:00 2001 From: Nikita Date: Tue, 27 Jan 2026 11:35:14 -0800 Subject: [PATCH 01/16] KAFKA-19960: Close pending tasks on shutdown. Added logic to close pending tasks to init. Made standby task closure similar to the one for active tasks. Added a separate method for getting standby tasks from task registry. Added an integration test that reproduces the issue. --- .../integration/KafkaStreamsWrapper.java | 6 +- .../RebalanceTaskClosureIntegrationTest.java | 213 ++++++++++++++++++ .../processor/internals/TaskManager.java | 15 +- .../streams/processor/internals/Tasks.java | 11 +- .../processor/internals/TasksRegistry.java | 2 + .../processor/internals/TaskManagerTest.java | 34 ++- .../processor/internals/TasksTest.java | 16 ++ 7 files changed, 286 insertions(+), 11 deletions(-) create mode 100644 streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java index 3178377656078..7f7bea6964455 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java @@ -48,7 +48,11 @@ public List streamThreads() { public void setStreamThreadStateListener(final StreamThread.StateListener listener) { if (state == State.CREATED) { for (final StreamThread thread : threads) { - thread.setStateListener(listener); + StreamThread.StateListener originalListener = thread.getStateListener(); + thread.setStateListener((t, newState, oldState) -> { + originalListener.onChange(t, newState, oldState); + listener.onChange(t, newState, oldState); + }); } } else { throw new IllegalStateException("Can only set StateListener in CREATED state. " + diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java new file mode 100644 index 0000000000000..efd409d76e16f --- /dev/null +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -0,0 +1,213 @@ +/* + * 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.LongSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.streams.CloseOptions; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.TopologyWrapper; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.StateStoreContext; +import org.apache.kafka.streams.processor.internals.StreamThread; +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.streams.state.internals.CacheFlushListener; +import org.apache.kafka.streams.state.internals.CachedStateStore; +import org.apache.kafka.streams.state.internals.RocksDBStore; +import org.apache.kafka.test.MockApiProcessorSupplier; +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.List; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class RebalanceTaskClosureIntegrationTest { + + private static final int NUM_BROKERS = 1; + protected static final String INPUT_TOPIC_NAME = "input-topic"; + private static final int NUM_PARTITIONS = 3; + + private final EmbeddedKafkaCluster cluster = new EmbeddedKafkaCluster(NUM_BROKERS); + + private KafkaStreamsWrapper streams1; + private KafkaStreamsWrapper streams2; + private String safeTestName; + + @BeforeEach + public void before(final TestInfo testInfo) throws InterruptedException, IOException { + cluster.start(); + cluster.createTopic(INPUT_TOPIC_NAME, NUM_PARTITIONS, 1); + safeTestName = safeUniqueTestName(testInfo); + + } + + @AfterEach + public void after() { + cluster.stop(); + if (streams1 != null) { + streams1.close(Duration.ofSeconds(30)); + } + if (streams2 != null) { + streams2.close(Duration.ofSeconds(30)); + } + } + + @Test + public void shouldClosePendingTasksToInitAfterRebalance() throws Exception { + final CountDownLatch recycleLatch = new CountDownLatch(1); + final CountDownLatch pendingShutdownLatch = new CountDownLatch(1); + // Count how many times we initialize and close stores + final AtomicInteger initCount = new AtomicInteger(); + final AtomicInteger closeCount = new AtomicInteger(); + final StoreBuilder> storeBuilder = new AbstractStoreBuilder<>("testStateStore", Serdes.Integer(), Serdes.ByteArray(), new MockTime()) { + + @Override + public KeyValueStore build() { + return new TestRocksDBStore(name, recycleLatch, pendingShutdownLatch, initCount, closeCount); + } + }; + + final TopologyWrapper topology = new TopologyWrapper(); + topology.addSource("ingest", INPUT_TOPIC_NAME); + topology.addProcessor("my-processor", new MockApiProcessorSupplier<>(), "ingest"); + topology.addStateStore(storeBuilder, "my-processor"); + + streams1 = new KafkaStreamsWrapper(topology, props("1")); + streams1.setStreamThreadStateListener((t, newState, oldState) -> { + if (newState == StreamThread.State.PENDING_SHUTDOWN) { + pendingShutdownLatch.countDown(); + } + }); + streams1.start(); + + TestUtils.waitForCondition(() -> streams1.state() == KafkaStreams.State.RUNNING, "Streams never reached RUNNING state"); + + streams2 = new KafkaStreamsWrapper(topology, props("2")); + streams2.start(); + + TestUtils.waitForCondition(() -> streams2.state() == KafkaStreams.State.RUNNING, "Streams never reached RUNNING state"); + + // starting the second KS app triggered a rebalance. Which in turn will recycle active tasks that need to become standby. + // That's exactly what we are waiting for + recycleLatch.await(); + + // sending a message to disable retries in the consumer client. if there are no messages, it retries the whole sequence of actions, + // including the rebalance data. which we don't want, because we just staged the right condition + IntegrationTestUtils.produceKeyValuesSynchronously(INPUT_TOPIC_NAME, List.of(new KeyValue<>(1L, "key")), + TestUtils.producerConfig(cluster.bootstrapServers(), LongSerializer.class, StringSerializer.class, new Properties()), cluster.time); + // Now we can close both apps. The StreamThreadStateListener will unblock the clearCache call, letting the rebalance finish. + // We don't want it to happen any sooner, because we want the stream thread to stop before it gets to moving messages from task registry to state updater. + streams1.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); + streams2.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); + + assertEquals(initCount.get(), closeCount.get()); + } + + private Properties props(final String storePathSuffix) { + final Properties streamsConfiguration = new Properties(); + + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, safeTestName); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.put(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, 1000); + streamsConfiguration.put(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, 1000); + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath() + "/" + storePathSuffix); + 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.LongSerde.class); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); + streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); + streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); + + return streamsConfiguration; + } + + private static class TestRocksDBStore extends RocksDBStore implements CachedStateStore { + + private final CountDownLatch recycleLatch; + private final CountDownLatch pendingShutdownLatch; + private final AtomicInteger initCount; + private final AtomicInteger closeCount; + + public TestRocksDBStore(final String name, + final CountDownLatch recycleLatch, + final CountDownLatch pendingShutdownLatch, + final AtomicInteger initCount, + final AtomicInteger closeCount) { + super(name, "rocksdb"); + this.recycleLatch = recycleLatch; + this.pendingShutdownLatch = pendingShutdownLatch; + this.initCount = initCount; + this.closeCount = closeCount; + } + + @Override + public void init(final StateStoreContext stateStoreContext, + final StateStore root) { + initCount.incrementAndGet(); + super.init(stateStoreContext, root); + } + + @Override + public boolean setFlushListener(final CacheFlushListener listener, + final boolean sendOldValues) { + return false; + } + + @Override + public void flushCache() { + } + + @Override + public void clearCache() { + // Clear cache is called during recycle, so we use it as a hook + recycleLatch.countDown(); + try { + pendingShutdownLatch.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public synchronized void close() { + closeCount.incrementAndGet(); + super.close(); + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index fdc3256bb087a..05f7c10b2e3ca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -1414,10 +1414,19 @@ void shutdown(final boolean clean) { // TODO: change type to `StreamTask` final Set activeTasks = new TreeSet<>(Comparator.comparing(Task::id)); activeTasks.addAll(tasks.activeTasks()); + final Set standbyTasks = new TreeSet<>(Comparator.comparing(Task::id)); + standbyTasks.addAll(tasks.standbyTasks()); + for (Task pendingTask : tasks.pendingTasksToInit()) { + if (pendingTask.isActive()) { + activeTasks.add(pendingTask); + } else { + standbyTasks.add(pendingTask); + } + } executeAndMaybeSwallow( clean, - () -> closeAndCleanUpTasks(activeTasks, standbyTaskIterable(), clean), + () -> closeAndCleanUpTasks(activeTasks, standbyTasks, clean), e -> firstException.compareAndSet(null, e), e -> log.warn("Ignoring an exception while unlocking remaining task directories.", e) ); @@ -1523,7 +1532,7 @@ private Collection tryCloseCleanActiveTasks(final Collection activeT final boolean clean, final AtomicReference firstException) { if (!clean) { - return activeTaskIterable(); + return activeTasksToClose; } final Comparator byId = Comparator.comparing(Task::id); final Set tasksToCommit = new TreeSet<>(byId); @@ -1616,7 +1625,7 @@ private Collection tryCloseCleanStandbyTasks(final Collection standb final boolean clean, final AtomicReference firstException) { if (!clean) { - return standbyTaskIterable(); + return standbyTasksToClose; } final Set tasksToCloseDirty = new TreeSet<>(Comparator.comparing(Task::id)); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java index f5d007a591590..bfb5ca1286c79 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java @@ -193,7 +193,9 @@ public synchronized void removeTask(final Task taskToRemove) { throw new IllegalStateException("Attempted to remove a task that is not closed or suspended: " + taskId); } - if (taskToRemove.isActive()) { + if (pendingTasksToInit.contains(taskToRemove)) { + pendingTasksToInit.remove(taskToRemove); + } else if (taskToRemove.isActive()) { if (activeTasksPerId.remove(taskId) == null) { throw new IllegalArgumentException("Attempted to remove an active task that is not owned: " + taskId); } @@ -203,7 +205,7 @@ public synchronized void removeTask(final Task taskToRemove) { throw new IllegalArgumentException("Attempted to remove a standby task that is not owned: " + taskId); } } - failedTaskIds.remove(taskToRemove.id()); + failedTaskIds.remove(taskId); } @Override @@ -301,6 +303,11 @@ public synchronized Collection activeTasks() { return Collections.unmodifiableCollection(activeTasksPerId.values()); } + @Override + public synchronized Collection standbyTasks() { + return Collections.unmodifiableCollection(standbyTasksPerId.values()); + } + /** * All tasks returned by any of the getters are read-only and should NOT be modified; * and the returned task could be modified by other threads concurrently diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java index 09c5a79ae0f56..eae21bf219265 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java @@ -71,6 +71,8 @@ public interface TasksRegistry { Collection activeTasks(); + Collection standbyTasks(); + Set allTasks(); Set allNonFailedTasks(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 798d2d3406b14..11c2b294bf569 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -312,12 +312,8 @@ public void shouldLockTasksOnClose() { final StreamTask activeTask1 = statefulTask(taskId00, taskId00ChangelogPartitions) .inState(State.RUNNING) .withInputPartitions(taskId00Partitions).build(); - final StreamTask activeTask2 = statefulTask(taskId01, taskId01ChangelogPartitions) - .inState(State.RUNNING) - .withInputPartitions(taskId01Partitions).build(); final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks, true); - when(tasks.allTasks()).thenReturn(Set.of(activeTask1, activeTask2)); final KafkaFuture mockFuture = KafkaFuture.completedFuture(null); when(schedulingTaskManager.lockTasks(any())).thenReturn(mockFuture); @@ -3501,7 +3497,6 @@ public void shouldCloseActiveTasksAndIgnoreExceptionsOnUncleanShutdown() { .when(task02).suspend(); doThrow(new RuntimeException("whatever")).when(activeTaskCreator).close(); - when(tasks.allTasks()).thenReturn(Set.of(task00, task01, task02)); when(tasks.activeTasks()).thenReturn(Set.of(task00, task01, task02)); taskManager.shutdown(false); @@ -3535,6 +3530,7 @@ public void shouldCloseStandbyTasksOnShutdown() { when(stateUpdater.tasks()).thenReturn(Set.of(standbyTask00)).thenReturn(Set.of()); when(stateUpdater.standbyTasks()).thenReturn(Set.of(standbyTask00)); + when(tasks.standbyTasks()).thenReturn(Set.of(standbyTask00)); final CompletableFuture futureForStandbyTask = new CompletableFuture<>(); when(stateUpdater.remove(taskId00)).thenReturn(futureForStandbyTask); @@ -3667,6 +3663,34 @@ public void shouldShutDownStateUpdaterAndCloseDirtyTasksFailedDuringRemoval() { verify(removedFailedStandbyTaskDuringRemoval).closeDirty(); } + @Test + public void shouldShutDownPendingTasksToInit() { + final TasksRegistry tasks = mock(TasksRegistry.class); + final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks); + + final StandbyTask standbyTask00 = standbyTask(taskId00, taskId00ChangelogPartitions) + .inState(State.RUNNING) + .withInputPartitions(taskId00Partitions) + .build(); + + final StreamTask activeTask01 = statefulTask(taskId01, taskId00ChangelogPartitions) + .inState(State.RUNNING) + .withInputPartitions(taskId00Partitions).build(); + + when(tasks.pendingTasksToInit()).thenReturn(Set.of(standbyTask00, activeTask01)); + + taskManager.shutdown(true); + + verify(standbyTask00).prepareCommit(true); + verify(standbyTask00).postCommit(true); + verify(standbyTask00).suspend(); + verify(standbyTask00).closeClean(); + + verify(activeTask01).prepareCommit(true); + verify(activeTask01).suspend(); + verify(activeTask01).closeClean(); + } + @Test public void shouldInitializeNewStandbyTasks() { final StandbyTask task01 = standbyTask(taskId01, taskId01ChangelogPartitions) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java index ec4d672f9c214..0c6407fafc33b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java @@ -25,6 +25,7 @@ import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Set; import static org.apache.kafka.common.utils.Utils.mkEntry; @@ -79,6 +80,7 @@ public void shouldKeepAddedTasks() { assertEquals(standbyTask, tasks.task(standbyTask.id())); assertEquals(Set.of(statefulTask, statelessTask), new HashSet<>(tasks.activeTasks())); + assertEquals(Set.of(standbyTask), new HashSet<>(tasks.standbyTasks())); assertEquals(Set.of(statefulTask, statelessTask, standbyTask), tasks.allTasks()); assertEquals(Set.of(statefulTask, standbyTask), tasks.tasks(Set.of(statefulTask.id(), standbyTask.id()))); assertEquals(Set.of(statefulTask.id(), statelessTask.id(), standbyTask.id()), tasks.allTaskIds()); @@ -230,4 +232,18 @@ public void shouldClearAllPendingTasks() { assertTrue(tasks.pendingActiveTasksToCreate().isEmpty()); assertTrue(tasks.pendingStandbyTasksToCreate().isEmpty()); } + + @Test + public void shouldRemovePendingTask() { + final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)) + .inState(State.SUSPENDED).build(); + tasks.addPendingTasksToInit(List.of(activeTask1)); + + tasks.removeTask(activeTask1); + assertFalse(tasks.pendingTasksToInit().contains(activeTask1)); + assertFalse(tasks.allTasks().contains(activeTask1)); + + tasks.addPendingTasksToInit(List.of(activeTask1)); + assertTrue(tasks.pendingTasksToInit().contains(activeTask1)); + } } From 35276715811b2fcdc7f3c94c1bee19a9d852a2d4 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 29 Jan 2026 21:52:06 -0800 Subject: [PATCH 02/16] Update streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java Co-authored-by: Matthias J. Sax --- .../streams/integration/RebalanceTaskClosureIntegrationTest.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index efd409d76e16f..9afd2241bda2c 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -72,7 +72,6 @@ public void before(final TestInfo testInfo) throws InterruptedException, IOExcep cluster.start(); cluster.createTopic(INPUT_TOPIC_NAME, NUM_PARTITIONS, 1); safeTestName = safeUniqueTestName(testInfo); - } @AfterEach From 8ee99ee82df2cf89fbb769d26c66395fd3a63761 Mon Sep 17 00:00:00 2001 From: Nikita Date: Mon, 2 Feb 2026 10:40:38 -0800 Subject: [PATCH 03/16] Added a comment on what the test actually tests. Minor refactoring. --- .../RebalanceTaskClosureIntegrationTest.java | 34 ++++++++++++++++--- 1 file changed, 29 insertions(+), 5 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index 9afd2241bda2c..da4d62a26acbf 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -40,6 +40,7 @@ import org.apache.kafka.streams.state.internals.RocksDBStore; import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.TestUtils; + import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -85,6 +86,31 @@ public void after() { } } + /** + * The conditions that we need to meet: + *

    + *
  • There is a task with an open store in {@link org.apache.kafka.streams.processor.internals.TasksRegistry#pendingTasksToInit}
  • + *
  • StreamThread gets into PENDING_SHUTDOWN state, so that {@link StreamThread#isStartingRunningOrPartitionAssigned} return false + * before we call {@link StreamThread#checkStateUpdater} that would move the task to the StateUpdater
  • + *

+ * If all conditions are met, {@code TaskManager} needs to correctly close the open store during shutdown. + *

+ * In order to have a task with an open store in the pending task list we first need to have an active task that gets converted + * to a standby one during rebalance(see {@link org.apache.kafka.streams.processor.internals.TaskManager#closeAndRecycleTasks}). + *

+ * For that this test: + *

    + *
  • starts a KS app and waits for it to fully start
  • + *
  • starts another KS app which will trigger reassignment
  • + *
  • waits for {@link CachedStateStore#clearCache} to be called(it's called during task recycle) and locks on it
  • + *
  • shutdowns the first KS app and waits for the stream tread to get into PENDING_SHUTDOWN state
  • + *
  • releases the lock
  • + *

+ * At this point {@link org.apache.kafka.streams.processor.internals.TaskManager#shutdown} will be called, + * and we will have a pending task to init with an open store(because tasks keep their stores open during recycle). + *

+ * This test verifies that the open store is closed during shutdown. + */ @Test public void shouldClosePendingTasksToInitAfterRebalance() throws Exception { final CountDownLatch recycleLatch = new CountDownLatch(1); @@ -124,12 +150,12 @@ public KeyValueStore build() { // That's exactly what we are waiting for recycleLatch.await(); - // sending a message to disable retries in the consumer client. if there are no messages, it retries the whole sequence of actions, - // including the rebalance data. which we don't want, because we just staged the right condition + // sending a message to avoid retries in the consumer client. if there are no messages, it retries both poll for new messages and reassignment. + // which we don't want, because we just staged the right condition(see ClassicKafkaConsumer#poll()). IntegrationTestUtils.produceKeyValuesSynchronously(INPUT_TOPIC_NAME, List.of(new KeyValue<>(1L, "key")), TestUtils.producerConfig(cluster.bootstrapServers(), LongSerializer.class, StringSerializer.class, new Properties()), cluster.time); // Now we can close both apps. The StreamThreadStateListener will unblock the clearCache call, letting the rebalance finish. - // We don't want it to happen any sooner, because we want the stream thread to stop before it gets to moving messages from task registry to state updater. + // We don't want it to happen any sooner, because we want the stream thread to stop before it gets to moving tasks from task registry to state updater. streams1.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); streams2.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); @@ -141,7 +167,6 @@ private Properties props(final String storePathSuffix) { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, safeTestName); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); - streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG, 1000); streamsConfiguration.put(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, 1000); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath() + "/" + storePathSuffix); @@ -149,7 +174,6 @@ private Properties props(final String storePathSuffix) { streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.LongSerde.class); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); - streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); streamsConfiguration.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); return streamsConfiguration; From 87933478f6d6499f593b9ae9b5106a7cde526a88 Mon Sep 17 00:00:00 2001 From: Nikita Date: Tue, 3 Feb 2026 13:05:42 -0800 Subject: [PATCH 04/16] Added PendingTasksToClose. --- .../processor/internals/TaskManager.java | 14 ++++---- .../streams/processor/internals/Tasks.java | 34 +++++++++++++++++-- .../processor/internals/TasksRegistry.java | 8 +++++ .../processor/internals/TaskManagerTest.java | 3 +- .../processor/internals/TasksTest.java | 25 +++++++++++--- 5 files changed, 70 insertions(+), 14 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index 05f7c10b2e3ca..d332123c2cea6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -1416,13 +1416,13 @@ void shutdown(final boolean clean) { activeTasks.addAll(tasks.activeTasks()); final Set standbyTasks = new TreeSet<>(Comparator.comparing(Task::id)); standbyTasks.addAll(tasks.standbyTasks()); - for (Task pendingTask : tasks.pendingTasksToInit()) { - if (pendingTask.isActive()) { - activeTasks.add(pendingTask); - } else { - standbyTasks.add(pendingTask); - } - } + + Set pendingActiveTasks = tasks.drainPendingActiveTasksToInit(); + activeTasks.addAll(pendingActiveTasks); + tasks.addPendingTasksToClose(pendingActiveTasks); + Set pendingStandbyTasks = tasks.drainPendingStandbyTasksToInit(); + standbyTasks.addAll(pendingStandbyTasks); + tasks.addPendingTasksToClose(pendingStandbyTasks); executeAndMaybeSwallow( clean, diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java index bfb5ca1286c79..17afe61236e10 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java @@ -58,6 +58,7 @@ class Tasks implements TasksRegistry { private final Map> pendingActiveTasksToCreate = new HashMap<>(); private final Map> pendingStandbyTasksToCreate = new HashMap<>(); private final Set pendingTasksToInit = new HashSet<>(); + private final Set pendingTasksToClose = new HashSet<>(); private final Set failedTaskIds = new HashSet<>(); // TODO: convert to Stream/StandbyTask when we remove TaskManager#StateMachineTask with mocks @@ -124,6 +125,20 @@ public Set drainPendingActiveTasksToInit() { return result; } + @Override + public Set drainPendingStandbyTasksToInit() { + final Set result = new HashSet<>(); + final Iterator iterator = pendingTasksToInit.iterator(); + while (iterator.hasNext()) { + final Task task = iterator.next(); + if (!task.isActive()) { + result.add(task); + iterator.remove(); + } + } + return result; + } + @Override public Set pendingTasksToInit() { return Collections.unmodifiableSet(pendingTasksToInit); @@ -139,6 +154,21 @@ public boolean hasPendingTasksToInit() { return !pendingTasksToInit.isEmpty(); } + @Override + public Set pendingTasksToClose() { + return Collections.unmodifiableSet(pendingTasksToClose); + } + + @Override + public void addPendingTasksToClose(Collection tasks) { + pendingTasksToClose.addAll(tasks); + } + + @Override + public boolean hasPendingTasksToClose() { + return !pendingTasksToClose.isEmpty(); + } + @Override public void addActiveTasks(final Collection newTasks) { if (!newTasks.isEmpty()) { @@ -193,8 +223,8 @@ public synchronized void removeTask(final Task taskToRemove) { throw new IllegalStateException("Attempted to remove a task that is not closed or suspended: " + taskId); } - if (pendingTasksToInit.contains(taskToRemove)) { - pendingTasksToInit.remove(taskToRemove); + if (pendingTasksToClose.contains(taskToRemove)) { + pendingTasksToClose.remove(taskToRemove); } else if (taskToRemove.isActive()) { if (activeTasksPerId.remove(taskId) == null) { throw new IllegalArgumentException("Attempted to remove an active task that is not owned: " + taskId); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java index eae21bf219265..6099efb5bc759 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TasksRegistry.java @@ -39,12 +39,20 @@ public interface TasksRegistry { Set drainPendingActiveTasksToInit(); + Set drainPendingStandbyTasksToInit(); + Set pendingTasksToInit(); void addPendingTasksToInit(final Collection tasks); boolean hasPendingTasksToInit(); + Set pendingTasksToClose(); + + void addPendingTasksToClose(final Collection tasks); + + boolean hasPendingTasksToClose(); + void addActiveTasks(final Collection tasks); void addStandbyTasks(final Collection tasks); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 11c2b294bf569..ac5306b6e0908 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -3677,7 +3677,8 @@ public void shouldShutDownPendingTasksToInit() { .inState(State.RUNNING) .withInputPartitions(taskId00Partitions).build(); - when(tasks.pendingTasksToInit()).thenReturn(Set.of(standbyTask00, activeTask01)); + when(tasks.drainPendingStandbyTasksToInit()).thenReturn(Set.of(standbyTask00)); + when(tasks.drainPendingActiveTasksToInit()).thenReturn(Set.of(activeTask01)); taskManager.shutdown(true); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java index 0c6407fafc33b..3776078785e2e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java @@ -166,6 +166,23 @@ public void shouldVerifyIfPendingActiveTaskToInitAreDrained() { assertTrue(tasks.pendingTasksToInit().containsAll(Set.of(standbyTask1, standbyTask2))); } + @Test + public void shouldVerifyIfPendingStandbyTaskToInitAreDrained() { + final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).build(); + final StreamTask activeTask2 = statefulTask(TASK_0_1, Set.of(TOPIC_PARTITION_B_1)).build(); + final StandbyTask standbyTask1 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_A_0)).build(); + final StandbyTask standbyTask2 = standbyTask(TASK_1_1, Set.of(TOPIC_PARTITION_A_1)).build(); + tasks.addPendingTasksToInit(Set.of(activeTask1, activeTask2, standbyTask1, standbyTask2)); + + final Set standbyTasksToInit = tasks.drainPendingStandbyTasksToInit(); + assertEquals(2, standbyTasksToInit.size()); + assertTrue(standbyTasksToInit.containsAll(Set.of(standbyTask1, standbyTask2))); + assertFalse(standbyTasksToInit.containsAll(Set.of(activeTask1, activeTask2))); + assertEquals(2, tasks.pendingTasksToInit().size()); + assertTrue(tasks.hasPendingTasksToInit()); + assertTrue(tasks.pendingTasksToInit().containsAll(Set.of(activeTask1, activeTask2))); + } + @Test public void shouldAddFailedTask() { final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).build(); @@ -234,16 +251,16 @@ public void shouldClearAllPendingTasks() { } @Test - public void shouldRemovePendingTask() { + public void shouldRemovePendingTaskToClose() { final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)) .inState(State.SUSPENDED).build(); - tasks.addPendingTasksToInit(List.of(activeTask1)); + tasks.addPendingTasksToClose(List.of(activeTask1)); tasks.removeTask(activeTask1); assertFalse(tasks.pendingTasksToInit().contains(activeTask1)); assertFalse(tasks.allTasks().contains(activeTask1)); - tasks.addPendingTasksToInit(List.of(activeTask1)); - assertTrue(tasks.pendingTasksToInit().contains(activeTask1)); + tasks.addPendingTasksToClose(List.of(activeTask1)); + assertTrue(tasks.pendingTasksToClose().contains(activeTask1)); } } From 1d7586a5496d338410d5bc7ae12499f2f1db5aa1 Mon Sep 17 00:00:00 2001 From: Nikita Date: Wed, 4 Feb 2026 13:44:28 -0800 Subject: [PATCH 05/16] Updated the comment to better explain the sequence of actions. --- .../integration/RebalanceTaskClosureIntegrationTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index da4d62a26acbf..73b48079c2980 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -150,8 +150,9 @@ public KeyValueStore build() { // That's exactly what we are waiting for recycleLatch.await(); - // sending a message to avoid retries in the consumer client. if there are no messages, it retries both poll for new messages and reassignment. - // which we don't want, because we just staged the right condition(see ClassicKafkaConsumer#poll()). + // sending a message to avoid retries in the consumer client. if there are no messages, it retries both poll for new messages and reassignment(see ClassicKafkaConsumer#poll()). + // during reassignment we close pending tasks to init(see TaskManager#handleTasksPendingInitialization()). + // hence during first attempt we will put the task into the pending to init list, and during next attempt we will delete it from the list and close it IntegrationTestUtils.produceKeyValuesSynchronously(INPUT_TOPIC_NAME, List.of(new KeyValue<>(1L, "key")), TestUtils.producerConfig(cluster.bootstrapServers(), LongSerializer.class, StringSerializer.class, new Properties()), cluster.time); // Now we can close both apps. The StreamThreadStateListener will unblock the clearCache call, letting the rebalance finish. From be6646ac99e4416ca1f304bdb9ab8258d315a5ec Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 14:48:59 -0800 Subject: [PATCH 06/16] Apply suggestion from @mjsax Co-authored-by: Matthias J. Sax --- .../kafka/streams/processor/internals/TaskManagerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index ac5306b6e0908..4cf99db6ddbc1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -3664,7 +3664,7 @@ public void shouldShutDownStateUpdaterAndCloseDirtyTasksFailedDuringRemoval() { } @Test - public void shouldShutDownPendingTasksToInit() { + public void shouldClosePendingTasksToInitDuringShutdown() { final TasksRegistry tasks = mock(TasksRegistry.class); final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks); From d4f598ce8adf6e31fdee81d5f5f9e18c3d044dd1 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 14:49:29 -0800 Subject: [PATCH 07/16] Apply suggestion from @mjsax Co-authored-by: Matthias J. Sax --- .../integration/RebalanceTaskClosureIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index 73b48079c2980..c21db1b6d440f 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -90,7 +90,7 @@ public void after() { * The conditions that we need to meet: *

    *
  • There is a task with an open store in {@link org.apache.kafka.streams.processor.internals.TasksRegistry#pendingTasksToInit}
  • - *
  • StreamThread gets into PENDING_SHUTDOWN state, so that {@link StreamThread#isStartingRunningOrPartitionAssigned} return false + *
  • StreamThread gets into PENDING_SHUTDOWN state, so that {@link StreamThread#isStartingRunningOrPartitionAssigned} returns false * before we call {@link StreamThread#checkStateUpdater} that would move the task to the StateUpdater
  • *

* If all conditions are met, {@code TaskManager} needs to correctly close the open store during shutdown. From 6b67cc2a008faecd55e5c6ef2e2c2a58e6a914e1 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 14:59:27 -0800 Subject: [PATCH 08/16] Apply suggestion from @mjsax Co-authored-by: Matthias J. Sax --- .../org/apache/kafka/streams/processor/internals/TasksTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java index 3776078785e2e..db739719f30ae 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java @@ -175,6 +175,7 @@ public void shouldVerifyIfPendingStandbyTaskToInitAreDrained() { tasks.addPendingTasksToInit(Set.of(activeTask1, activeTask2, standbyTask1, standbyTask2)); final Set standbyTasksToInit = tasks.drainPendingStandbyTasksToInit(); + assertEquals(2, standbyTasksToInit.size()); assertTrue(standbyTasksToInit.containsAll(Set.of(standbyTask1, standbyTask2))); assertFalse(standbyTasksToInit.containsAll(Set.of(activeTask1, activeTask2))); From db98faad675f7194b532dac31dc983935b47d3fd Mon Sep 17 00:00:00 2001 From: Nikita Date: Thu, 5 Feb 2026 15:08:22 -0800 Subject: [PATCH 09/16] Added missing finals. Modified the test so it actually reproduces the issue. Small refactoring. --- .../kafka/streams/integration/KafkaStreamsWrapper.java | 2 +- .../RebalanceTaskClosureIntegrationTest.java | 10 +++++----- .../kafka/streams/processor/internals/TaskManager.java | 7 ++++--- .../kafka/streams/processor/internals/Tasks.java | 2 +- .../streams/processor/internals/TaskManagerTest.java | 5 +++-- .../kafka/streams/processor/internals/TasksTest.java | 4 +--- 6 files changed, 15 insertions(+), 15 deletions(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java index 7f7bea6964455..847b7df46a4f7 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KafkaStreamsWrapper.java @@ -48,7 +48,7 @@ public List streamThreads() { public void setStreamThreadStateListener(final StreamThread.StateListener listener) { if (state == State.CREATED) { for (final StreamThread thread : threads) { - StreamThread.StateListener originalListener = thread.getStateListener(); + final StreamThread.StateListener originalListener = thread.getStateListener(); thread.setStateListener((t, newState, oldState) -> { originalListener.onChange(t, newState, oldState); listener.onChange(t, newState, oldState); diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index c21db1b6d440f..2879377737cc9 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -97,12 +97,14 @@ public void after() { *

* In order to have a task with an open store in the pending task list we first need to have an active task that gets converted * to a standby one during rebalance(see {@link org.apache.kafka.streams.processor.internals.TaskManager#closeAndRecycleTasks}). + * Second, we need to avoid the second rebalance. *

* For that this test: *

    *
  • starts a KS app and waits for it to fully start
  • *
  • starts another KS app which will trigger reassignment
  • *
  • waits for {@link CachedStateStore#clearCache} to be called(it's called during task recycle) and locks on it
  • + *
  • sends a message with wrong types to crash the stream thread
  • *
  • shutdowns the first KS app and waits for the stream tread to get into PENDING_SHUTDOWN state
  • *
  • releases the lock
  • *

@@ -150,11 +152,9 @@ public KeyValueStore build() { // That's exactly what we are waiting for recycleLatch.await(); - // sending a message to avoid retries in the consumer client. if there are no messages, it retries both poll for new messages and reassignment(see ClassicKafkaConsumer#poll()). - // during reassignment we close pending tasks to init(see TaskManager#handleTasksPendingInitialization()). - // hence during first attempt we will put the task into the pending to init list, and during next attempt we will delete it from the list and close it - IntegrationTestUtils.produceKeyValuesSynchronously(INPUT_TOPIC_NAME, List.of(new KeyValue<>(1L, "key")), - TestUtils.producerConfig(cluster.bootstrapServers(), LongSerializer.class, StringSerializer.class, new Properties()), cluster.time); + // sending a message with wrong key and value types to trigger a stream thread failure and avoid the second rebalance + IntegrationTestUtils.produceKeyValuesSynchronously(INPUT_TOPIC_NAME, List.of(new KeyValue<>("key", 1L)), + TestUtils.producerConfig(cluster.bootstrapServers(), StringSerializer.class, LongSerializer.class, new Properties()), cluster.time); // Now we can close both apps. The StreamThreadStateListener will unblock the clearCache call, letting the rebalance finish. // We don't want it to happen any sooner, because we want the stream thread to stop before it gets to moving tasks from task registry to state updater. streams1.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index d332123c2cea6..8e6433c0090f6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -1414,13 +1414,14 @@ void shutdown(final boolean clean) { // TODO: change type to `StreamTask` final Set activeTasks = new TreeSet<>(Comparator.comparing(Task::id)); activeTasks.addAll(tasks.activeTasks()); + // TODO: change type to `StandbyTask` final Set standbyTasks = new TreeSet<>(Comparator.comparing(Task::id)); standbyTasks.addAll(tasks.standbyTasks()); - Set pendingActiveTasks = tasks.drainPendingActiveTasksToInit(); + final Set pendingActiveTasks = tasks.drainPendingActiveTasksToInit(); activeTasks.addAll(pendingActiveTasks); tasks.addPendingTasksToClose(pendingActiveTasks); - Set pendingStandbyTasks = tasks.drainPendingStandbyTasksToInit(); + final Set pendingStandbyTasks = tasks.drainPendingStandbyTasksToInit(); standbyTasks.addAll(pendingStandbyTasks); tasks.addPendingTasksToClose(pendingStandbyTasks); @@ -1590,7 +1591,7 @@ private Collection tryCloseCleanActiveTasks(final Collection activeT } } - for (final Task task : activeTaskIterable()) { + for (final Task task : activeTasksToClose) { try { task.postCommit(true); } catch (final RuntimeException e) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java index 17afe61236e10..4af90d181c669 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Tasks.java @@ -160,7 +160,7 @@ public Set pendingTasksToClose() { } @Override - public void addPendingTasksToClose(Collection tasks) { + public void addPendingTasksToClose(final Collection tasks) { pendingTasksToClose.addAll(tasks); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 4cf99db6ddbc1..86e732185fb0f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -3669,12 +3669,12 @@ public void shouldClosePendingTasksToInitDuringShutdown() { final TaskManager taskManager = setUpTaskManager(ProcessingMode.AT_LEAST_ONCE, tasks); final StandbyTask standbyTask00 = standbyTask(taskId00, taskId00ChangelogPartitions) - .inState(State.RUNNING) + .inState(State.CREATED) .withInputPartitions(taskId00Partitions) .build(); final StreamTask activeTask01 = statefulTask(taskId01, taskId00ChangelogPartitions) - .inState(State.RUNNING) + .inState(State.CREATED) .withInputPartitions(taskId00Partitions).build(); when(tasks.drainPendingStandbyTasksToInit()).thenReturn(Set.of(standbyTask00)); @@ -3688,6 +3688,7 @@ public void shouldClosePendingTasksToInitDuringShutdown() { verify(standbyTask00).closeClean(); verify(activeTask01).prepareCommit(true); + verify(activeTask01).postCommit(true); verify(activeTask01).suspend(); verify(activeTask01).closeClean(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java index db739719f30ae..0887c98287351 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TasksTest.java @@ -256,12 +256,10 @@ public void shouldRemovePendingTaskToClose() { final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)) .inState(State.SUSPENDED).build(); tasks.addPendingTasksToClose(List.of(activeTask1)); + assertTrue(tasks.pendingTasksToClose().contains(activeTask1)); tasks.removeTask(activeTask1); assertFalse(tasks.pendingTasksToInit().contains(activeTask1)); assertFalse(tasks.allTasks().contains(activeTask1)); - - tasks.addPendingTasksToClose(List.of(activeTask1)); - assertTrue(tasks.pendingTasksToClose().contains(activeTask1)); } } From f522db8877ea7daaa29652094355450cf255e49f Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 20:37:16 -0800 Subject: [PATCH 10/16] Apply suggestion from @mjsax Co-authored-by: Matthias J. Sax --- .../integration/RebalanceTaskClosureIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index 2879377737cc9..2cf86e2ea8d42 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -97,7 +97,7 @@ public void after() { *

* In order to have a task with an open store in the pending task list we first need to have an active task that gets converted * to a standby one during rebalance(see {@link org.apache.kafka.streams.processor.internals.TaskManager#closeAndRecycleTasks}). - * Second, we need to avoid the second rebalance. + * Second, we need to avoid the second rebalance, to avoid that the pending tasks is closed during such a rebalance, ie, before we enter the shutdown phase. *

* For that this test: *

    From df08c00ec2464ca1f491881d65b45bce99223963 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 20:37:36 -0800 Subject: [PATCH 11/16] Apply suggestion from @mjsax Co-authored-by: Matthias J. Sax --- .../integration/RebalanceTaskClosureIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index 2cf86e2ea8d42..03e55b2010209 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -104,7 +104,7 @@ public void after() { *
  • starts a KS app and waits for it to fully start
  • *
  • starts another KS app which will trigger reassignment
  • *
  • waits for {@link CachedStateStore#clearCache} to be called(it's called during task recycle) and locks on it
  • - *
  • sends a message with wrong types to crash the stream thread
  • + *
  • sends a message with wrong types to crash the stream thread (this avoids a second rebalance, and enters shutdown directly)
  • *
  • shutdowns the first KS app and waits for the stream tread to get into PENDING_SHUTDOWN state
  • *
  • releases the lock
  • *

From 8a1b56d33337353f24bf802fcbadb772e05bab98 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 20:37:55 -0800 Subject: [PATCH 12/16] Update streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java Co-authored-by: Matthias J. Sax --- .../streams/integration/RebalanceTaskClosureIntegrationTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index 03e55b2010209..a3340c88303ed 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -153,6 +153,7 @@ public KeyValueStore build() { recycleLatch.await(); // sending a message with wrong key and value types to trigger a stream thread failure and avoid the second rebalance + // note that writing this message does not trigger the crash right away -- the thread is still blocked inside `poll()` waiting for the shutdown latch to unlock to complete the previous, still ongoing rebalance IntegrationTestUtils.produceKeyValuesSynchronously(INPUT_TOPIC_NAME, List.of(new KeyValue<>("key", 1L)), TestUtils.producerConfig(cluster.bootstrapServers(), StringSerializer.class, LongSerializer.class, new Properties()), cluster.time); // Now we can close both apps. The StreamThreadStateListener will unblock the clearCache call, letting the rebalance finish. From ff7509647ac584c536309b848f1956bd420cd84e Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 20:38:12 -0800 Subject: [PATCH 13/16] Update streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java Co-authored-by: Matthias J. Sax --- .../integration/RebalanceTaskClosureIntegrationTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index a3340c88303ed..59fbe0f74fe1f 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -222,6 +222,10 @@ public void clearCache() { // Clear cache is called during recycle, so we use it as a hook recycleLatch.countDown(); try { + // after we signaled via recycleLatch, that the task was converted into a "pending task", + // we block the rebalance to complete, until we get the shutdown signal, + // to avoid that the "pending task" get fully initialized + // (otherwise, we don't have a pending task when the shutdown happens) pendingShutdownLatch.await(); } catch (InterruptedException e) { throw new RuntimeException(e); From ef20d15450146441fe142d0f3639ac75d154264e Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 20:38:22 -0800 Subject: [PATCH 14/16] Update streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java Co-authored-by: Matthias J. Sax --- .../integration/RebalanceTaskClosureIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index 59fbe0f74fe1f..89e5e6f5cc04f 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -105,7 +105,7 @@ public void after() { *

  • starts another KS app which will trigger reassignment
  • *
  • waits for {@link CachedStateStore#clearCache} to be called(it's called during task recycle) and locks on it
  • *
  • sends a message with wrong types to crash the stream thread (this avoids a second rebalance, and enters shutdown directly)
  • - *
  • shutdowns the first KS app and waits for the stream tread to get into PENDING_SHUTDOWN state
  • + *
  • shutdowns the first KS app
  • *
  • releases the lock
  • *

    * At this point {@link org.apache.kafka.streams.processor.internals.TaskManager#shutdown} will be called, From 6e8e87e7abccdc50758b79ccae6fa3971f4ede7b Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 20:38:31 -0800 Subject: [PATCH 15/16] Update streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java Co-authored-by: Matthias J. Sax --- .../integration/RebalanceTaskClosureIntegrationTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index 89e5e6f5cc04f..f04139f3c51bf 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -157,6 +157,8 @@ public KeyValueStore build() { IntegrationTestUtils.produceKeyValuesSynchronously(INPUT_TOPIC_NAME, List.of(new KeyValue<>("key", 1L)), TestUtils.producerConfig(cluster.bootstrapServers(), StringSerializer.class, LongSerializer.class, new Properties()), cluster.time); // Now we can close both apps. The StreamThreadStateListener will unblock the clearCache call, letting the rebalance finish. + // After the rebalance finished, the "poison pill" record gets picked up crashing the thread, + // and starting the shutdown directly // We don't want it to happen any sooner, because we want the stream thread to stop before it gets to moving tasks from task registry to state updater. streams1.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); streams2.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); From ca632f2482c47cdd066cea85f8d01b0deab37b96 Mon Sep 17 00:00:00 2001 From: Nikita Shupletsov Date: Thu, 5 Feb 2026 20:38:38 -0800 Subject: [PATCH 16/16] Update streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java Co-authored-by: Matthias J. Sax --- .../integration/RebalanceTaskClosureIntegrationTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java index f04139f3c51bf..d8acd9fdeefd0 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java @@ -159,7 +159,7 @@ public KeyValueStore build() { // Now we can close both apps. The StreamThreadStateListener will unblock the clearCache call, letting the rebalance finish. // After the rebalance finished, the "poison pill" record gets picked up crashing the thread, // and starting the shutdown directly - // We don't want it to happen any sooner, because we want the stream thread to stop before it gets to moving tasks from task registry to state updater. + // We don't want to let the rebalance finish before we trigger the shutdown, because we want the stream thread to stop before it gets to moving pending tasks from task registry to state updater. streams1.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP)); streams2.close(CloseOptions.groupMembershipOperation(CloseOptions.GroupMembershipOperation.LEAVE_GROUP));