Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id D5178200D6F for ; Mon, 18 Dec 2017 03:44:06 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id D3A81160C28; Mon, 18 Dec 2017 02:44:06 +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 25C77160C17 for ; Mon, 18 Dec 2017 03:44:05 +0100 (CET) Received: (qmail 86005 invoked by uid 500); 18 Dec 2017 02:44:04 -0000 Mailing-List: contact dev-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@kafka.apache.org Delivered-To: mailing list dev@kafka.apache.org Received: (qmail 85994 invoked by uid 99); 18 Dec 2017 02:44:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 18 Dec 2017 02:44:04 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 0E3351A09EE for ; Mon, 18 Dec 2017 02:44:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RCVD_IN_DNSWL_NONE=-0.0001, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id ry_6wfrBgxOF for ; Mon, 18 Dec 2017 02:44:02 +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 DDF635F297 for ; Mon, 18 Dec 2017 02:44:01 +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 32578E0AA3 for ; Mon, 18 Dec 2017 02:44:01 +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 16915240DE for ; Mon, 18 Dec 2017 02:44:00 +0000 (UTC) Date: Mon, 18 Dec 2017 02:44:00 +0000 (UTC) From: "Andy Bryant (JIRA)" To: dev@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (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 archived-at: Mon, 18 Dec 2017 02:44:07 -0000 Andy Bryant created KAFKA-6378: ---------------------------------- Summary: 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)