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 2021/08/18 10:59:14 UTC

[GitHub] [kafka] jlprat opened a new pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

jlprat opened a new pull request #11228:
URL: https://github.com/apache/kafka/pull/11228


   Instead of letting all `RuntimeException`s go through and be processed by
   the uncaught exception handler, `IllegalStateException` and
   `IllegalArgumentException` are not passed through and fail fast.
   
   Added test checking this new case.
   For the test I use an existing test as a baseline that was checking the uncaught exception handler was called and checked that it wasn't called for this new particular case.
   
   Possible extensions would be to add more types of `RuntimeException`s on the new `catch` clause.
   
   ### 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   cc. @guozhangwang Feel free to review this PR and let me know if this is what you had in mind with https://issues.apache.org/jira/browse/KAFKA-12887


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat closed pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

Posted by GitBox <gi...@apache.org>.
jlprat closed pull request #11228:
URL: https://github.com/apache/kafka/pull/11228


   


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on a change in pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -495,9 +498,25 @@ private void replaceStreamThread(final Throwable throwable) {
         }
     }
 
+    private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable,
+                                                                                        final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action;
+        // Exception might we wrapped within a StreamsException one

Review comment:
       https://github.com/jlprat/kafka/blob/KAFKA-12887-no-trigger-exceptionhandler-for-rte/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java#L756
   This is where I can see RTE are always wrapped in a `StreamsException`. I'm not sure if this is always happening or there are cases in which the RTE is not wrapped.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on a change in pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -509,7 +526,7 @@ private void handleStreamsUncaughtException(final Throwable throwable,
                 break;
             case SHUTDOWN_CLIENT:
                 log.error("Encountered the following exception during processing " +
-                        "and the registered exception handler opted to " + action + "." +
+                        "and Kafka Streams opted to " + action + "." +

Review comment:
       Changed the error log line as mentioned.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -495,9 +498,25 @@ private void replaceStreamThread(final Throwable throwable) {
         }
     }
 
+    private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable,
+                                                                                        final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action;
+        // Exception might we wrapped within a StreamsException one

Review comment:
       Updated now, the `if` statement now only checks the wrapped exception if it exists, and decides upon that. See next 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   @guozhangwang What do you mean exactly by "this would require us to add another private function like handleStreamsUncaughtException(StreamThreadExceptionResponse)"? 
   Wouldn't it suffice to check the `throwable` variable in `KafkaStreams#handleStreamsUncaughtException()` before line 500 where the user-specified exception handler is executed?
   BTW, I agree with you that we should shutdown the Streams client instead of just killing a stream thread.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang merged pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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






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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Test failure is related to https://issues.apache.org/jira/browse/KAFKA-8138 but not the same, as in this particular instance it seems more like a failure in the test itself (topic already exists)


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Ping @cadonna and @guozhangwang Is there anything else you'd like changed in the 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Ping @cadonna and @guozhangwang Is there anything else you'd like changed in the 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Reopening to re-trigger CI


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat closed pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

Posted by GitBox <gi...@apache.org>.
jlprat closed pull request #11228:
URL: https://github.com/apache/kafka/pull/11228


   


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Tests failures seem related, I will investigate them


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   > Wouldn't it suffice to check the throwable variable in KafkaStreams#handleStreamsUncaughtException() before line 500 where the user-specified exception handler is executed?
   
   Ah yes, that's even 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Thanks for your feedback @guozhangwang 
   I thought "simply" killing the thread was the original intention.
   I'll look into killing the whole instance once @cadona provides his feedback on this (just in case there isn't something easy to overlook)


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   > But it doesn't seem related to the changes in this PR. I would say it seems a newly found regression (there isn't a ticket yet).
   Any opinions @cadonna @guozhangwang ?
   
   I think I agree, let's wait and see if this is transient.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Test failure was https://issues.apache.org/jira/browse/KAFKA-8529 which was apparently resolved. I reopened the issue.
   I rebased this PR to the trunk of a couple of hours ago.
   


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] fabian-wiesman commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

Posted by GitBox <gi...@apache.org>.
fabian-wiesman commented on pull request #11228:
URL: https://github.com/apache/kafka/pull/11228#issuecomment-910378434


   Aiven Inc - https://aiven.io/ is misusing of Apache Kafka trademark as 'Aiven Kafka' [1] and violates
   Apache trademarks. See - [1] https://help.aiven.io/en/?q=%22aiven+kafka%22 This simply means Aiven Inc / contributors have no idea of Apache projects / communities or how open source works in general. CC @ivanyu , @hnousiainen, Kafka Community / Project Management Committee.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Current failure is https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-11228/5/testReport/org.apache.kafka.streams.integration/NamedTopologyIntegrationTest/Build___JDK_8_and_Scala_2_12___shouldAllowPatternSubscriptionWithMultipleNamedTopologies/
   
   But it doesn't seem related to the changes in this PR. I would say it seems a newly found regression (there isn't a ticket yet).
   Any opinions @cadonna @guozhangwang ?


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Ping @cadonna and @guozhangwang
   Let me know what you think about this version. Thanks!


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Test failure is related to https://issues.apache.org/jira/browse/KAFKA-8138 but not the same, as in this particular instance it seems more like a failure in the test itself (topic already exists)


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   @cadonna Applied your feedback as well, thanks for the comments. I also realized the `AtomicBoolean` variables in the test could be final and added that as well.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Thanks for the reviews @guozhangwang and @cadonna !


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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


   Merged to trunk, thanks @jlprat !


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] cadonna commented on a change in pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -495,9 +498,23 @@ private void replaceStreamThread(final Throwable throwable) {
         }
     }
 
+    private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable,
+                                                                                        final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {

Review comment:
       ```suggestion
       private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable,
                                                                                                   final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -495,9 +498,23 @@ private void replaceStreamThread(final Throwable throwable) {
         }
     }
 
+    private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable,
+                                                                                        final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action;
+        // Exception might we wrapped within a StreamsException one
+        if (throwable.getCause() != null && EXCEPTIONS_NOT_TO_BE_HANDLED_BY_USERS.contains(throwable.getCause().getClass())) {

Review comment:
       Could you please put this check in a method with a meaningful name like `wrappedExceptionIsIn()`? Then the check would be 
   ```
   if (wrappedExceptionIsIn(EXCEPTIONS_NOT_TO_BE_HANDLED_BY_USERS))
   ``` 
   and you can remove all other inline comments because the code per se contains all needed info.

##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java
##########
@@ -163,6 +164,27 @@ public void shouldShutdownClient() throws InterruptedException {
         }
     }
 
+
+    @Test
+    public void shouldShutdownClientWhenIllegalStateException() throws InterruptedException {

Review comment:
       Could you please also add a test for the `IllegalArgumentException`? 




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on a change in pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -495,9 +498,25 @@ private void replaceStreamThread(final Throwable throwable) {
         }
     }
 
+    private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable,
+                                                                                        final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action;
+        // Exception might we wrapped within a StreamsException one

Review comment:
       Hmm I think it should be "always" wrapped, if not then that would be a bug.. before line 500 we can check if `cause()` exists and if that's illegal-state etc.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] jlprat commented on a change in pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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



##########
File path: streams/src/test/java/org/apache/kafka/streams/integration/EmitOnChangeIntegrationTest.java
##########
@@ -104,7 +104,7 @@ public void shouldEmitSameRecordAfterFailover() throws Exception {
             .toStream()
             .map((key, value) -> {
                 if (shouldThrow.compareAndSet(true, false)) {
-                    throw new IllegalStateException("Kaboom");
+                    throw new RuntimeException("Kaboom");

Review comment:
       Throwing an `IllegalStateException` would after this change, cause a client shut 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.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on a change in pull request #11228: KAFKA-12887 Skip some RuntimeExceptions from exception handler

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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -495,9 +498,25 @@ private void replaceStreamThread(final Throwable throwable) {
         }
     }
 
+    private StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse getActionForThrowable(final Throwable throwable,
+                                                                                        final StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse action;
+        // Exception might we wrapped within a StreamsException one
+        if (EXCEPTIONS_NOT_TO_BE_HANDLED_BY_USERS.contains(throwable.getClass())
+            || (throwable.getCause() != null && EXCEPTIONS_NOT_TO_BE_HANDLED_BY_USERS.contains(throwable.getCause().getClass()))) {
+            // The exception should not be passed over to the user defined uncaught exception handler.
+            // Something unexpected happened, we should shut down the client
+            log.warn("Exception bypassed the user defined uncaught exception handler."); // No need to print the exception, it will be printed in the next log line.

Review comment:
       nit: instead of adding a `warn` before the `error` below, I think it's sufficient to just change the error log (a.k.a.line 530) and remove `the registered exception opted to` instead just say `and Kafka Streams opted to`. Currently it would reads like, first "exception bypassed the handler" and then "handler opted to shutdown" which is self-conflicting.




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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org