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