You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/10/23 20:46:01 UTC

[GitHub] [kafka] wcarlson5 opened a new pull request #9487: KAFKA-9331 add a streams handler

wcarlson5 opened a new pull request #9487:
URL: https://github.com/apache/kafka/pull/9487


   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331 add a streams handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r511149017



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -346,26 +351,92 @@ 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:
+//                log.error("Encountered the following exception during processing " +
+//                        "and the the stream thread will be replaced: ", e);
+//            this.addStreamsThread();
+//                break;
+            case SHUTDOWN_CLIENT:
+                log.error("Encountered the following exception during processing " +
+                        "and the client is going to shut down: ", e);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (e instanceof Error) {
+                    log.error("This option requires the thread to stay running to start the shutdown." +
+                            "Therefore it is not suitable for Error types.");
+                }
+//                for (final StreamThread streamThread: threads) {

Review comment:
       moved into stream thread because of a concurrent operation exception that appeared




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523089743



##########
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:
       If that's the case, then we really should just set a flag on KafkaStreams to indicate whether that handler has been set.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524824649



##########
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:
       Mm ok actually I think this should be fine. I was thinking of the handler as just "swallowing" the exception, but in reality the user would still let the current thread die and just spin up a new one in its place. And then the new one would hit this UnsupportedVersionException and so on, until the brokers are upgraded. So there shouldn't be any way to get into a bad state




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524490211



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHanlder) {
+            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 throwable,
+                                                final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(throwable);
+        if (oldHanlder) {
+            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. ", throwable);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (throwable 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.", throwable);
+                }
+                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();

Review comment:
       It doesn't really matter to me, though I think that non blocking is probably  preferable.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525169791



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread use that threw the exception, Thread.currentThread().

Review comment:
       There is something wrong in this sentence.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread use that threw the exception, 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHanlder) {

Review comment:
       `oldHanlder` -> `oldHandler`

##########
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) {
+                final String errorMessage = e.getMessage();
+                if (errorMessage != null &&
+                        errorMessage.startsWith("Broker unexpectedly doesn't support requireStable flag on version ")) {
+
+                    log.error("Shutting down because the Kafka cluster seems to be on a too old version. " +
+                                    "Setting {}=\"{}\" requires broker version 2.5 or higher.",
+                            StreamsConfig.PROCESSING_GUARANTEE_CONFIG,
+                            EXACTLY_ONCE_BETA);
+

Review comment:
       nit: remove 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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517464968



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java
##########
@@ -19,7 +19,7 @@
 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;

Review comment:
       Could you please also add the needed changes to system test `streams_upgrade_test.py::StreamsUpgradeTest.test_version_probing_upgrade` to this PR.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523322776



##########
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:
       But TaskMigratedException should never be thrown all the way up to the exception handler. Is that what you're seeing?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331 add a streams handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r511148601



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -782,7 +849,12 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder,
                 cacheSizePerThread,
                 stateDirectory,
                 delegatingStateRestoreListener,
-                i + 1);
+                i + 1,
+                KafkaStreams.this::close,

Review comment:
       This will call closeToError but I am testing if that has a problem. So far it does not




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#issuecomment-729475252


   Merged to trunk 🥳 


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331 add a streams handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r512320447



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -659,7 +727,6 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder,
                          final Time time) throws StreamsException {
         this.config = config;
         this.time = time;
-

Review comment:
       Method was a few lines too long




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518335631



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

Review comment:
       it can be removed




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518840602



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ReferenceContainer.java
##########
@@ -30,7 +30,7 @@
     public Admin adminClient;
     public TaskManager taskManager;
     public StreamsMetadataState streamsMetadataState;
-    public final AtomicInteger assignmentErrorCode = new AtomicInteger();
+    public AtomicInteger assignmentErrorCode = new AtomicInteger();

Review comment:
       I was changing it intentionally but I think I can get away with not




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r520077048



##########
File path: streams/src/main/resources/common/message/SubscriptionInfoData.json
##########
@@ -57,6 +57,11 @@
       "name": "uniqueField",
       "versions": "8+",
       "type": "int8"
+    },
+    {
+      "name": "shutdownRequested",
+      "versions": "9+",
+      "type": "int8"

Review comment:
       I'm not really worried that we'd run out of space, I just think it sends a signal that the Assignment and Subscription error codes are semantically distinct and don't refer to the same underlying concept. So it seems better to go with the simpler approach than over-optimize to save an occasional three bytes 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518938852



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());

Review comment:
       That is a good idea, Ill change the log




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518837250



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -567,10 +589,34 @@ void runLoop() {
                 }
             } catch (final TaskMigratedException e) {
                 handleTaskMigrated(e);
+            } catch (final Exception e) {
+                if (this.streamsUncaughtExceptionHandler.handle(e)) {

Review comment:
       We could do the logic inline how ever this does make it slightly simpler. Also we only expose the `streamsUncaughtExceptionHandler` to the user and @vvcephei had a problem with the wrapping that again with the same type. So we introduced a wrapper class. if we renamed it from `Handler` to `streamsUncaughtExceptionHandlerWrapper` would that make it more clear?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517621757



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -294,7 +304,10 @@ public static StreamThread create(final InternalTopologyBuilder builder,
                                       final long cacheSizeBytes,
                                       final StateDirectory stateDirectory,
                                       final StateRestoreListener userStateRestoreListener,
-                                      final int threadIdx) {
+                                      final int threadIdx,
+                                      final ShutdownErrorHook shutdownErrorHook,
+                                      final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler,
+                                      final AtomicInteger assignmentErrorCode) {

Review comment:
       You are right it seems that it is not necessary 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518478117



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -567,10 +589,34 @@ void runLoop() {
                 }
             } catch (final TaskMigratedException e) {
                 handleTaskMigrated(e);
+            } catch (final Exception e) {
+                if (this.streamsUncaughtExceptionHandler.handle(e)) {

Review comment:
       I had a little trouble following the `Handler` class. Some trivial things -- eg the handler in the StreamThread is named `streamsUncaughtExceptionHandler` but it's actually _not_ a `StreamsUncaughtExceptionHandler`. Also the usage of the return value; IIUC it's supposed to indicate whether to use the new handler or fall back on the old one. To me it sounds like if `handle` returns `true` that means we should handle it, ie we should _not_ rethrow the exception, but this looks like the opposite of what we do now. Honestly either interpretation is ok with me, as long as it's documented somewhere
   
   Do we really need the `Handler` in the first place though? It's already pretty confusing that we have to deal with two types of handlers (old and new) so I'd prefer not to add a third unless it's really necessary. It seems like we can just inline the logic of whether to invoke the new handler or rethrow the exception, which would also clear up the confusion around the meaning of the return value. But I might be missing something here -- WDYT?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -282,6 +283,17 @@ public boolean isRunning() {
     private final Admin adminClient;
     private final InternalTopologyBuilder builder;
 
+    private Handler streamsUncaughtExceptionHandler;
+    private ShutdownErrorHook shutdownErrorHook;
+    private AtomicInteger assignmentErrorCode;
+    public interface ShutdownErrorHook {
+        void shutdown();
+    }

Review comment:
       Seems like we can just pass in a Runnable with `KafkaStreams::closeToError` instead of adding a whole `ShutdownErrorHook` functional interface

##########
File path: streams/src/main/resources/common/message/SubscriptionInfoData.json
##########
@@ -57,6 +57,11 @@
       "name": "uniqueField",
       "versions": "8+",
       "type": "int8"
+    },
+    {
+      "name": "shutdownRequested",
+      "versions": "9+",
+      "type": "int8"

Review comment:
       I think we should mirror the `errorCode` in the AssignmentInfo here, both in terms of naming and type. If we're going to use the same AssignorError for both, then they should really be the same. And we may want to send other kinds of error codes in the subscription going forward: better to just encode a single `int` than a separate `byte` for every logical error code. I don't think we'll notice the extra three bytes since Subscriptions aren't sent that frequently

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ReferenceContainer.java
##########
@@ -30,7 +30,7 @@
     public Admin adminClient;
     public TaskManager taskManager;
     public StreamsMetadataState streamsMetadataState;
-    public final AtomicInteger assignmentErrorCode = new AtomicInteger();
+    public AtomicInteger assignmentErrorCode = new AtomicInteger();

Review comment:
       This should probably stay `final` so we don't accidentally change it ever

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());

Review comment:
       Should this be logged at error?

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());
+        } else {
+            log.info("Transitioning to ERROR state");
+            stateDirCleaner.shutdownNow();
+            if (rocksDBMetricsRecordingService != null) {
+                rocksDBMetricsRecordingService.shutdownNow();
+            }
+
+            // wait for all threads to join in a separate thread;
+            // save the current thread so that if it is a stream thread
+            // we don't attempt to join it and cause a deadlock
+            final Thread shutdownThread = new Thread(() -> {
+                // notify all the threads to stop; avoid deadlocks by stopping any
+                // further state reports from the thread since we're shutting down
+                for (final StreamThread thread : threads) {
+                    thread.shutdown();
+                }
+
+                for (final StreamThread thread : threads) {
+                    try {
+                        if (!thread.isRunning()) {
+                            thread.join();
+                        }
+                    } catch (final InterruptedException ex) {
+                        Thread.currentThread().interrupt();
+                    }
+                }
+
+                if (globalStreamThread != null) {
+                    globalStreamThread.shutdown();
+                }
+
+                if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
+                    try {
+                        globalStreamThread.join();
+                    } catch (final InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
+                    globalStreamThread = null;
+                }
+
+                adminClient.close();
+
+                streamsMetrics.removeAllClientLevelMetrics();
+                metrics.close();
+                setState(State.ERROR);
+            }, "kafka-streams-close-thread");
+
+            shutdownThread.setDaemon(true);
+            shutdownThread.start();
+            setState(State.ERROR);

Review comment:
       Looks like we call `setState(ERROR)` three times in this method, is that intentional?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -567,10 +589,34 @@ void runLoop() {
                 }
             } catch (final TaskMigratedException e) {
                 handleTaskMigrated(e);
+            } catch (final Exception e) {
+                if (this.streamsUncaughtExceptionHandler.handle(e)) {
+                    throw e;
+                }
             }
         }
     }
 
+    /**
+     * Sets the streams uncaught exception handler.
+     *
+     * @param streamsUncaughtExceptionHandler the user handler wrapped in shell to execute the action
+     */
+    public void setStreamsUncaughtExceptionHandler(final Handler streamsUncaughtExceptionHandler) {
+        this.streamsUncaughtExceptionHandler = streamsUncaughtExceptionHandler;
+    }
+
+    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");

Review comment:
       ```suggestion
                   "All clients in this app will now begin to shutdown");
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -255,8 +255,9 @@ public ByteBuffer subscriptionUserData(final Set<String> topics) {
             taskManager.processId(),
             userEndPoint,
             taskManager.getTaskOffsetSums(),
-            uniqueField)
-                .encode();
+            uniqueField,
+            (byte) assignmentErrorCode.get()

Review comment:
       This cast makes me kind of uncomfortable...either the `assignmentErrorCode` that we have in the AssignmentInfo is conceptually the same as the one we're adding to the SubscriptionInfo (in which case it should be the same type), or it's not the same, in which case we should use a different variable to track it.
   
   Personally I think it's probably simpler to keep them the same, and just add an `int` errorCode field to the Subscription instead of a `byte` shutdownRequested field. 

##########
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:
       It probably doesn't matter too much since `handleRebalanceComplete` doesn't do anything that important at the mometn, but it seems like we should call it before shutting down, not after.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +373,84 @@ 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 StreamThread.Handler 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 boolean handleStreamsUncaughtExceptionDefaultWrapper(final Throwable throwable) {
+        if (Thread.getDefaultUncaughtExceptionHandler() != null) {
+            return true;
+        }
+        return handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT);
+    }
+
+    private boolean 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 stream thread cause the application to attempt to shutdown." +
+                            " This action will succeed only if there is at least one StreamThread running on ths client." +
+                            " Currently there is no running threads so will now close the client.");

Review comment:
       ```suggestion
                       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.");
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517576758



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
##########
@@ -311,6 +317,22 @@ public void run() {
                 "Updating global state failed. You can restart KafkaStreams to recover from this error.",
                 recoverableException
             );
+        } 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.");

Review comment:
       I think it is simpler to check in the Stream thread because we don't in KafkaStreams if the handlers have been set so we would have to check the stream thread a global thread so it would be much easier to just check in the thread. I do agree that it should be bumped down to warn through.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517485257



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -997,6 +1064,72 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            // if transition failed, it means it was either in PENDING_SHUTDOWN
+            // or NOT_RUNNING already; just check that all threads have been stopped
+            log.info("Can not close to error from state " + state());

Review comment:
       That works




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524487609



##########
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:
       That is probably fine. We can really get into it when we add the replace option, as now all calls to the handler are fatal.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] lct45 commented on a change in pull request #9487: KAFKA-9331 add a streams handler

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525692960



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }

Review comment:
       Eh, I wouldn't bother with an AK ticket if this will be tackled in the next PR. I'll just make a list of all the minor followup work somewhere to keep track




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523375614



##########
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:
       Oh you're totally right, sorry for letting my paranoia start spreading conspiracy theories here 🙂  Given all this I'd still claim that the FSM is in need to being cleaned up a bit (or a lot), but if you'd prefer to hold off on that until the add thread work then I'm all good here. Thanks for humoring me and explaining the state of things. I just wanted/want to make sure we don't overlook anything, since there's a lot going on.
   
   For example in the current code, if the global thread dies with the old handler still in use then we'll transition to ERROR. However the user still has to be responsible for closing the client themselves, and it will ultimately transition from ERROR to NOT_RUNNING. Whereas if we transition to ERROR as the result of a SHUTDOWN_APPLICATION error code, the user should NOT try to invoke close themselves, and the ERROR state will be terminal. That's pretty confusing eg for users who use a state listener and wait for the transition to ERROR to call close(). We should make sure that ERROR has the same semantics across the board by the end of all this work.
   
   Anyways I'm just thinking out loud here, to reiterate I'm perfectly happy to merge this as-is. But for reasons like the above, I think it's important to tackle the FSM in the next PR and make sure it all gets sorted out by the next AK release




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523337226



##########
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:
       Not quite. If I remove the handler and just run it there is an illegal state exception which runs endlessly until the handler can exit the loop. It looks like the thread hadn't started all the way before the TaskMigratedExcpetion is thrown
   
   `INFO State transition from STARTING to PENDING_SHUTDOWN (org.apache.kafka.streams.processor.internals.StreamThread:223)
   [`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525734417



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }
 
-                if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
-                    try {
-                        globalStreamThread.join();
-                    } catch (final InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                    globalStreamThread = null;
+            if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
+                try {
+                    globalStreamThread.join();
+                } catch (final InterruptedException e) {
+                    Thread.currentThread().interrupt();
                 }
+                globalStreamThread = null;
+            }
 
-                adminClient.close();
+            adminClient.close();
 
-                streamsMetrics.removeAllClientLevelMetrics();
-                metrics.close();
+            streamsMetrics.removeAllClientLevelMetrics();
+            metrics.close();
+            if (!error) {
                 setState(State.NOT_RUNNING);
-            }, "kafka-streams-close-thread");
+            }
+        }, "kafka-streams-close-thread");
+    }
+
+    private boolean close(final long timeoutMs) {
+        if (!setState(State.PENDING_SHUTDOWN)) {

Review comment:
       WDYT about having both NOT_RUNNING and ERROR go through PENDING_SHUTDOWN, rather than just transitioning directly and permanently to ERROR? At a high level I think it just makes sense for ERROR and NOT_RUNNING to be symmetric. Also any benefit to having an intermediate PENDING_SHUTDOWN for the NOT_RUNNING case presumably applies to the ERROR case as well. eg, it indicates whether Streams has completed its shutdown or not: users know that an app in PENDING_SHUTDOWN should never be killed, its only safe to do so once it reaches NOT_RUNNING. We should provide the same functionality and only transition to ERROR after the shutdown is complete




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518371511



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
##########
@@ -60,6 +60,9 @@ 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()) {

Review comment:
       added unit test




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331 add a streams handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r514566700



##########
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:
       It will. I don't know if we should merge as comment or just add it later




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525442248



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread use that threw the exception, 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHanlder) {

Review comment:
       oops




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525444958



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread use that threw the exception, Thread.currentThread().

Review comment:
       need to remove `use`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523302976



##########
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:
       Ah ok I thought we executed this cleanup logic in the GlobalStreamThread's `shutdown` method but now I see that's not true. Sorry for the confusion there.
   I do see some minor outstanding issues here, mainly around the state diagram. Let's say the user opts to `SHUTDOWN_CLIENT` in the new handler: the intended semantics are to end up in `NOT_RUNNING` 
   But I think what would happen is that from the global thread we would immediately call `KafkaStreams#close` , which kicks off a shutdown thread to wait for all threads to join and then sets the state to `NOT_RUNNING`. Then when the handler returns, it would transition the global thread to `PENDING_SHUTDOWN` and then finally to `DEAD`. And during the transition to `DEAD`, we would actually end up transitioning the KafkaStreams instance to `ERROR`, rather than `NOT_RUNNING` as intended. So probably, we just need to update the `onChange` method in KafkaStreams.
   This also reminds me of another thing, we need to update the FSM diagram and allowed transitions in KafkaStreams to reflect the new semantics we decided on for ERROR (which IIRC is basically just to make it a terminal state). Does that sound right to you?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525658639



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

Review comment:
       Hm ok this might be a problem. Since this is thrown from another catch block and not from the try block, it won't be caught by the catch block below and will slip through the exception handler. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523028163



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

Review comment:
       nit: usually we indent 4 spaces, not 8.

##########
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:
       I guess, you wanted to do this
   ```suggestion
                           "and the registered exception handler opted to " + action + "." +
   ```
   

##########
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:
       Are you sure this is the correct method to call? As far as I understand the the javadocs and the decompiled code, this method does not return the handler you can set on a `Thread` with `setUncaughtExceptionHandler()`.

##########
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:
       I still have a question here. Since the stream thread is alive when it calls `close()` there will not be a deadlock anymore. So, why do we call `close()` with duration zero?

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

Review comment:
       Please use a more meaningful parameter name.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518264927



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -997,6 +1061,72 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            // if transition failed, it means it was either in PENDING_SHUTDOWN
+            // or NOT_RUNNING already; just check that all threads have been stopped

Review comment:
       I don't think we actually need it either way so I will just remove it




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525636554



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread that threw the exception, 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHandler) {
+            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 throwable,
+                                                final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(throwable);
+        if (oldHandler) {
+            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. ", throwable);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (throwable 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.", throwable);
+                }
+                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);

Review comment:
       I think it makes more sense to transition to ERROR in this case than to NOT_RUNNING. But let's put this on file with the other FSM-related work planned for following PRs




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518838421



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());

Review comment:
       In the normal close method the corresponding log is also info. As multiple thread will be calling this at once I would rather not flood the logs with error unnecessarily.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525640088



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }

Review comment:
       Why do we shut down the global thread only after all stream threads have completed their shutdown? Seems like it would be more efficient to send the shutdown signal to everyone first, and then wait for all the threads to join. Can you try this out in the followup PR?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524819063



##########
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:
       Just to clarify I think it's ok to leave this as-is for now, since as Walker said all handler options are fatal at this point 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518838586



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());
+        } else {
+            log.info("Transitioning to ERROR state");
+            stateDirCleaner.shutdownNow();
+            if (rocksDBMetricsRecordingService != null) {
+                rocksDBMetricsRecordingService.shutdownNow();
+            }
+
+            // wait for all threads to join in a separate thread;
+            // save the current thread so that if it is a stream thread
+            // we don't attempt to join it and cause a deadlock
+            final Thread shutdownThread = new Thread(() -> {
+                // notify all the threads to stop; avoid deadlocks by stopping any
+                // further state reports from the thread since we're shutting down
+                for (final StreamThread thread : threads) {
+                    thread.shutdown();
+                }
+
+                for (final StreamThread thread : threads) {
+                    try {
+                        if (!thread.isRunning()) {
+                            thread.join();
+                        }
+                    } catch (final InterruptedException ex) {
+                        Thread.currentThread().interrupt();
+                    }
+                }
+
+                if (globalStreamThread != null) {
+                    globalStreamThread.shutdown();
+                }
+
+                if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
+                    try {
+                        globalStreamThread.join();
+                    } catch (final InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
+                    globalStreamThread = null;
+                }
+
+                adminClient.close();
+
+                streamsMetrics.removeAllClientLevelMetrics();
+                metrics.close();
+                setState(State.ERROR);
+            }, "kafka-streams-close-thread");
+
+            shutdownThread.setDaemon(true);
+            shutdownThread.start();
+            setState(State.ERROR);

Review comment:
       No, I hadn't seen that




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518840121



##########
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:
       We can do that, it doesn't seem make difference which order it is called. However if it is not called it will get stuck continually rebalancing. We return because setting the state to partitions assigned will cause an error




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525686843



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

Review comment:
       like in stream thread we can just add a call to the handler




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525678234



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }

Review comment:
       You are right I think. I just copied from the normal close method because I knew it worked. In a follow up we can maybe change both of these. Do you think that there should be a ak ticket to track it?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525680874



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread that threw the exception, 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHandler) {
+            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 throwable,
+                                                final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(throwable);
+        if (oldHandler) {
+            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. ", throwable);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (throwable 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.", throwable);
+                }
+                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);

Review comment:
       I am on the fence about this. I do think its would be consistent to be not running but also it did shutdown cleanly. We made this choice when ERROR still meant all threads had died and that is not true now. In the end I just went with what we had in the KIP rather than try to change it. Though I could be swayed to leave this in ERROR.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524437940



##########
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:
       +1 to sorting out FSM before next release, I have a ticket to track the work. I started to change it and it ballooned out to be much more expansive than I thought. This PR is already complicated enough, so we can add is later.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#issuecomment-764961250


   @wcarlson5 Seems we forgot the update the docs for this KIP. Can you do a follow up PR? This follow up PR should also cover the changes of https://github.com/apache/kafka/pull/9487 that is part if the same KIP.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523303062



##########
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:
       I suspect the tests didn't catch this because we would still transition out of ERROR to PENDING_SHUTDOWN and finally NOT_RUNNING in this case. But really, we shouldn't transition to ERROR in the first place




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517481718



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +374,66 @@ 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}.
+     * @throws NullPointerException @NotNull if streamsUncaughtExceptionHandler is null.

Review comment:
       I don't remember putting it there so it was probably a mistake




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r520104884



##########
File path: streams/src/main/resources/common/message/SubscriptionInfoData.json
##########
@@ -57,6 +57,11 @@
       "name": "uniqueField",
       "versions": "8+",
       "type": "int8"
+    },
+    {
+      "name": "shutdownRequested",
+      "versions": "9+",
+      "type": "int8"

Review comment:
       https://github.com/apache/kafka/pull/9273#discussion_r486597512
   I originally had it at int32, but john suggested int16, now it is int8.
   
   would you be good with int16 or do you think int32 is the way?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523327338



##########
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:
       Well it's not exactly a default, technically this method is always used to decide which handler to invoke (which may or may not invoke a default handler). Any of these would be fine by me but I'll throw one more idea out there: `invokeOldOrNewUncaughtExceptionHandler` 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mjsax edited a comment on pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
mjsax edited a comment on pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#issuecomment-764961250


   @wcarlson5 Seems we forgot the update the docs for this KIP. Can you do a follow up PR? This follow up PR should also cover the changes of https://issues.apache.org/jira/browse/KAFKA-10810 that is part if the same KIP.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r520104884



##########
File path: streams/src/main/resources/common/message/SubscriptionInfoData.json
##########
@@ -57,6 +57,11 @@
       "name": "uniqueField",
       "versions": "8+",
       "type": "int8"
+    },
+    {
+      "name": "shutdownRequested",
+      "versions": "9+",
+      "type": "int8"

Review comment:
       https://github.com/apache/kafka/pull/9273#discussion_r486597512
   I originally had it at int32, but @vvcephei suggested int16, now it is int8.
   
   would you be good with int16 or do you think int32 is the way?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525161434



##########
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:
       >  I thought we would still be in danger of deadlock if we use the blocking close(), since close() will not return until every thread has joined but the StreamThread that called close() would be stuck in this blocking call and thus never stop/join
   
   OK, I think you are right. I focused too much on 
   
   ```
   if (!thread.isRunning()) {
       thread.join();
   }
   ```
   
   without considering that before the stream threads are shutdown which makes them not running.
   
   In the meantime, I understood a bit better the motivation of the shutdown thread in `close()`. The shutdown thread ensures that the timeout is still consiered in case `close()` is called by a stream thread. I think we should revisit it. But that is outside the scope of this PR.
   
   To unblock this PR, I am fine with `close(Duration.Zero)`, but I have the feeling we could do better.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525681642



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }
 
-                if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
-                    try {
-                        globalStreamThread.join();
-                    } catch (final InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                    globalStreamThread = null;
+            if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
+                try {
+                    globalStreamThread.join();
+                } catch (final InterruptedException e) {
+                    Thread.currentThread().interrupt();
                 }
+                globalStreamThread = null;
+            }
 
-                adminClient.close();
+            adminClient.close();
 
-                streamsMetrics.removeAllClientLevelMetrics();
-                metrics.close();
+            streamsMetrics.removeAllClientLevelMetrics();
+            metrics.close();
+            if (!error) {
                 setState(State.NOT_RUNNING);
-            }, "kafka-streams-close-thread");
+            }
+        }, "kafka-streams-close-thread");
+    }
+
+    private boolean close(final long timeoutMs) {
+        if (!setState(State.PENDING_SHUTDOWN)) {

Review comment:
       This is currently the plan to remove that transition. It is pretty much the only change we plan to make to the FSM.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523334335



##########
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:
       I don't think it will actually transition to `ERROR` because the handler will call close before the global thread is dead, which will transition to PEDING_SHUTDOWN, there is no transition to ERROR from either PENDING_SHUTDOWN or NOT_RUNNING.
   
   the FSM will be part of the add thread work as it doesn't really make sense to remove the change to error until we can add threads




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524817135



##########
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:
       > Since the stream thread is alive when it calls close() there will not be a deadlock anymore. So, why do we call close() with duration zero
   
   @cadonna can you clarify? I thought we would still be in danger of deadlock if we use the blocking `close()`, since `close()` will not return until every thread has joined but the StreamThread that called `close()` would be stuck in this blocking call and thus never stop/join




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523296833



##########
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:
       Is there an extra `uses` in there or am I not looking at this sentence from the right angle?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518267956



##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);

Review comment:
       Thats a good idea, I didn't see that option




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518326776



##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);

Review comment:
       Actually the latch ensures the rebalance gets processed




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331 add a streams handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r514567028



##########
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:
       Same as the other use in KS




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518271215



##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);
+
+            assertThat(processorValueCollector.size(), equalTo(2));
+            assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.ERROR));
+            assertThat("handler was called", flag.get());

Review comment:
       good idea




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517627843



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java
##########
@@ -19,7 +19,7 @@
 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;

Review comment:
       thanks for the reminder. I think I I under stood the test ad incrementing to the next version, as the version is now 9




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r516994424



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +374,66 @@ 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

Review comment:
       ```suggestion
        * @param streamsUncaughtExceptionHandler the uncaught exception handler of type {@link StreamsUncaughtExceptionHandler} for all internal threads
   ```
   
   In L389, we say that we throw an exception if the handler is null, which sounds like a more reasonable API to me.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +374,66 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh
         }
     }
 
+    /**
+     * Set the handler invoked when a {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread}

Review comment:
       ```suggestion
        * Set the handler invoked when an {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread}
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -997,6 +1064,72 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            // if transition failed, it means it was either in PENDING_SHUTDOWN
+            // or NOT_RUNNING already; just check that all threads have been stopped
+            log.info("Can not close to error from state " + state());

Review comment:
       ```suggestion
               log.info("Can not transition to error from state " + state());
   ```
   
   Didn't follow the prior message. Is this what you meant?

##########
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:
       It's normally kinda weird to merge commented-out code. I'd either delete it or instead have a todo, like `// TODO KAFKA-XXXX: add case REPLACE_STREAM_THREAD once KIP-??? is implemented`, where `KAFKA-XXXX` is a follow-up ticket you create to implement this feature.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +374,66 @@ 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}.
+     * @throws NullPointerException @NotNull if streamsUncaughtExceptionHandler is null.

Review comment:
       What's up with the `@NotNull` on this line? I don't think I've seen that before.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +374,66 @@ 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}.
+     * @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:
+//                log.error("Encountered the following exception during processing " +
+//                        "and the the stream thread will be replaced: ", e);
+//            this.addStreamsThread();
+//                break;
+            case SHUTDOWN_CLIENT:
+                log.error("Encountered the following exception during processing " +
+                        "and the client is going to shut down: ", e);

Review comment:
       ```suggestion
                   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);
   ```
   
   Just a little extra information, so we don't always have to pull up this code block to remember what exact response action this message corresponds to.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +374,66 @@ 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}.
+     * @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:
+//                log.error("Encountered the following exception during processing " +
+//                        "and the the stream thread will be replaced: ", e);
+//            this.addStreamsThread();
+//                break;
+            case SHUTDOWN_CLIENT:
+                log.error("Encountered the following exception during processing " +
+                        "and the client is going to shut down: ", e);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (e instanceof Error) {
+                    log.error("This option requires the thread to stay running to start the shutdown." +
+                            "Therefore it is not suitable for Error types.");

Review comment:
       ```suggestion
                       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);
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -997,6 +1064,72 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            // if transition failed, it means it was either in PENDING_SHUTDOWN
+            // or NOT_RUNNING already; just check that all threads have been stopped
+            log.info("Can not close to error from state " + state());
+        } else {
+            log.info("closing to ERROR");

Review comment:
       ```suggestion
               log.info("Transitioning to ERROR state");
   ```
   
   Similar confusion here...

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -294,7 +304,10 @@ public static StreamThread create(final InternalTopologyBuilder builder,
                                       final long cacheSizeBytes,
                                       final StateDirectory stateDirectory,
                                       final StateRestoreListener userStateRestoreListener,
-                                      final int threadIdx) {
+                                      final int threadIdx,
+                                      final ShutdownErrorHook shutdownErrorHook,
+                                      final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler,
+                                      final AtomicInteger assignmentErrorCode) {

Review comment:
       It doesn't look like this needs to be shared outside of this thread. It seems like it just needs to be shared between the StreamThread and its Consumer?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
##########
@@ -311,6 +317,22 @@ public void run() {
                 "Updating global state failed. You can restart KafkaStreams to recover from this error.",
                 recoverableException
             );
+        } 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.");

Review comment:
       This doesn't look like an "error". At best it's a "warn" log, but only if we think that this combination definitely looks like a misconfiguration. Even then, why wouldn't we check for the misconfiguration in KafkaStreams, since both the new and old handlers would be set over there?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
##########
@@ -311,6 +317,22 @@ public void run() {
                 "Updating global state failed. You can restart KafkaStreams to recover from this error.",
                 recoverableException
             );
+        } 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;
+            } else {
+                log.warn("Exception in global stream thread cause the application to attempt to shutdown." +
+                        " This action will succeed only if there is at least one StreamThread running on ths client");
+            }

Review comment:
       ```suggestion
               if (this.streamsUncaughtExceptionHandler.handle(e) = StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION) {
                   log.warn("Exception in global stream thread cause the application to attempt to shutdown." +
                           " This action will succeed only if there is at least one StreamThread running on ths client");
               }
   ```
   
   This looked a bit off...




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#issuecomment-729396557


   System test run (still running but so far it's all PASS) -- https://jenkins.confluent.io/job/system-test-kafka-branch-builder/4292/


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525663640



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -559,18 +552,51 @@ 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) {
+                final String errorMessage = e.getMessage();
+                if (errorMessage != null &&
+                        errorMessage.startsWith("Broker unexpectedly doesn't support requireStable flag on version ")) {
+
+                    log.error("Shutting down because the Kafka cluster seems to be on a too old version. " +

Review comment:
       We should remember to update the wording here when we add the REPLACE_THREAD functionality




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman merged pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman merged pull request #9487:
URL: https://github.com/apache/kafka/pull/9487


   


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517543638



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java
##########
@@ -19,7 +19,7 @@
 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;

Review comment:
       Can you also leave a comment here reminding us to fix the version probing system test whenever this protocol number is bumped? Since we apparently always forget




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518263937



##########
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:
       It might be but I do not think that it is necessary




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518274359



##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);
+
+            assertThat(processorValueCollector.size(), equalTo(2));

Review comment:
       I use 2 threads there to make sure the old behavior is being followed. Just one thread dies and then the next thread is tries. The second thread makes sure that the new path is not closing the client unintentionally.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523334732



##########
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:
       I agree we shouldn't remove the valid test cases. Maybe the ones that are more complicated I can just set an idempotent old handler and mark as deprecated and we can file tickets to update. Either we work them down or when we go to remove the old handler they will fail and we need to fix them then.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r526211409



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }
 
-                if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
-                    try {
-                        globalStreamThread.join();
-                    } catch (final InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                    globalStreamThread = null;
+            if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
+                try {
+                    globalStreamThread.join();
+                } catch (final InterruptedException e) {
+                    Thread.currentThread().interrupt();
                 }
+                globalStreamThread = null;
+            }
 
-                adminClient.close();
+            adminClient.close();
 
-                streamsMetrics.removeAllClientLevelMetrics();
-                metrics.close();
+            streamsMetrics.removeAllClientLevelMetrics();
+            metrics.close();
+            if (!error) {
                 setState(State.NOT_RUNNING);
-            }, "kafka-streams-close-thread");
+            }
+        }, "kafka-streams-close-thread");
+    }
+
+    private boolean close(final long timeoutMs) {
+        if (!setState(State.PENDING_SHUTDOWN)) {

Review comment:
       I do think that Error should not have direct transition. However I don't like using `PENDING_SHUTDOWN` , mostly because we can already distinguish between the two states and it would be best to inform right away. Also it could be a problem if we went to set Error and some how it went from PENDING_SHUTDOWN to NOT_RUNNING. I am in favor of adding something like `PENDING_ERROR` just to be more precise. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#issuecomment-764961250


   @wcarlson5 Seems we forgot the update the docs for this KIP. Can you do a follow up PR? This follow up PR should also cover the changes of https://github.com/apache/kafka/pull/9487 that is part if the same KIP.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518850419



##########
File path: streams/src/main/resources/common/message/SubscriptionInfoData.json
##########
@@ -57,6 +57,11 @@
       "name": "uniqueField",
       "versions": "8+",
       "type": "int8"
+    },
+    {
+      "name": "shutdownRequested",
+      "versions": "9+",
+      "type": "int8"

Review comment:
       I think I agree on the name, I am not sure about the type. We should be able to fit thousands of different error code into the byte so we should not run out of space. The reason the errorCode. is an integer in the first place is because there is not `AtomicByte` that I know of.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518265803



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

Review comment:
       sure




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523337785



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
##########
@@ -115,6 +119,10 @@ private SubscriptionInfo(final SubscriptionInfoData subscriptionInfoData) {
         this.data = subscriptionInfoData;
     }
 
+    public int errorCode() {
+        return data.errorCode();

Review comment:
       Good idea. It does not seem to do anything. but good to have a test for it




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r526477258



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }
 
-                if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
-                    try {
-                        globalStreamThread.join();
-                    } catch (final InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                    globalStreamThread = null;
+            if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
+                try {
+                    globalStreamThread.join();
+                } catch (final InterruptedException e) {
+                    Thread.currentThread().interrupt();
                 }
+                globalStreamThread = null;
+            }
 
-                adminClient.close();
+            adminClient.close();
 
-                streamsMetrics.removeAllClientLevelMetrics();
-                metrics.close();
+            streamsMetrics.removeAllClientLevelMetrics();
+            metrics.close();
+            if (!error) {
                 setState(State.NOT_RUNNING);
-            }, "kafka-streams-close-thread");
+            }
+        }, "kafka-streams-close-thread");
+    }
+
+    private boolean close(final long timeoutMs) {
+        if (!setState(State.PENDING_SHUTDOWN)) {

Review comment:
       Sounds reasonable




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525701691



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread that threw the exception, 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHandler) {
+            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 throwable,
+                                                final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(throwable);
+        if (oldHandler) {
+            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. ", throwable);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (throwable 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.", throwable);
+                }
+                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);

Review comment:
       That's fair. I guess I was thinking less about the inherent meaning of ERROR vs NOT_RUNNING, and more about not behaving differently in this special case. ie if there _are_ still StreamThreads running when a user selects SHUTDOWN_APPLICATION, then we ultimately transition to ERROR. So it strikes me as a bit odd to transition to NOT_RUNNING just because we didn't happen to have any threads left.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524540416



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -255,8 +255,9 @@ public ByteBuffer subscriptionUserData(final Set<String> topics) {
             taskManager.processId(),
             userEndPoint,
             taskManager.getTaskOffsetSums(),
-            uniqueField)
-                .encode();
+            uniqueField,
+            (byte) assignmentErrorCode.get()

Review comment:
       I guess I must have misunderstood  your earlier comment. I thought you wanted it to stay a byte so that is why I pushed back. But if you have no objections I will just change it 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523319677



##########
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:
       I think any test that's trying to verify some unrelated behavior and just using the "one thread dies at a time" paradigm as a tool to do so should not be deleted. I'm sure in most if not all cases, there's some way to modify the test to verify that specific behavior either using the new handler or multiple apps or rewriting it altogether. 
   
   But, there are a lot of tests that do this and a lot of them are pretty tricky, so I wouldn't want to stall this PR on waiting for all of these tests to be updated/adapted. I think we should file tickets for all of these tests and just try to pick up one or two of them every so often. Maybe that's being overly optimistic about our inclination to pick up small tasks even over a long period, but it's better than losing track of them altogether. WDYT?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523372464



##########
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:
       Ah ok so there's some other IllegalStateException that would get swallowed if we just used `e -> {}` like in the other tests, so we need to explicitly rethrow it? That seems fine, although it makes me think that we should go ahead and use a "real" handler in _all_ of the tests, not just this one. Otherwise there could be some bug which causes an unexpected exception, but the test would just swallow it and silently pass.
   Can we just use the default handler wrapper for all of these tests so they reflect realistic scenarios?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518269493



##########
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:
       Why not? It would be much cleaner. We would close all stuff like admin client and the metrics, remove the client metrics and set the state to NOT_RUNNING which is not necessarily done with timeout zero (probably not because of the death lock). Additionally, we would get an nice info debug saying `Streams client stopped completely` instead of `Streams client cannot stop completely within the timeout`. ;-)




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523327104



##########
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:
       I appreciate the benefit of the doubt :) but you are right there is an extra `uses`




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524435241



##########
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:
       The default is in KafkaStreams, but I see your point. We can make all of them rethrow then we will not have to worry about swallowing




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518842747



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -282,6 +283,17 @@ public boolean isRunning() {
     private final Admin adminClient;
     private final InternalTopologyBuilder builder;
 
+    private Handler streamsUncaughtExceptionHandler;
+    private ShutdownErrorHook shutdownErrorHook;
+    private AtomicInteger assignmentErrorCode;
+    public interface ShutdownErrorHook {
+        void shutdown();
+    }

Review comment:
       Yes we can




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518371722



##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);

Review comment:
       good questions




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518913514



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());

Review comment:
       Gotcha. In that case maybe we shouldn't log anything here at all? Or just reword it to clarify that this is expected (eg `"Skipping shutdown since we are already in ERROR"`) since "Can not transition..." kind of sounds like something went wrong




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518271918



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

Review comment:
       we probably don't need all of them. I will trim them down




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523080590



##########
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:
       The name is a bit confusing. The best I could come up is `handleStreamsUncaughtExceptionByDefault()`, but I am sure there is a better name.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518488577



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -255,8 +255,9 @@ public ByteBuffer subscriptionUserData(final Set<String> topics) {
             taskManager.processId(),
             userEndPoint,
             taskManager.getTaskOffsetSums(),
-            uniqueField)
-                .encode();
+            uniqueField,
+            (byte) assignmentErrorCode.get()

Review comment:
       This cast makes me kind of uncomfortable...either the `assignmentErrorCode` that we have in the AssignmentInfo is conceptually the same as the one we're adding to the SubscriptionInfo (in which case it should be the same type), or it's not the same, in which case we should use a different variable to track it.
   
   Personally I think it's probably simpler to keep them the same, and just add an `int` errorCode field to the Subscription instead of a `byte` shutdownRequested field. But it's your choice




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r523311219



##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
##########
@@ -115,6 +119,10 @@ private SubscriptionInfo(final SubscriptionInfoData subscriptionInfoData) {
         this.data = subscriptionInfoData;
     }
 
+    public int errorCode() {
+        return data.errorCode();

Review comment:
       What happens if we try to read the error code of an earlier subscription version? I genuinely don't know what the generated code does, but we should make sure it doesn't throw an NPE or something. Could you add a unit test for this case?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r522596527



##########
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:
       This wording is a little difficult to parse

##########
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:
       Just curious, what's the motivation for doing it like this vs just immediately throwing the exception?

##########
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:
       Hmm...this one seems like it should be a fatal error, so is it safe to just pass it along to the user and let them potentially just keep replacing the thread? (I know that option doesn't exist yet, but it will). There are some instances where we interpret errors as permanently fatal and choose to shut down the entire application, eg some errors during assignment. Should we do the same here? cc @abbccdda or @mjsax for more context on this error

##########
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:
       Does the comment relate to the `@deprecation` suppression? Either way this probably makes more sense as a comment on the PR than in the code.  Given how bad we are about updating comments, I'd try to avoid anything that describes a change and reserve code comments for describing what's currently going on (or better yet, "why")

##########
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:
       Why set the exception handler in this test and no others?

##########
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:
       Is everything after this line the same as the code in the regular `close()`? Might be a good idea to move it to a separate method so we don't accidentally forget to update one of them if we ever need to make changes to how we 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);
+            }
+        } else {
+            handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT);
+        }
+    }
+
+    private void handleStreamsUncaughtException(final Throwable e,
+                                                   final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {

Review comment:
       nit: parameters unaligned

##########
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:
       It seems like we shouldn't both handle the exception in the catch block AND shut down the client in the finally block. If the new handler is used, then we've already shut down the client or possibly started to shut down the whole application. It's tricky, though, because if the old handler was used then we _do_ want to make sure that the global thread is all cleaned up before rethrowing the exception. 
   Seems like we need some way to detect whether we're using the old or the new handler after all. But I think you can do it without too many changes, since basically the rule is "if they set a new handler at all OR didn't set either handler, then use the new one". So maybe you can just make the `StreamsUncaughtExceptionHandler` a local field instead of the `Consumer<>`, and leave it as `null` to indicate that the old handler should be used and therefore this shutdown logic should be invoked. Otherwise just call the new handler directly. Or something like that...you'd know this code better than me, WDYT?

##########
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's a lot of line breaks 🙃 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
##########
@@ -77,6 +77,20 @@ private static void validateVersions(final int version, final int latestSupporte
         }
     }
 
+    public SubscriptionInfo(final int version,
+                            final int latestSupportedVersion,
+                            final UUID processId,
+                            final String userEndPoint,
+                            final Map<TaskId, Long> taskOffsetSums,
+                            final byte uniqueField,
+                            final byte errorCode) {

Review comment:
       I think we should add the `errorCode` parameter to the existing constructor rather than add a new one. It shouldn't be possible to construct a version 9 subscription that doesn't have an `errorCode`

##########
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:
       same here, what is the comment referring to? Also what does it mean for a test to be deprecated 🤔 

##########
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:
       Nice, thanks for the comment. Btw anytime we bump this protocol version we should add the corresponding unit tests, eg `SubscriptionInfoTest#shouldEncodeAndDecodeVersion8()`

##########
File path: streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStreamThreadTest.java
##########
@@ -113,7 +113,8 @@ public String newStoreName(final String prefix) {
             new StreamsMetricsImpl(new Metrics(), "test-client", StreamsConfig.METRICS_LATEST, time),
             time,
             "clientId",
-            stateRestoreListener
+            stateRestoreListener,
+            e -> { }

Review comment:
       We should probably use an actual handler here to make sure it works with the GlobalThread. Actually maybe we should add a few unit tests here to make sure that it closes down and rethrows when the old handler is used, but handles the exception internally when the new handler is used, etc

##########
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:
       ditto here

##########
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:
       Is this the only property that changed? Might be clearer if you just override what you need to here

##########
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:
       Is the latch ever being counted down anywhere? You might want to take a look at some of the test utils, there's a lot of useful stuff so you don't have to implement everything from scratch. If you just want to make sure that the client gets to `CLOSED` within 15s then I'd recommend `TestUtils#waitForCondition` 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r525650632



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) {
         return close(timeoutMs);
     }
 
-    private boolean close(final long timeoutMs) {
-        if (!setState(State.PENDING_SHUTDOWN)) {
-            // if transition failed, it means it was either in PENDING_SHUTDOWN
-            // or NOT_RUNNING already; just check that all threads have been stopped
-            log.info("Already in the pending shutdown state, wait to complete shutdown");
-        } else {
-            stateDirCleaner.shutdownNow();
-            if (rocksDBMetricsRecordingService != null) {
-                rocksDBMetricsRecordingService.shutdownNow();
-            }
+    private Thread shutdownHelper(final boolean error) {
+        stateDirCleaner.shutdownNow();
+        if (rocksDBMetricsRecordingService != null) {
+            rocksDBMetricsRecordingService.shutdownNow();
+        }
 
-            // wait for all threads to join in a separate thread;
-            // save the current thread so that if it is a stream thread
-            // we don't attempt to join it and cause a deadlock
-            final Thread shutdownThread = new Thread(() -> {
-                // notify all the threads to stop; avoid deadlocks by stopping any
-                // further state reports from the thread since we're shutting down
-                for (final StreamThread thread : threads) {
-                    thread.shutdown();
-                }
+        // wait for all threads to join in a separate thread;
+        // save the current thread so that if it is a stream thread
+        // we don't attempt to join it and cause a deadlock
+        return new Thread(() -> {
+            // notify all the threads to stop; avoid deadlocks by stopping any
+            // further state reports from the thread since we're shutting down
+            for (final StreamThread thread : threads) {
+                thread.shutdown();
+            }
 
-                for (final StreamThread thread : threads) {
-                    try {
-                        if (!thread.isRunning()) {
-                            thread.join();
-                        }
-                    } catch (final InterruptedException ex) {
-                        Thread.currentThread().interrupt();
+            for (final StreamThread thread : threads) {
+                try {
+                    if (!thread.isRunning()) {
+                        thread.join();
                     }
+                } catch (final InterruptedException ex) {
+                    Thread.currentThread().interrupt();
                 }
+            }
 
-                if (globalStreamThread != null) {
-                    globalStreamThread.shutdown();
-                }
+            if (globalStreamThread != null) {
+                globalStreamThread.shutdown();
+            }
 
-                if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
-                    try {
-                        globalStreamThread.join();
-                    } catch (final InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                    }
-                    globalStreamThread = null;
+            if (globalStreamThread != null && !globalStreamThread.stillRunning()) {
+                try {
+                    globalStreamThread.join();
+                } catch (final InterruptedException e) {
+                    Thread.currentThread().interrupt();
                 }
+                globalStreamThread = null;
+            }
 
-                adminClient.close();
+            adminClient.close();
 
-                streamsMetrics.removeAllClientLevelMetrics();
-                metrics.close();
+            streamsMetrics.removeAllClientLevelMetrics();
+            metrics.close();
+            if (!error) {
                 setState(State.NOT_RUNNING);
-            }, "kafka-streams-close-thread");
+            }
+        }, "kafka-streams-close-thread");
+    }
+
+    private boolean close(final long timeoutMs) {
+        if (!setState(State.PENDING_SHUTDOWN)) {

Review comment:
       I just realized that this is going to be a problem with the way the ERROR state is being used. IF we `closeToError` then we transition to ERROR and shut down, however `ERROR -> PENDING_SHUTDOWN` is still an allowed transition so there's nothing to prevent the shutdown from being triggered again when a user calls `close()`. And note that a lot of users most likely have a state listener at the moment which does exactly that, ie when it sees a transition to ERROR it immediately invokes close (because that's what you should do with the current semantics)
   Just another thing that I think we can fix with some minor rewiring of the FSM. 




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r517474129



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -346,18 +351,21 @@ 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.

Review comment:
       I guess this should be 2.8.0, shouldn't it?

##########
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:
       Wouldn't it also be possible to start a shutdown thread here which closes the client without timeout? I think the other shutdown thread in close is rather useless (or I do simply not get its value).

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);

Review comment:
       Why do clean the state twice?

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

Review comment:
       Why do you need to set all these properties?

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
##########
@@ -60,6 +60,9 @@ 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()) {

Review comment:
       Unit tests for this case are missing.

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

Review comment:
       ```suggestion
           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)
               )
           );
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -997,6 +1061,72 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            // if transition failed, it means it was either in PENDING_SHUTDOWN
+            // or NOT_RUNNING already; just check that all threads have been stopped

Review comment:
       Is this comment correct? In this code path we do NOT check that all threads have been stopped. 

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
##########
@@ -60,6 +60,9 @@ 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("An application is requesting Shutdown");

Review comment:
       ```suggestion
               log.error("A Kafka Streams client in this Kafka Streams application is requesting to shutdown");
   ```
   An application is actually a group of Kafka Streams clients (or instances).

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

Review comment:
       I would remove these comments. 

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);
+
+            assertThat(processorValueCollector.size(), equalTo(2));

Review comment:
       I had a hard time to understand this. We write just one record to the topic, but we end up processing two records. This is true, because we use two stream threads and there is no commit between the processing of the record of the first stream thread and the processing of the second stream thread. Why do you use two stream threads here?

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

Review comment:
       The name is a bit ambiguous. I would go for `StreamsUncaughtExceptionHandlerIntegrationTest`

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);

Review comment:
       What is the benefit of using a latch versus simply sleeping here? 
   Actually, you should use `StreamsTestUtils.startKafkaStreamsAndWaitForRunningState()` to avoid flakiness coming from the Kafka Streams client not being in state RUNNING before the verifications.

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);
+
+            assertThat(processorValueCollector.size(), equalTo(2));
+            assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.ERROR));
+            assertThat("handler was called", flag.get());

Review comment:
       You could wait for this flag to become true with `TestUtils.waitForCondition()` before you verify the other criteria.

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##########
@@ -0,0 +1,274 @@
+/*
+ * 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);
+
+        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 shouldShutdownThreadUsingOldHandler() 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);
+            final AtomicBoolean flag = new AtomicBoolean(false);
+            kafkaStreams.setUncaughtExceptionHandler((t, e) -> flag.set(true));
+
+            kafkaStreams.start();
+
+            produceMessages(0L, inputTopic, "A");
+            latch.await(15, TimeUnit.SECONDS);
+
+            assertThat(processorValueCollector.size(), equalTo(2));
+            assertThat(kafkaStreams.state(), equalTo(KafkaStreams.State.ERROR));
+            assertThat("handler was called", flag.get());
+        }
+    }
+
+    @Test
+    public void shouldShutdownClient() throws Exception {

Review comment:
       Most of the above comments also apply to the other tests.

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

Review comment:
       Why are those fields all package-private instead of private? 
   We usually define string constants as `private static final String IDEMPOTENT_TOPIC = "idempotentTopic"`.

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

Review comment:
       I do not understand the motivation behind this topic. Could you clarify? 

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -659,7 +727,6 @@ private KafkaStreams(final InternalTopologyBuilder internalTopologyBuilder,
                          final Time time) throws StreamsException {
         this.config = config;
         this.time = time;
-

Review comment:
       IMO, it would be better to extract code to methods instead of removing some lines. 

##########
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:
       I would also remove the commented-out code.

##########
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:
       Could you also remove the commented-out code here.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524448160



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHanlder) {
+            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 throwable,
+                                                final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(throwable);
+        if (oldHanlder) {
+            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. ", throwable);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (throwable 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.", throwable);
+                }
+                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();

Review comment:
       Likewise, here, it seems better to do a non-blocking close.

##########
File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -255,8 +255,9 @@ public ByteBuffer subscriptionUserData(final Set<String> topics) {
             taskManager.processId(),
             userEndPoint,
             taskManager.getTaskOffsetSums(),
-            uniqueField)
-                .encode();
+            uniqueField,
+            (byte) assignmentErrorCode.get()

Review comment:
       I think I'd like to re-raise Sophie's concern here. It doesn't compute for me why we are casting an int to a byte here..

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread use Thread.currentThread()

Review comment:
       ```suggestion
        * In order to get the thread that threw the exception, use Thread.currentThread().
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +377,90 @@ 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.
+     * In order to get the thread 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 defaultStreamsUncaughtExceptionHandler(final Throwable throwable) {
+        if (oldHanlder) {
+            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 throwable,
+                                                final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action = streamsUncaughtExceptionHandler.handle(throwable);
+        if (oldHanlder) {
+            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. ", throwable);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (throwable 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.", throwable);
+                }
+                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();
+                } 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: ", throwable);

Review comment:
       ```suggestion
                               "and sent shutdown request for the entire application.", throwable);
   ```

##########
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:
       I think I'd personally still prefer the non-blocking version. It seems better to avoid blocking indefinitely when a thread is trying to shut itself down due to some unknown exception (or error).

##########
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:
       Personally, as long as users have the information available to understand the nature of the error, it's fine to let them make their own decision about how to handle it. Maybe another team is in the middle of a broker upgrade, for example, and the owner of this app would like to just keep trying until the broker team gets it together.




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524433881



##########
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:
       It's actually not always used. It is only used until a new handler is set in which it is over written. Once that happens we don't want the old handler to be set so we do not wrap a user provided handler with this method




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518283539



##########
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:
       My last comment is not true! Sorry! Everything alright!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] lct45 commented on a change in pull request #9487: KAFKA-9331 add a streams handler

Posted by GitBox <gi...@apache.org>.
lct45 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r514527605



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

Review comment:
       Is this spacing on purpose?




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
cadonna commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518283539



##########
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:
       My last comment is not true! Sorry!




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] mjsax edited a comment on pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
mjsax edited a comment on pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#issuecomment-764961250


   @wcarlson5 Seems we forgot the update the docs for this KIP. Can you do a follow up PR? This follow up PR should also cover the changes of https://issues.apache.org/jira/browse/KAFKA-10810 that is part if the same KIP.


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
wcarlson5 commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518263368



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -346,18 +351,21 @@ 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.

Review comment:
       yes




----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#issuecomment-729474257


   System tests passed, all three Java builds passed. Merging now


----------------------------------------------------------------
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:
users@infra.apache.org



[GitHub] [kafka] ableegoldman commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

Posted by GitBox <gi...@apache.org>.
ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r524814932



##########
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:
       That's a fair point about broker upgrades, but don't we require the brokers to be upgraded to a version that supports EOS _before_ turning on eos-beta?
   Anyways I was wondering if there was something special about this exception such that ignoring it could violate eos or corrupt the state of the program. I'll ping the eos experts to assuage my concerns




----------------------------------------------------------------
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:
users@infra.apache.org