wcarlson5 commented on a change in pull request #9273: URL: https://github.com/apache/kafka/pull/9273#discussion_r503568832
########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -931,6 +1033,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) { return close(timeoutMs); } + private void closeToError() { Review comment: This does that the same thing as close except state change. A user can still call close and it will change the error. There was talk about making Error terminal in that case this will not change. ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -364,6 +370,98 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when a {@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; {@code null} deletes the current handler + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler handler = throwable -> handleStreamsUncaughtException(throwable, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + for (final StreamThread thread : threads) { + if (streamsUncaughtExceptionHandler != null) { + thread.setStreamsUncaughtExceptionHandler(handler); + } else { + final StreamsUncaughtExceptionHandler defaultHandler = exception -> + StreamsUncaughtExceptionHandler.StreamsUncaughtExceptionHandlerResponse.SHUTDOWN_STREAM_THREAD; + thread.setStreamsUncaughtExceptionHandler(defaultHandler); + } + + } + if (globalStreamThread != null && streamsUncaughtExceptionHandler != null) { + globalStreamThread.setUncaughtExceptionHandler(new StreamsUncaughtExceptionHandler() { + @Override + public StreamsUncaughtExceptionHandlerResponse handle(final Throwable exception) { + return handler.handle(exception); + } + + @Override + public StreamsUncaughtExceptionHandlerResponseGlobalThread handleExceptionInGlobalThread(final Throwable exception) { + return handleStreamsUncaughtExceptionGlobalThread(exception, streamsUncaughtExceptionHandler); + } + }); + } + } else { + throw new IllegalStateException("Can only set UncaughtExceptionHandler in CREATED state. " + + "Current state is: " + state); + } + } + } + + private StreamsUncaughtExceptionHandler.StreamsUncaughtExceptionHandlerResponseGlobalThread handleStreamsUncaughtExceptionGlobalThread(final Throwable e, + final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler.StreamsUncaughtExceptionHandlerResponseGlobalThread action = streamsUncaughtExceptionHandler.handleExceptionInGlobalThread(e); + switch (action) { Review comment: currently there is only one option but there may be more added with 406 so I will make as easy to extend as I can ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -780,11 +874,13 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder, cacheSizePerThread, stateDirectory, delegatingStateRestoreListener, - i + 1); + i + 1, + new AtomicInteger(), + shutdownErrorHook + ); threadState.put(threads[i].getId(), threads[i].state()); storeProviders.add(new StreamThreadStateStoreProvider(threads[i], internalTopologyBuilder)); } - Review comment: removed 3 empty lines to keep the constructor under the line limit ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -364,6 +370,73 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when a {@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; {@code null} deletes the current handler + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler handler = throwable -> handleStreamsUncaughtException(throwable, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + for (final StreamThread thread : threads) { + if (streamsUncaughtExceptionHandler != null) { Review comment: Changed this to require not null ########## File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java ########## @@ -226,7 +226,9 @@ private StreamThread createStreamThread(@SuppressWarnings("SameParameterValue") 0, stateDirectory, new MockStateRestoreListener(), - threadIdx + threadIdx, + new AtomicInteger(), + () -> { } Review comment: This is an empty shutdown hook. I think it is better than null but I am not sure ########## File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java ########## @@ -364,6 +370,73 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } + /** + * Set the handler invoked when a {@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; {@code null} deletes the current handler + * @throws IllegalStateException if this {@code KafkaStreams} instance is not in state {@link State#CREATED CREATED}. + */ + public void setUncaughtExceptionHandler(final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) { + final StreamsUncaughtExceptionHandler handler = throwable -> handleStreamsUncaughtException(throwable, streamsUncaughtExceptionHandler); + synchronized (stateLock) { + if (state == State.CREATED) { + for (final StreamThread thread : threads) { Review comment: We ended up going with adding a new method to the handler ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java ########## @@ -281,6 +285,14 @@ public boolean isRunning() { private final Admin adminClient; private final InternalTopologyBuilder builder; + private final AtomicInteger assignmentErrorCode; + + private final ShutdownErrorHook shutdownErrorHook; + + public interface ShutdownErrorHook { Review comment: in order to let the rebalance listen shutdown with an error we just use this so we don't have to pass the KafkaStreams object and make a new public method ########## File path: streams/src/main/java/org/apache/kafka/streams/errors/StreamsUncaughtExceptionHandler.java ########## @@ -0,0 +1,73 @@ +/* + * 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.errors; + +public interface StreamsUncaughtExceptionHandler { + /** + * Inspect the exception received in a stream thread and respond with an action. + * @param exception the actual exception + */ + StreamsUncaughtExceptionHandler.StreamsUncaughtExceptionHandlerResponse handle(final Throwable exception); Review comment: By making this not default we allow the handler to be over written by a lambda, this means we have to provide a default in the constructor but I think that is worth it. If a user only wants to set the global handler they will have to set this as well but if they are doing that I assume they only have a global thread and in that case it does not matter what the handle function is as it will never be called. ########## File path: streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java ########## @@ -617,7 +619,19 @@ public void shouldThrowExceptionSettingUncaughtExceptionHandlerNotInCreateState( final KafkaStreams streams = new KafkaStreams(getBuilderWithSource().build(), props, supplier, time); streams.start(); try { - streams.setUncaughtExceptionHandler(null); + streams.setUncaughtExceptionHandler((Thread.UncaughtExceptionHandler) null); + fail("Should throw IllegalStateException"); + } catch (final IllegalStateException e) { + // expected + } + } + + @Test + public void shouldThrowExceptionSettingStreamsUncaughtExceptionHandlerNotInCreateState() { Review comment: That makes sense. Thank you for clarifying. I have added the null test ########## File path: streams/src/test/java/org/apache/kafka/streams/integration/AppShutdownIntegrationTest.java ########## @@ -0,0 +1,279 @@ +/* + * 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.errors.StreamsUncaughtExceptionHandler; +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.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 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.StreamsUncaughtExceptionHandlerResponse.SHUTDOWN_KAFKA_STREAMS_APPLICATION; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest; +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) +public class AppShutdownIntegrationTest { + @ClassRule + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + @Rule + public TestName testName = new TestName(); + + String inputTopic; + StreamsBuilder builder; + Properties properties; + List<String> processorValueCollector; + String idempotentTopic = "idempotentTopic"; + String appId = ""; + + @Before + public void setup() { + final String testId = safeUniqueTestName(getClass(), testName); + appId = "appId_" + testId; + inputTopic = "input" + testId; + cleanStateBeforeTest(CLUSTER, idempotentTopic, inputTopic); + + + 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_STANDBY_REPLICAS_CONFIG, "5"), + mkEntry(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, "6"), + mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "7"), + mkEntry(StreamsConfig.PROBING_REBALANCE_INTERVAL_MS_CONFIG, "480000"), + 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 shouldShutdownThread() throws Exception { + // + // + // Also note that this is an integration test because so many components have to come together to + // ensure these configurations wind up where they belong, and any number of future code changes + // could break this change. + + 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 -> StreamsUncaughtExceptionHandler.StreamsUncaughtExceptionHandlerResponse.SHUTDOWN_STREAM_THREAD); + + kafkaStreams.start(); + + produceMessages(0L, inputTopic, "A"); + latch.await(15, TimeUnit.SECONDS); + + assertThat(processorValueCollector.size(), equalTo(2)); + assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.ERROR)); Review comment: This will be running when we make it so that no threads stays in running in KIP-663 ---------------------------------------------------------------- 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