Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
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,157 @@
/*
* 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.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;

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));
}
}

/**
* The conditions that we need to meet:
* <p><ul>
* <li>We have an unhandled task in {@link org.apache.kafka.streams.processor.internals.DefaultStateUpdater}</li>
* <li>StreamThread is not running, so {@link org.apache.kafka.streams.processor.internals.TaskManager#handleExceptionsFromStateUpdater} is not called anymore</li>
* <li>The task throws exception in {@link org.apache.kafka.streams.processor.internals.Task#maybeCheckpoint(boolean)} while being processed by {@code DefaultStateUpdater}</li>
* <li>{@link org.apache.kafka.streams.processor.internals.TaskManager#shutdownStateUpdater} tries to clean up all tasks that are left in the {@code DefaultStateUpdater}</li>
* </ul><p>
* If all conditions are met, {@code TaskManager} needs to be able to handle the failed task from the {@code DefaultStateUpdater} correctly and not hang.
*/
@Test
public void correctlyHandleFlushErrorsDuringRebalance() throws Exception {
final AtomicInteger numberOfStoreInits = new AtomicInteger();
final CountDownLatch pendingShutdownLatch = new CountDownLatch(1);

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() {
// 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)
// 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
if (numberOfStoreInits.get() == 9) {
try {
pendingShutdownLatch.await();
} 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) -> {
if (newState == KafkaStreams.State.PENDING_SHUTDOWN) {
pendingShutdownLatch.countDown();
}
});
streams.start();

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

streams.removeStreamThread();

TestUtils.waitForCondition(() -> streams.state() == KafkaStreams.State.REBALANCING, TimeUnit.MINUTES.toMillis(2), "Streams never reached REBALANCING state");
Copy link
Member

Choose a reason for hiding this comment

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

Why do we need this only for 4.1 branch?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

for some reason in 4.1 branch rebalancing kicks in only after ~1 minute after the thread is removed. it didn't happen in trunk


// Before shutting down, we want the tasks to be reassigned
TestUtils.waitForCondition(() -> numberOfStoreInits.get() == 9, "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 @@ -349,23 +349,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 swallow) {
log.warn("Checkpoint failed for corrupted task {}", task.id(), swallow);
}
}

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 +521,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 @@ -637,14 +640,19 @@ private void removeTask(final TaskId taskId) {
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();
try {
measureCheckpointLatency(() -> task.maybeCheckpoint(true));
pausedTasks.put(taskId, task);
updatingTasks.remove(taskId);
if (task.isActive()) {
transitToUpdateStandbysIfOnlyStandbysLeft();
}
log.info((task.isActive() ? "Active" : "Standby")
+ " task " + task.id() + " was paused from the updating tasks and added to the paused tasks.");

} catch (final StreamsException streamsException) {
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 +679,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 +719,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 @@ -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 @@ -704,7 +705,7 @@ private StateUpdater.RemovedTaskResult waitForFuture(final TaskId taskId,
final CompletableFuture<StateUpdater.RemovedTaskResult> future) {
final StateUpdater.RemovedTaskResult removedTaskResult;
try {
removedTaskResult = future.get();
removedTaskResult = future.get(5, TimeUnit.MINUTES);
if (removedTaskResult == null) {
throw new IllegalStateException("Task " + taskId + " was not found in the state updater. "
+ BUG_ERROR_MESSAGE);
Expand All @@ -719,6 +720,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 @@ -1499,6 +1504,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);
}

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 @@ -1507,24 +1518,31 @@ private void shutdownStateUpdater() {
final Set<Task> tasksToCloseClean = new HashSet<>();
final Set<Task> tasksToCloseDirty = new HashSet<>();
addToTasksToClose(futures, tasksToCloseClean, tasksToCloseDirty);
stateUpdater.shutdown(Duration.ofMillis(Long.MAX_VALUE));
// at this point we removed all tasks, so the shutdown should not take a lot of time
stateUpdater.shutdown(Duration.ofMinutes(1L));

for (final Task task : tasksToCloseClean) {
tasks.addTask(task);
}
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);
}
}
}

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

Expand Down
Loading