mjsax commented on code in PR #18292:
URL: https://github.com/apache/kafka/pull/18292#discussion_r1927611085


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java:
##########
@@ -209,13 +209,14 @@ public void process(final Record<KIn, VIn> record) {
             // (instead of `RuntimeException`) to work well with those 
languages
             final ErrorHandlerContext errorHandlerContext = new 
DefaultErrorHandlerContext(
                 null, // only required to pass for 
DeserializationExceptionHandler
-                internalProcessorContext.topic(),
-                internalProcessorContext.partition(),
-                internalProcessorContext.offset(),
-                internalProcessorContext.headers(),
+                internalProcessorContext.recordContext().topic(),

Review Comment:
   Fair question. Looking into the code, I believe it should never be `null`.
   
   While `AbstractProcessorContext` has a `null`-check, eg
   ```
       public String topic() {
           if (recordContext == null) {
               // This is only exposed via the deprecated ProcessorContext,
               // in which case, we're preserving the pre-existing behavior
               // of returning dummy values when the record context is 
undefined.
               // For topic, the dummy value is `null`.
               return null;
           } else {
               return recordContext.topic();
           }
       }
   ```
   
   the comment seems to indicate that it could only be `null` for the old 
Processor API which we did remove. I am also not 100% sure, but my read it, it 
should never be `null`. -- We did not see any failing tests either, what I 
would hope is verification enough?
   
   Looking into the code a little bit more, I did not find any call to 
`setRecordContext` which would pass in `null` (to unset it). So `recordContext` 
would be `null` only initially when the `ProcessorContext` is created, but it 
seems it's always set properly before we process records or call punctuation. 
-- So the only issue with being `null` could be if we would not set it after 
`ProcessorContext` creation and before processing data or calling punctuations, 
but it seems we are doing the right thing (after it was set once, we could 
maybe have a _wrong_ object if we forget to call `setRecordContext` correctly, 
but it could not be `null` any longer).
   
   Does this help?



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to