loicgreffier commented on code in PR #16675:
URL: https://github.com/apache/kafka/pull/16675#discussion_r1697564800


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java:
##########
@@ -214,9 +214,13 @@ public void process(final Record<KIn, VIn> record) {
                 internalProcessorContext.currentNode().name(),
                 internalProcessorContext.taskId());
 
-            final ProcessingExceptionHandler.ProcessingHandlerResponse 
response = processingExceptionHandler
-                .handle(errorHandlerContext, record, e);
+            final ProcessingExceptionHandler.ProcessingHandlerResponse 
response;
 
+            try {
+                response = 
processingExceptionHandler.handle(errorHandlerContext, record, e);
+            } catch (final Exception fatalUserException) {
+                throw new StreamsException("Fatal user code error in 
processing error callback", fatalUserException);

Review Comment:
   > One more thing: if we throw `StreamsException` here, we might catch it in 
an upstream `process()`, right? So should we rather wrap this 
`StreamsException` with a `FailedProcessingException`?
   
   @mjsax Correct. `FailedProcessingException` is thrown as is:
   
   
https://github.com/apache/kafka/blob/683b8a2beeb5a75ef0a2ac878cf5a963f9b9d915/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java#L205
   
   > Do we actually have a test for "passing through" of 
`FailedProcessingException`
   
   We do:
   
   
https://github.com/apache/kafka/blob/683b8a2beeb5a75ef0a2ac878cf5a963f9b9d915/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java#L130
   
   It asserts `FailedProcessingException` is not handled and thrown as is. 
Another test has been added to ensure that the cause of the caught exception is 
from the processing 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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to