You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Matthias J. Sax (JIRA)" <ji...@apache.org> on 2017/12/22 22:02:00 UTC

[jira] [Commented] (KAFKA-6378) NullPointerException on KStream-GlobalKTable leftJoin when KeyValueMapper returns null

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

Matthias J. Sax commented on KAFKA-6378:
----------------------------------------

In {{KafkaStreams}} a {{KTable}} or {{GlobalKTable}} cannot contain data with {{key == null}} (same for {{value == null}} btw). All those records are dropped and never inserted into the table in the first place.

Thus, the join on {{null}} can never succeed (this hold for Kafka {{0.11}}, too). Maybe because you do a {{leftJoin()}} you never noticed -- but I am confident that the lookup with {{key==null}} never returned any value from the {{GlobalKTable}} in {{0.11}}.

In {{1.0}} we put more checks into the code to detect buggy user code, and returning {{null}} from {{KeyValueMapper}} is not valid for this case.

A workaround would be, to use sentinel key or values for {{null}} -- for a {{GlobalKTable}} you would need to read the original topic and to a {{map()}} that replaces {{null}} with this sentinels, and write the result back into a new topic. This new topic can than be used a {{GlobalKTable}} input topic.

Does this make sense? I think, we can close this ticket as "not a problem".

> NullPointerException on KStream-GlobalKTable leftJoin when KeyValueMapper returns null
> --------------------------------------------------------------------------------------
>
>                 Key: KAFKA-6378
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6378
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 1.0.0
>            Reporter: Andy Bryant
>
> On a Stream->GlobalKTable leftJoin if the KeyValueMapper returns null, the stream fails with a NullPointerException (see stacktrace below). On Kafka 0.11.0.0 the stream processes this successfully, calling the ValueJoiner with the table value set to null.
> The use-case for this is joining a stream to a table containing reference data where the stream foreign key may be null. There is no straight-forward workaround in this case with Kafka 1.0.0 without having to resort to either generating a key that will never match or branching the stream for records that don't have the foreign key.
> Exception in thread "workshop-simple-example-client-StreamThread-1" java.lang.NullPointerException
> 	at java.base/java.util.Objects.requireNonNull(Objects.java:221)
> 	at org.apache.kafka.streams.state.internals.CachingKeyValueStore.get(CachingKeyValueStore.java:136)
> 	at org.apache.kafka.streams.state.internals.CachingKeyValueStore.get(CachingKeyValueStore.java:35)
> 	at org.apache.kafka.streams.state.internals.InnerMeteredKeyValueStore.get(InnerMeteredKeyValueStore.java:184)
> 	at org.apache.kafka.streams.state.internals.MeteredKeyValueBytesStore.get(MeteredKeyValueBytesStore.java:116)
> 	at org.apache.kafka.streams.kstream.internals.KTableSourceValueGetterSupplier$KTableSourceValueGetter.get(KTableSourceValueGetterSupplier.java:49)
> 	at org.apache.kafka.streams.kstream.internals.KStreamKTableJoinProcessor.process(KStreamKTableJoinProcessor.java:56)
> 	at org.apache.kafka.streams.processor.internals.ProcessorNode$1.run(ProcessorNode.java:46)
> 	at org.apache.kafka.streams.processor.internals.StreamsMetricsImpl.measureLatencyNs(StreamsMetricsImpl.java:208)
> 	at org.apache.kafka.streams.processor.internals.ProcessorNode.process(ProcessorNode.java:124)
> 	at org.apache.kafka.streams.processor.internals.ProcessorContextImpl.forward(ProcessorContextImpl.java:85)
> 	at org.apache.kafka.streams.processor.internals.SourceNode.process(SourceNode.java:80)
> 	at org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:216)
> 	at org.apache.kafka.streams.processor.internals.AssignedTasks.process(AssignedTasks.java:403)
> 	at org.apache.kafka.streams.processor.internals.TaskManager.process(TaskManager.java:317)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.processAndMaybeCommit(StreamThread.java:942)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:822)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:774)
> 	at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:744)



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)