From jira-return-8706-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Thu Jan 11 00:54:11 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id 044F618076D for ; Thu, 11 Jan 2018 00:54:11 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id E89D4160C2E; Wed, 10 Jan 2018 23:54:10 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 43337160C40 for ; Thu, 11 Jan 2018 00:54:10 +0100 (CET) Received: (qmail 93447 invoked by uid 500); 10 Jan 2018 23:54:09 -0000 Mailing-List: contact jira-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@kafka.apache.org Delivered-To: mailing list jira@kafka.apache.org Received: (qmail 93436 invoked by uid 99); 10 Jan 2018 23:54:09 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 10 Jan 2018 23:54:09 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 01B98C162A for ; Wed, 10 Jan 2018 23:54:09 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.911 X-Spam-Level: X-Spam-Status: No, score=-99.911 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_LOW=-0.7, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id NPtGybb6dnx4 for ; Wed, 10 Jan 2018 23:54:08 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 518615F36B for ; Wed, 10 Jan 2018 23:54:07 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id DE222E0EFA for ; Wed, 10 Jan 2018 23:54:05 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 23323274D5 for ; Wed, 10 Jan 2018 23:54:04 +0000 (UTC) Date: Wed, 10 Jan 2018 23:54:04 +0000 (UTC) From: "Andy Bryant (JIRA)" To: jira@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (KAFKA-6378) NullPointerException on KStream-GlobalKTable leftJoin when KeyValueMapper returns null MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/KAFKA-6378?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16321374#comment-16321374 ] Andy Bryant commented on KAFKA-6378: ------------------------------------ A sentinel value is ok where only a subset of the available values for a type are valid, but it does seem messy to have to convert {{null}} values to the sentinel before the join then back to {{null}} again in the merge function afterwards. Also it doesn't cater for the case where you can't pick a sentinel because all values of a type are valid. Since as Matthias pointed out {{null}} can never be a valid key explicitly calling it out as indicating no match in the docs and updated the code so it doesn't crash (a two line change by the looks) seems like a nice clean approach to me. > 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)