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