[ 
https://issues.apache.org/jira/browse/KAFKA-9517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17031953#comment-17031953
 ] 

John Roesler commented on KAFKA-9517:
-------------------------------------

Thanks for the update, [~psnively].

I've been trying to reproduce the issue you've reported (building on the fix 
already for KAFKA-9500), and I haven't been able to produce an NPE. Do you 
happen to have the stacktrace handy?

To address the concern you voiced about the comment in the code, the valueSerde 
is set to `null` to indicate that we do not know the serde a priori. This sets 
us up to use the following precedence rules at run time.

If an operator needs to serialize some data, it will use:
#1 The serde explicily passed to it, via Materialized, Produced, Grouped, etc.
#2 If no explicit serde is passed, then use the serde passed from the upstream 
operator, if applicable*
#3 If no explicit or upstream serde is available, use the "default" serde 
provided in config

* this is where passing a null downstream indicates that no applicable serde is 
available.

So, what I would expect to see is a ClassCastException if the "default" serde 
isn't for the same type as the foreign key join result. But I think maybe I 
misunderstood the scenario you provided.

Thanks,
-John

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> ----------------------------------------------------------------------------------
>
>                 Key: KAFKA-9517
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9517
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 2.4.0
>            Reporter: Paul Snively
>            Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to