wcarlson5 commented on a change in pull request #9487: URL: https://github.com/apache/kafka/pull/9487#discussion_r523069611
########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -366,6 +375,93 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} + * throws an unexpected exception. + * These might be exceptions indicating rare bugs in Kafka Streams, or they + * might be exceptions thrown by your code, for example a NullPointerException thrown from your processor + * logic. + * The handler will execute on the thread that produced the exception. + * So inorder to get the thread as the java handler type uses use Thread.currentThread() Review comment: changed to ` In order to get the thread uses use Thread.currentThread()` Does that work better? ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java ########## @@ -60,6 +60,11 @@ public void onPartitionsAssigned(final Collection<TopicPartition> partitions) { } else if (assignmentErrorCode.get() == AssignorError.ASSIGNMENT_ERROR.code()) { log.error("Received error code {}", AssignorError.ASSIGNMENT_ERROR); throw new TaskAssignmentException("Hit an unexpected exception during task assignment phase of rebalance"); + } else if (assignmentErrorCode.get() == AssignorError.SHUTDOWN_REQUESTED.code()) { + log.error("A Kafka Streams client in this Kafka Streams application is requesting to shutdown the application"); + streamThread.shutdownToError(); + taskManager.handleRebalanceComplete(); Review comment: For the same reason I had to add to the other cases as the close from the new handler will not finish otherwise ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -366,6 +374,63 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} + * throws an unexpected exception. + * These might be exceptions indicating rare bugs in Kafka Streams, or they + * might be exceptions thrown by your code, for example a NullPointerException thrown from your processor + * logic. + * <p> + * Note, this handler must be threadsafe, since it will be shared among all threads, and invoked from any + * thread that encounters such an exception. + * + * @param streamsUncaughtExceptionHandler the uncaught exception handler of type {@link StreamsUncaughtExceptionHandler} for all internal threads + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + * @throws NullPointerException if streamsUncaughtExceptionHandler is null. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler handler = exception -> handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + Objects.requireNonNull(streamsUncaughtExceptionHandler); + for (final StreamThread thread : threads) { + thread.setStreamsUncaughtExceptionHandler(handler); + } + if (globalStreamThread != null) { + globalStreamThread.setUncaughtExceptionHandler(handler); + } + } else { + throw new IllegalStateException("Can only set UncaughtExceptionHandler in CREATED state. " + + "Current state is: " + state); + } + } + } + + private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse handleStreamsUncaughtException(final Throwable e, + final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(e); + switch (action) { + case SHUTDOWN_CLIENT: + log.error("Encountered the following exception during processing " + + "and the registered exception handler opted to \" + action + \"." + + " The streams client is going to shut down now. ", e); + close(Duration.ZERO); Review comment: we should be able to change it to `close()` ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -366,6 +375,93 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} + * throws an unexpected exception. + * These might be exceptions indicating rare bugs in Kafka Streams, or they + * might be exceptions thrown by your code, for example a NullPointerException thrown from your processor + * logic. + * The handler will execute on the thread that produced the exception. + * So inorder to get the thread as the java handler type uses use Thread.currentThread() + * <p> + * Note, this handler must be threadsafe, since it will be shared among all threads, and invoked from any + * thread that encounters such an exception. + * + * @param streamsUncaughtExceptionHandler the uncaught exception handler of type {@link StreamsUncaughtExceptionHandler} for all internal threads + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + * @throws NullPointerException if streamsUncaughtExceptionHandler is null. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final Consumer<Throwable> handler = exception -> handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + Objects.requireNonNull(streamsUncaughtExceptionHandler); + for (final StreamThread thread : threads) { + thread.setStreamsUncaughtExceptionHandler(handler); + } + if (globalStreamThread != null) { + globalStreamThread.setUncaughtExceptionHandler(handler); + } + } else { + throw new IllegalStateException("Can only set UncaughtExceptionHandler in CREATED state. " + + "Current state is: " + state); + } + } + } + + private void handleStreamsUncaughtExceptionDefaultWrapper(final Throwable throwable) { + if (Thread.getDefaultUncaughtExceptionHandler() != null) { + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } else if (throwable instanceof Error) { + throw (Error) throwable; + } else { + throw new RuntimeException("Unexpected checked exception caught in the uncaught exception handler", throwable); + } Review comment: We have to do the casting in order to throw the exception. Otherwise the compiler complains about checked vs unchecked exceptions ########## File path: streams/src/test/java/org/apache/kafka/streams/integration/EosIntegrationTest.java ########## @@ -647,6 +647,7 @@ public void shouldNotViolateEosIfOneTaskGetsFencedUsingIsolatedAppInstances() th return data; } + @SuppressWarnings("deprecation") //the threads should no longer fail one thread one at a time Review comment: When we remove the old handler we either need to remove the test or remove the suppression. That is what I am hoping the comment will do ########## File path: streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java ########## @@ -144,6 +143,7 @@ public void whenShuttingDown() throws IOException { } @Test + @Deprecated //a single thread should no longer die Review comment: So the problem that I am facing is that many tests are set up to work with the old handler. I was able to adapt most to use the new handler but not all. Some, like a few EOS tests, require one thread to die at a time. So I either suppress the deprecation or tag the test as deprecated, thus indicating it should be removed when the old handler is. Another problem is that a few tests rely on the threads dying one at a time or they test behavior in this case but they do not set an old handler. So I can either 1) set an old handler and mark for deletion or 2) adapt for the new out come. For the ones I could, I changed to the new flow but I could not do that with all of them. @vvcephei @ableegoldman @cadonna How would you suggest updating these tests? ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -366,6 +375,93 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} + * throws an unexpected exception. + * These might be exceptions indicating rare bugs in Kafka Streams, or they + * might be exceptions thrown by your code, for example a NullPointerException thrown from your processor + * logic. + * The handler will execute on the thread that produced the exception. + * So inorder to get the thread as the java handler type uses use Thread.currentThread() + * <p> + * Note, this handler must be threadsafe, since it will be shared among all threads, and invoked from any + * thread that encounters such an exception. + * + * @param streamsUncaughtExceptionHandler the uncaught exception handler of type {@link StreamsUncaughtExceptionHandler} for all internal threads + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + * @throws NullPointerException if streamsUncaughtExceptionHandler is null. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final Consumer<Throwable> handler = exception -> handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + Objects.requireNonNull(streamsUncaughtExceptionHandler); + for (final StreamThread thread : threads) { + thread.setStreamsUncaughtExceptionHandler(handler); + } + if (globalStreamThread != null) { + globalStreamThread.setUncaughtExceptionHandler(handler); + } + } else { + throw new IllegalStateException("Can only set UncaughtExceptionHandler in CREATED state. " + + "Current state is: " + state); + } + } + } + + private void handleStreamsUncaughtExceptionDefaultWrapper(final Throwable throwable) { + if (Thread.getDefaultUncaughtExceptionHandler() != null) { + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } else if (throwable instanceof Error) { + throw (Error) throwable; + } else { + throw new RuntimeException("Unexpected checked exception caught in the uncaught exception handler", throwable); + } + } else { + handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT); + } + } + + private void handleStreamsUncaughtException(final Throwable e, + final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(e); + if (Thread.getDefaultUncaughtExceptionHandler() != null) { + log.warn("Stream's new uncaught exception handler is set as well as the deprecated old handler." + + "The old handler will be ignored as long as a new handler is set."); + } + switch (action) { + case SHUTDOWN_CLIENT: + log.error("Encountered the following exception during processing " + + "and the registered exception handler opted to \" + action + \"." + + " The streams client is going to shut down now. ", e); + close(Duration.ZERO); + break; + case SHUTDOWN_APPLICATION: + if (e instanceof Error) { + log.error("This option requires running threads to shut down the application." + + "but the uncaught exception was an Error, which means this runtime is no " + + "longer in a well-defined state. Attempting to send the shutdown command anyway.", e); + } + if (Thread.currentThread().equals(globalStreamThread) && threads.stream().noneMatch(StreamThread::isRunning)) { + log.error("Exception in global thread caused the application to attempt to shutdown." + + " This action will succeed only if there is at least one StreamThread running on this client." + + " Currently there are no running threads so will now close the client."); + close(Duration.ZERO); + } else { + for (final StreamThread streamThread : threads) { + streamThread.sendShutdownRequest(AssignorError.SHUTDOWN_REQUESTED); + } + log.error("Encountered the following exception during processing " + + "and the application is going to shut down: ", e); + } + break; + } + } + + + + Review comment: that is a lot of line breaks ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java ########## @@ -19,7 +19,8 @@ public final class StreamsAssignmentProtocolVersions { public static final int UNKNOWN = -1; public static final int EARLIEST_PROBEABLE_VERSION = 3; - public static final int LATEST_SUPPORTED_VERSION = 8; + public static final int LATEST_SUPPORTED_VERSION = 9; + //When changing the versions update this test: streams_upgrade_test.py::StreamsUpgradeTest.test_version_probing_upgrade Review comment: I'll add that to the comment, and add a test ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -366,6 +375,93 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} + * throws an unexpected exception. + * These might be exceptions indicating rare bugs in Kafka Streams, or they + * might be exceptions thrown by your code, for example a NullPointerException thrown from your processor + * logic. + * The handler will execute on the thread that produced the exception. + * So inorder to get the thread as the java handler type uses use Thread.currentThread() + * <p> + * Note, this handler must be threadsafe, since it will be shared among all threads, and invoked from any + * thread that encounters such an exception. + * + * @param streamsUncaughtExceptionHandler the uncaught exception handler of type {@link StreamsUncaughtExceptionHandler} for all internal threads + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + * @throws NullPointerException if streamsUncaughtExceptionHandler is null. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final Consumer<Throwable> handler = exception -> handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + Objects.requireNonNull(streamsUncaughtExceptionHandler); + for (final StreamThread thread : threads) { + thread.setStreamsUncaughtExceptionHandler(handler); + } + if (globalStreamThread != null) { + globalStreamThread.setUncaughtExceptionHandler(handler); + } + } else { + throw new IllegalStateException("Can only set UncaughtExceptionHandler in CREATED state. " + + "Current state is: " + state); + } + } + } + + private void handleStreamsUncaughtExceptionDefaultWrapper(final Throwable throwable) { + if (Thread.getDefaultUncaughtExceptionHandler() != null) { + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } else if (throwable instanceof Error) { + throw (Error) throwable; + } else { + throw new RuntimeException("Unexpected checked exception caught in the uncaught exception handler", throwable); + } + } else { + handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT); + } + } + + private void handleStreamsUncaughtException(final Throwable e, + final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(e); + if (Thread.getDefaultUncaughtExceptionHandler() != null) { + log.warn("Stream's new uncaught exception handler is set as well as the deprecated old handler." + + "The old handler will be ignored as long as a new handler is set."); + } + switch (action) { + case SHUTDOWN_CLIENT: + log.error("Encountered the following exception during processing " + + "and the registered exception handler opted to \" + action + \"." + Review comment: yes good catch ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -366,6 +375,93 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} + * throws an unexpected exception. + * These might be exceptions indicating rare bugs in Kafka Streams, or they + * might be exceptions thrown by your code, for example a NullPointerException thrown from your processor + * logic. + * The handler will execute on the thread that produced the exception. + * So inorder to get the thread as the java handler type uses use Thread.currentThread() + * <p> + * Note, this handler must be threadsafe, since it will be shared among all threads, and invoked from any + * thread that encounters such an exception. + * + * @param streamsUncaughtExceptionHandler the uncaught exception handler of type {@link StreamsUncaughtExceptionHandler} for all internal threads + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + * @throws NullPointerException if streamsUncaughtExceptionHandler is null. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final Consumer<Throwable> handler = exception -> handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + Objects.requireNonNull(streamsUncaughtExceptionHandler); + for (final StreamThread thread : threads) { + thread.setStreamsUncaughtExceptionHandler(handler); + } + if (globalStreamThread != null) { + globalStreamThread.setUncaughtExceptionHandler(handler); + } + } else { + throw new IllegalStateException("Can only set UncaughtExceptionHandler in CREATED state. " + + "Current state is: " + state); + } + } + } + + private void handleStreamsUncaughtExceptionDefaultWrapper(final Throwable throwable) { Review comment: how about `defaultStreamsUncaughtExceptionHandler`? ########## File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java ########## @@ -0,0 +1,233 @@ +/* + * 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.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Named; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.StreamsTestUtils; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; +import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION; +import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; + +@Category(IntegrationTest.class) +@SuppressWarnings("deprecation") //Need to call the old handler, will remove those calls when the old handler is removed +public class StreamsUncaughtExceptionHandlerIntegrationTest { + @ClassRule + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + @Rule + public TestName testName = new TestName(); + + private static String inputTopic; + private static StreamsBuilder builder; + private static Properties properties; + private static List<String> processorValueCollector; + private static String appId = ""; + + @Before + public void setup() { + final String testId = safeUniqueTestName(getClass(), testName); + appId = "appId_" + testId; + inputTopic = "input" + testId; + IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic); + + builder = new StreamsBuilder(); + + processorValueCollector = new ArrayList<>(); + + final KStream<String, String> stream = builder.stream(inputTopic); + stream.process(() -> new ShutdownProcessor(processorValueCollector), Named.as("process")); + + properties = mkObjectProperties( + mkMap( + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId), + mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()), + mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2), + mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), + mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class) + ) + ); + } + + @After + public void teardown() throws IOException { + purgeLocalStreamsState(properties); + } + + @Test + public void shouldShutdownThreadUsingOldHandler() throws Exception { + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { + final CountDownLatch latch = new CountDownLatch(1); + final AtomicBoolean flag = new AtomicBoolean(false); + kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true)); + + StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); + + produceMessages(0L, inputTopic, "A"); + latch.await(15, TimeUnit.SECONDS); + + TestUtils.waitForCondition(flag::get, "Handler was called"); + assertThat(processorValueCollector.size(), equalTo(2)); + assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.ERROR)); + } + } + + @Test + public void shouldShutdownClient() throws Exception { + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { + final CountDownLatch latch = new CountDownLatch(1); + kafkaStreams.setUncaughtExceptionHandler((t, e) -> fail("should not hit old handler")); + + kafkaStreams.setUncaughtExceptionHandler(exception -> SHUTDOWN_CLIENT); + + StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); + + produceMessages(0L, inputTopic, "A"); + latch.await(15, TimeUnit.SECONDS); + + assertThat(processorValueCollector.size(), equalTo(1)); + assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.NOT_RUNNING)); + } + } + + @Test + public void shouldShutdownApplication() throws Exception { + final Topology topology = builder.build(); + + try (final KafkaStreams kafkaStreams = new KafkaStreams(topology, properties)) { + final KafkaStreams kafkaStreams1 = new KafkaStreams(topology, properties); + final CountDownLatch latch = new CountDownLatch(1); + kafkaStreams.setUncaughtExceptionHandler((t, e) -> fail("should not hit old handler")); + kafkaStreams1.setUncaughtExceptionHandler((t, e) -> fail("should not hit old handler")); + kafkaStreams.setUncaughtExceptionHandler(exception -> SHUTDOWN_APPLICATION); + kafkaStreams1.setUncaughtExceptionHandler(exception -> SHUTDOWN_APPLICATION); + + kafkaStreams.start(); + kafkaStreams1.start(); + + produceMessages(0L, inputTopic, "A"); + latch.await(30, TimeUnit.SECONDS); + + assertThat(processorValueCollector.size(), equalTo(1)); + assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.ERROR)); + assertThat(kafkaStreams1.state(), equalTo(KafkaStreams.State.ERROR)); + } + } + + @Test + public void shouldShutdownSingleThreadApplication() throws Exception { + final Properties properties = mkObjectProperties( + mkMap( + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId), + mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()), + mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1), Review comment: Agree ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java ########## @@ -311,6 +314,8 @@ public void run() { "Updating global state failed. You can restart KafkaStreams to recover from this error.", recoverableException ); + } catch (final Exception e) { + this.streamsUncaughtExceptionHandler.accept(e); Review comment: There is a logic to use the old handler if the conditions you laid out are true. The odd series of casts of exception types in `handleStreamsUncaughtExceptionDefaultWrapper` are what makes this happen. This is a bit tricky but I think we want to close the client either way. As we don't have plans to replace the global thread and shutting down the application is best effort. We talked about this a while back and we decided the global handler was mainly for information and the return type we would try to follow but we need to make sure we at least close the client. ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -996,6 +1093,60 @@ private boolean close(final long timeoutMs) { } } + private void closeToError() { + if (!setState(State.ERROR)) { + log.info("Skipping shutdown since we are already in " + state()); + } else { + log.info("Transitioning to ERROR state"); Review comment: Everything except the state we leave it in. We can move most of it to a helper ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java ########## @@ -559,18 +552,52 @@ void runLoop() { } } catch (final TaskCorruptedException e) { log.warn("Detected the states of tasks " + e.corruptedTaskWithChangelogs() + " are corrupted. " + - "Will close the task as dirty and re-create and bootstrap from scratch.", e); + "Will close the task as dirty and re-create and bootstrap from scratch.", e); try { taskManager.handleCorruption(e.corruptedTaskWithChangelogs()); } catch (final TaskMigratedException taskMigrated) { handleTaskMigrated(taskMigrated); } } catch (final TaskMigratedException e) { handleTaskMigrated(e); + } catch (final UnsupportedVersionException e) { Review comment: I think this is fine for now. When we add replace thread as an option we can include overrides when handling the response that prevent the thread from being restarted in certain error cases. ########## File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java ########## @@ -0,0 +1,233 @@ +/* + * 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.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.Topology; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.Named; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.StreamsTestUtils; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkObjectProperties; +import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION; +import static org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.fail; + +@Category(IntegrationTest.class) +@SuppressWarnings("deprecation") //Need to call the old handler, will remove those calls when the old handler is removed +public class StreamsUncaughtExceptionHandlerIntegrationTest { + @ClassRule + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + @Rule + public TestName testName = new TestName(); + + private static String inputTopic; + private static StreamsBuilder builder; + private static Properties properties; + private static List<String> processorValueCollector; + private static String appId = ""; + + @Before + public void setup() { + final String testId = safeUniqueTestName(getClass(), testName); + appId = "appId_" + testId; + inputTopic = "input" + testId; + IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic); + + builder = new StreamsBuilder(); + + processorValueCollector = new ArrayList<>(); + + final KStream<String, String> stream = builder.stream(inputTopic); + stream.process(() -> new ShutdownProcessor(processorValueCollector), Named.as("process")); + + properties = mkObjectProperties( + mkMap( + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()), + mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId), + mkEntry(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()), + mkEntry(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 2), + mkEntry(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class), + mkEntry(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.StringSerde.class) + ) + ); + } + + @After + public void teardown() throws IOException { + purgeLocalStreamsState(properties); + } + + @Test + public void shouldShutdownThreadUsingOldHandler() throws Exception { + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { + final CountDownLatch latch = new CountDownLatch(1); + final AtomicBoolean flag = new AtomicBoolean(false); + kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true)); + + StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); + + produceMessages(0L, inputTopic, "A"); + latch.await(15, TimeUnit.SECONDS); + + TestUtils.waitForCondition(flag::get, "Handler was called"); + assertThat(processorValueCollector.size(), equalTo(2)); + assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.ERROR)); + } + } + + @Test + public void shouldShutdownClient() throws Exception { + try (final KafkaStreams kafkaStreams = new KafkaStreams(builder.build(), properties)) { + final CountDownLatch latch = new CountDownLatch(1); + kafkaStreams.setUncaughtExceptionHandler((t, e) -> fail("should not hit old handler")); + + kafkaStreams.setUncaughtExceptionHandler(exception -> SHUTDOWN_CLIENT); + + StreamsTestUtils.startKafkaStreamsAndWaitForRunningState(kafkaStreams); + + produceMessages(0L, inputTopic, "A"); + latch.await(15, TimeUnit.SECONDS); Review comment: That is useful thanks. I went with `waitForApplicationState` ########## File path: streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java ########## @@ -1013,6 +1013,7 @@ public void shouldNotMakeStoreAvailableUntilAllStoresAvailable() throws Exceptio } @Test + @Deprecated //A single thread should no longer die Review comment: same as above ########## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java ########## @@ -1000,7 +1012,17 @@ public void restore(final Map<TaskId, Task> tasks) { CLIENT_ID, new LogContext(""), new AtomicInteger(), - new AtomicLong(Long.MAX_VALUE) + new AtomicLong(Long.MAX_VALUE), + null, + e -> { + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } else if (e instanceof Error) { + throw (Error) e; + } else { + throw new RuntimeException("Unexpected checked exception caught in the uncaught exception handler", e); + } Review comment: Because otherwise the task migrated exception sends it into a endless rebalance ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -366,6 +375,93 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} + * throws an unexpected exception. + * These might be exceptions indicating rare bugs in Kafka Streams, or they + * might be exceptions thrown by your code, for example a NullPointerException thrown from your processor + * logic. + * The handler will execute on the thread that produced the exception. + * So inorder to get the thread as the java handler type uses use Thread.currentThread() + * <p> + * Note, this handler must be threadsafe, since it will be shared among all threads, and invoked from any + * thread that encounters such an exception. + * + * @param streamsUncaughtExceptionHandler the uncaught exception handler of type {@link StreamsUncaughtExceptionHandler} for all internal threads + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + * @throws NullPointerException if streamsUncaughtExceptionHandler is null. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final Consumer<Throwable> handler = exception -> handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + Objects.requireNonNull(streamsUncaughtExceptionHandler); + for (final StreamThread thread : threads) { + thread.setStreamsUncaughtExceptionHandler(handler); + } + if (globalStreamThread != null) { + globalStreamThread.setUncaughtExceptionHandler(handler); + } + } else { + throw new IllegalStateException("Can only set UncaughtExceptionHandler in CREATED state. " + + "Current state is: " + state); + } + } + } + + private void handleStreamsUncaughtExceptionDefaultWrapper(final Throwable throwable) { + if (Thread.getDefaultUncaughtExceptionHandler() != null) { + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } else if (throwable instanceof Error) { + throw (Error) throwable; + } else { + throw new RuntimeException("Unexpected checked exception caught in the uncaught exception handler", throwable); + } + } else { + handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT); + } + } + + private void handleStreamsUncaughtException(final Throwable e, + final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(e); + if (Thread.getDefaultUncaughtExceptionHandler() != null) { Review comment: We can just set a flag through to be safe ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org