-
Notifications
You must be signed in to change notification settings - Fork 14.9k
KAFKA-19960: Close pending tasks on shutdown. #21365
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
Merged
+375
−12
Merged
Changes from all commits
Commits
Show all changes
17 commits
Select commit
Hold shift + click to select a range
ecbe338
KAFKA-19960: Close pending tasks on shutdown.
Nikita-Shupletsov 3527671
Update streams/integration-tests/src/test/java/org/apache/kafka/strea…
Nikita-Shupletsov 8ee99ee
Added a comment on what the test actually tests.
Nikita-Shupletsov 4f2bdf4
Merge branch 'apache:trunk' into KAFKA-19960
Nikita-Shupletsov 8793347
Added PendingTasksToClose.
Nikita-Shupletsov 1d7586a
Updated the comment to better explain the sequence of actions.
Nikita-Shupletsov be6646a
Apply suggestion from @mjsax
Nikita-Shupletsov d4f598c
Apply suggestion from @mjsax
Nikita-Shupletsov 6b67cc2
Apply suggestion from @mjsax
Nikita-Shupletsov db98faa
Added missing finals.
Nikita-Shupletsov f522db8
Apply suggestion from @mjsax
Nikita-Shupletsov df08c00
Apply suggestion from @mjsax
Nikita-Shupletsov 8a1b56d
Update streams/integration-tests/src/test/java/org/apache/kafka/strea…
Nikita-Shupletsov ff75096
Update streams/integration-tests/src/test/java/org/apache/kafka/strea…
Nikita-Shupletsov ef20d15
Update streams/integration-tests/src/test/java/org/apache/kafka/strea…
Nikita-Shupletsov 6e8e87e
Update streams/integration-tests/src/test/java/org/apache/kafka/strea…
Nikita-Shupletsov ca632f2
Update streams/integration-tests/src/test/java/org/apache/kafka/strea…
Nikita-Shupletsov File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Some comments aren't visible on the classic Files Changed page.
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
244 changes: 244 additions & 0 deletions
244
...c/test/java/org/apache/kafka/streams/integration/RebalanceTaskClosureIntegrationTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,244 @@ | ||
| /* | ||
| * 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)); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * The conditions that we need to meet: | ||
| * <p><ul> | ||
| * <li>There is a task with an open store in {@link org.apache.kafka.streams.processor.internals.TasksRegistry#pendingTasksToInit}</li> | ||
| * <li>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 </li> | ||
| * </ul><p> | ||
| * If all conditions are met, {@code TaskManager} needs to correctly close the open store during shutdown. | ||
| * <p> | ||
| * 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, to avoid that the pending tasks is closed during such a rebalance, ie, before we enter the shutdown phase. | ||
| * <p> | ||
| * For that this test: | ||
| * <p><ul> | ||
| * <li>starts a KS app and waits for it to fully start</li> | ||
| * <li>starts another KS app which will trigger reassignment</li> | ||
| * <li>waits for {@link CachedStateStore#clearCache} to be called(it's called during task recycle) and locks on it</li> | ||
| * <li>sends a message with wrong types to crash the stream thread (this avoids a second rebalance, and enters shutdown directly)</li> | ||
| * <li>shutdowns the first KS app</li> | ||
| * <li>releases the lock</li> | ||
| * </ul><p> | ||
| * 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). | ||
| * <p> | ||
| * This test verifies that the open store is closed during shutdown. | ||
| */ | ||
| @Test | ||
| public void shouldClosePendingTasksToInitAfterRebalance() throws Exception { | ||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 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<KeyValueStore<Bytes, byte[]>> storeBuilder = new AbstractStoreBuilder<>("testStateStore", Serdes.Integer(), Serdes.ByteArray(), new MockTime()) { | ||
|
|
||
| @Override | ||
| public KeyValueStore<Bytes, byte[]> 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 with wrong key and value types to trigger a stream thread failure and avoid the second rebalance | ||
Nikita-Shupletsov marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // 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. | ||
Nikita-Shupletsov marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // After the rebalance finished, the "poison pill" record gets picked up crashing the thread, | ||
| // and starting the shutdown directly | ||
| // 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)); | ||
|
|
||
| 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.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_STANDBY_REPLICAS_CONFIG, 1); | ||
|
|
||
| return streamsConfiguration; | ||
| } | ||
|
|
||
| private static class TestRocksDBStore extends RocksDBStore implements CachedStateStore<Bytes, byte[]> { | ||
|
|
||
| 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<Bytes, byte[]> listener, | ||
| final boolean sendOldValues) { | ||
| return false; | ||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| @Override | ||
| public void flushCache() { | ||
| } | ||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| @Override | ||
| public void clearCache() { | ||
mjsax marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| // 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(); | ||
Nikita-Shupletsov marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } catch (InterruptedException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized void close() { | ||
| closeCount.incrementAndGet(); | ||
| super.close(); | ||
| } | ||
| } | ||
|
|
||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.