lct45 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r514535737



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -282,6 +284,15 @@ public boolean isRunning() {
     private final Admin adminClient;
     private final InternalTopologyBuilder builder;
 
+

Review comment:
       two new lines in a row

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/errors/StreamsUncaughtExceptionHandler.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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
+     */
+    StreamThreadExceptionResponse handle(final Throwable exception);
+
+    /**
+     * Enumeration that describes the response from the exception handler.
+     */
+    enum StreamThreadExceptionResponse {
+        //        REPLACE_THREAD(0, "REPLACE_THREAD"),

Review comment:
       Supposed to be here?

##########
File path: streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java
##########
@@ -616,13 +623,22 @@ public void 
shouldNotSetGlobalRestoreListenerAfterStarting() {
     public void 
shouldThrowExceptionSettingUncaughtExceptionHandlerNotInCreateState() {
         final KafkaStreams streams = new 
KafkaStreams(getBuilderWithSource().build(), props, supplier, time);
         streams.start();
-        try {
-            streams.setUncaughtExceptionHandler(null);
-            fail("Should throw IllegalStateException");
-        } catch (final IllegalStateException e) {
-            // expected
-        }
+        assertThrows(IllegalStateException.class, () -> 
streams.setUncaughtExceptionHandler((StreamsUncaughtExceptionHandler) null));
+    }
+
+    @Test
+    public void 
shouldThrowExceptionSettingStreamsUncaughtExceptionHandlerNotInCreateState() {
+        final KafkaStreams streams = new 
KafkaStreams(getBuilderWithSource().build(), props, supplier, time);
+        streams.start();
+        assertThrows(IllegalStateException.class, () -> 
streams.setUncaughtExceptionHandler((StreamsUncaughtExceptionHandler) null));
+
     }
+    @Test
+    public void 
shouldThrowNullPointerExceptionSettingStreamsUncaughtExceptionHandlerIfNull() {
+        final KafkaStreams streams = new 
KafkaStreams(getBuilderWithSource().build(), props, supplier, time);
+        assertThrows(NullPointerException.class, () -> 
streams.setUncaughtExceptionHandler((StreamsUncaughtExceptionHandler) null));
+    }
+

Review comment:
       extra line

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -346,26 +351,89 @@ public void setStateListener(final 
KafkaStreams.StateListener listener) {
      * Set the handler invoked when a {@link 
StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} abruptly
      * terminates due to an uncaught exception.
      *
-     * @param eh the uncaught exception handler for all internal threads; 
{@code null} deletes the current handler
+     * @param uncaughtExceptionHandler the uncaught exception handler 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}.
+     *
+     * @Deprecated Since 2.7.0. Use {@link 
KafkaStreams#setUncaughtExceptionHandler(StreamsUncaughtExceptionHandler)} 
instead.
+     *
      */
-    public void setUncaughtExceptionHandler(final 
Thread.UncaughtExceptionHandler eh) {
+    public void setUncaughtExceptionHandler(final 
Thread.UncaughtExceptionHandler uncaughtExceptionHandler) {
         synchronized (stateLock) {
             if (state == State.CREATED) {
                 for (final StreamThread thread : threads) {
-                    thread.setUncaughtExceptionHandler(eh);
+                    
thread.setUncaughtExceptionHandler(uncaughtExceptionHandler);
                 }
 
                 if (globalStreamThread != null) {
-                    globalStreamThread.setUncaughtExceptionHandler(eh);
+                    
globalStreamThread.setUncaughtExceptionHandler(uncaughtExceptionHandler);
                 }
             } else {
                 throw new IllegalStateException("Can only set 
UncaughtExceptionHandler in CREATED state. " +
-                    "Current state is: " + state);
+                        "Current state is: " + state);
             }
         }
     }
 
+    /**
+     * 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}.
+     * @throws NullPointerException @NotNull 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 REPLACE_STREAM_THREAD:

Review comment:
       Is this section going to be re-added after the other thread handling 
stuff gets figured out?

##########
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,280 @@
+/*
+ * 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.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.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 StreamsHandlerIntegrationTest {
+    @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);
+

Review comment:
       line!

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -567,10 +590,46 @@ void runLoop() {
                 }
             } catch (final TaskMigratedException e) {
                 handleTaskMigrated(e);
+            } catch (final Exception e) {
+                if (this.streamsUncaughtExceptionHandler == null) {
+                    throw e;
+                }
+                if (Thread.getDefaultUncaughtExceptionHandler() != null && 
newHandler) {
+                    log.error("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.");
+                } else {
+                    throw e;
+                }
+                if (this.streamsUncaughtExceptionHandler.handle(e) != 
StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION)
 {
+                    throw e;
+                }
             }
         }
     }
 
+    /**
+     * Sets the streams uncaught exception handler.
+     *
+     * @param streamsUncaughtExceptionHandler the user handler wrapped in 
shell to execute the action
+     */
+    public void setStreamsUncaughtExceptionHandler(final 
StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        this.streamsUncaughtExceptionHandler = streamsUncaughtExceptionHandler;
+        this.newHandler = true;
+    }
+

Review comment:
       extra line

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -567,10 +590,46 @@ void runLoop() {
                 }
             } catch (final TaskMigratedException e) {
                 handleTaskMigrated(e);
+            } catch (final Exception e) {
+                if (this.streamsUncaughtExceptionHandler == null) {
+                    throw e;
+                }
+                if (Thread.getDefaultUncaughtExceptionHandler() != null && 
newHandler) {
+                    log.error("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.");
+                } else {
+                    throw e;
+                }
+                if (this.streamsUncaughtExceptionHandler.handle(e) != 
StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION)
 {
+                    throw e;
+                }
             }
         }
     }
 
+    /**
+     * Sets the streams uncaught exception handler.
+     *
+     * @param streamsUncaughtExceptionHandler the user handler wrapped in 
shell to execute the action
+     */
+    public void setStreamsUncaughtExceptionHandler(final 
StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        this.streamsUncaughtExceptionHandler = streamsUncaughtExceptionHandler;
+        this.newHandler = true;
+    }
+
+
+    public void shutdownToError() {
+        shutdownErrorHook.shutdown();
+    }
+
+    public void sendShutdownRequest(final AssignorError assignorError) {
+        log.warn("Detected that shutdown was requested. " +
+                "The all clients in this app will now begin to shutdown");
+        assignmentErrorCode.set(assignorError.code());
+        mainConsumer.enforceRebalance();
+    }
+

Review comment:
       extra line (: 




----------------------------------------------------------------
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


Reply via email to