|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | +package org.apache.kafka.streams.integration; |
| 18 | + |
| 19 | +import org.apache.kafka.clients.consumer.ConsumerConfig; |
| 20 | +import org.apache.kafka.common.serialization.Serdes; |
| 21 | +import org.apache.kafka.common.utils.MockTime; |
| 22 | +import org.apache.kafka.streams.KafkaStreams; |
| 23 | +import org.apache.kafka.streams.StreamsConfig; |
| 24 | +import org.apache.kafka.streams.TopologyWrapper; |
| 25 | +import org.apache.kafka.streams.errors.ProcessorStateException; |
| 26 | +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; |
| 27 | +import org.apache.kafka.streams.processor.StateStore; |
| 28 | +import org.apache.kafka.streams.processor.StateStoreContext; |
| 29 | +import org.apache.kafka.streams.state.KeyValueStore; |
| 30 | +import org.apache.kafka.streams.state.StoreBuilder; |
| 31 | +import org.apache.kafka.streams.state.internals.AbstractStoreBuilder; |
| 32 | +import org.apache.kafka.test.MockApiProcessorSupplier; |
| 33 | +import org.apache.kafka.test.MockKeyValueStore; |
| 34 | +import org.apache.kafka.test.TestUtils; |
| 35 | + |
| 36 | +import org.junit.jupiter.api.AfterEach; |
| 37 | +import org.junit.jupiter.api.BeforeEach; |
| 38 | +import org.junit.jupiter.api.Test; |
| 39 | +import org.junit.jupiter.api.TestInfo; |
| 40 | + |
| 41 | +import java.io.IOException; |
| 42 | +import java.time.Duration; |
| 43 | +import java.util.Properties; |
| 44 | +import java.util.concurrent.CountDownLatch; |
| 45 | +import java.util.concurrent.atomic.AtomicInteger; |
| 46 | + |
| 47 | +import static org.apache.kafka.streams.utils.TestUtils.safeUniqueTestName; |
| 48 | +import static org.junit.jupiter.api.Assertions.assertTrue; |
| 49 | + |
| 50 | +public class StateUpdaterFailureIntegrationTest { |
| 51 | + |
| 52 | + private static final int NUM_BROKERS = 1; |
| 53 | + protected static final String INPUT_TOPIC_NAME = "input-topic"; |
| 54 | + private static final int NUM_PARTITIONS = 6; |
| 55 | + |
| 56 | + private final EmbeddedKafkaCluster cluster = new EmbeddedKafkaCluster(NUM_BROKERS); |
| 57 | + |
| 58 | + private Properties streamsConfiguration; |
| 59 | + private final MockTime mockTime = cluster.time; |
| 60 | + private KafkaStreams streams; |
| 61 | + |
| 62 | + @BeforeEach |
| 63 | + public void before(final TestInfo testInfo) throws InterruptedException, IOException { |
| 64 | + cluster.start(); |
| 65 | + cluster.createTopic(INPUT_TOPIC_NAME, NUM_PARTITIONS, 1); |
| 66 | + streamsConfiguration = new Properties(); |
| 67 | + final String safeTestName = safeUniqueTestName(testInfo); |
| 68 | + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "app-" + safeTestName); |
| 69 | + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); |
| 70 | + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); |
| 71 | + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); |
| 72 | + streamsConfiguration.put(StreamsConfig.STATESTORE_CACHE_MAX_BYTES_CONFIG, 0); |
| 73 | + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100L); |
| 74 | + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.IntegerSerde.class); |
| 75 | + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class); |
| 76 | + streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2); |
| 77 | + |
| 78 | + } |
| 79 | + |
| 80 | + @AfterEach |
| 81 | + public void after() { |
| 82 | + cluster.stop(); |
| 83 | + if (streams != null) { |
| 84 | + streams.close(Duration.ofSeconds(30)); |
| 85 | + } |
| 86 | + } |
| 87 | + |
| 88 | + /** |
| 89 | + * The conditions that we need to meet: |
| 90 | + * <p><ul> |
| 91 | + * <li>We have an unhandled task in {@link org.apache.kafka.streams.processor.internals.DefaultStateUpdater}</li> |
| 92 | + * <li>StreamThread is not running, so {@link org.apache.kafka.streams.processor.internals.TaskManager#handleExceptionsFromStateUpdater} is not called anymore</li> |
| 93 | + * <li>The task throws exception in {@link org.apache.kafka.streams.processor.internals.Task#maybeCheckpoint(boolean)} while being processed by {@code DefaultStateUpdater}</li> |
| 94 | + * <li>{@link org.apache.kafka.streams.processor.internals.TaskManager#shutdownStateUpdater} tries to clean up all tasks that are left in the {@code DefaultStateUpdater}</li> |
| 95 | + * </ul><p> |
| 96 | + * If all conditions are met, {@code TaskManager} needs to be able to handle the failed task from the {@code DefaultStateUpdater} correctly and not hang. |
| 97 | + */ |
| 98 | + @Test |
| 99 | + public void correctlyHandleFlushErrorsDuringRebalance() throws Exception { |
| 100 | + final AtomicInteger numberOfStoreInits = new AtomicInteger(); |
| 101 | + final CountDownLatch pendingShutdownLatch = new CountDownLatch(1); |
| 102 | + |
| 103 | + final StoreBuilder<KeyValueStore<Object, Object>> storeBuilder = new AbstractStoreBuilder<>("testStateStore", Serdes.Integer(), Serdes.ByteArray(), new MockTime()) { |
| 104 | + |
| 105 | + @Override |
| 106 | + public KeyValueStore<Object, Object> build() { |
| 107 | + return new MockKeyValueStore(name, false) { |
| 108 | + |
| 109 | + @Override |
| 110 | + public void init(final StateStoreContext stateStoreContext, final StateStore root) { |
| 111 | + super.init(stateStoreContext, root); |
| 112 | + numberOfStoreInits.incrementAndGet(); |
| 113 | + } |
| 114 | + |
| 115 | + @Override |
| 116 | + public void flush() { |
| 117 | + // we want to throw the ProcessorStateException here only when the rebalance finished(we reassigned the 3 tasks from the removed thread to the existing thread) |
| 118 | + // we use waitForCondition to wait until the current state is PENDING_SHUTDOWN to make sure the Stream Thread will not handle the exception and we can get to in TaskManager#shutdownStateUpdater |
| 119 | + if (numberOfStoreInits.get() == 9) { |
| 120 | + try { |
| 121 | + pendingShutdownLatch.await(); |
| 122 | + } catch (final InterruptedException e) { |
| 123 | + throw new RuntimeException(e); |
| 124 | + } |
| 125 | + throw new ProcessorStateException("flush"); |
| 126 | + } |
| 127 | + } |
| 128 | + }; |
| 129 | + } |
| 130 | + }; |
| 131 | + |
| 132 | + final TopologyWrapper topology = new TopologyWrapper(); |
| 133 | + topology.addSource("ingest", INPUT_TOPIC_NAME); |
| 134 | + topology.addProcessor("my-processor", new MockApiProcessorSupplier<>(), "ingest"); |
| 135 | + topology.addStateStore(storeBuilder, "my-processor"); |
| 136 | + |
| 137 | + streams = new KafkaStreams(topology, streamsConfiguration); |
| 138 | + streams.setStateListener((newState, oldState) -> { |
| 139 | + if (newState == KafkaStreams.State.PENDING_SHUTDOWN) { |
| 140 | + pendingShutdownLatch.countDown(); |
| 141 | + } |
| 142 | + }); |
| 143 | + streams.start(); |
| 144 | + |
| 145 | + TestUtils.waitForCondition(() -> streams.state() == KafkaStreams.State.RUNNING, "Streams never reached RUNNING state"); |
| 146 | + |
| 147 | + streams.removeStreamThread(); |
| 148 | + |
| 149 | + // Before shutting down, we want the tasks to be reassigned |
| 150 | + TestUtils.waitForCondition(() -> numberOfStoreInits.get() == 9, "Streams never reinitialized the store enough times"); |
| 151 | + |
| 152 | + assertTrue(streams.close(Duration.ofSeconds(60))); |
| 153 | + } |
| 154 | +} |
0 commit comments