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 F0178200C39 for ; Thu, 2 Mar 2017 03:06:56 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id EEBEE160B78; Thu, 2 Mar 2017 02:06:56 +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 42219160B70 for ; Thu, 2 Mar 2017 03:06:56 +0100 (CET) Received: (qmail 16953 invoked by uid 500); 2 Mar 2017 02:06:50 -0000 Mailing-List: contact dev-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list dev@flink.apache.org Received: (qmail 16942 invoked by uid 99); 2 Mar 2017 02:06:50 -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; Thu, 02 Mar 2017 02:06:50 +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 D0F261A7B0C for ; Thu, 2 Mar 2017 02:06:49 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -2.347 X-Spam-Level: X-Spam-Status: No, score=-2.347 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-2.999, SPF_NEUTRAL=0.652] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id IJ60kVKtyaZT for ; Thu, 2 Mar 2017 02:06:49 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id C8A895FB61 for ; Thu, 2 Mar 2017 02:06:48 +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 8A819E04E7 for ; Thu, 2 Mar 2017 02:06:45 +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 48CB12415A for ; Thu, 2 Mar 2017 02:06:45 +0000 (UTC) Date: Thu, 2 Mar 2017 02:06:45 +0000 (UTC) From: "Xiaojun Jin (JIRA)" To: dev@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (FLINK-5947) NullPointerException in ContinuousProcessingTimeTrigger.clear() MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 02 Mar 2017 02:06:57 -0000 Xiaojun Jin created FLINK-5947: ---------------------------------- Summary: NullPointerException in ContinuousProcessingTimeTrigger.clear() Key: FLINK-5947 URL: https://issues.apache.org/jira/browse/FLINK-5947 Project: Flink Issue Type: Bug Components: DataStream API Affects Versions: 1.2.0 Reporter: Xiaojun Jin Priority: Critical The fireTimestamp may be null when deleting processing timer in the ContinuousProcessingTimerTrigger. Exception stack is as follows: {quote} Caused by: java.lang.NullPointerException at org.apache.flink.streaming.api.windowing.triggers.ContinuousProcessingTimeTrigger.clear(ContinuousProcessingTimeTrigger.java:89) at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$Context.clear(WindowOperator.java:761) at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$2.merge(WindowOperator.java:348) at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$2.merge(WindowOperator.java:336) at org.apache.flink.streaming.runtime.operators.windowing.MergingWindowSet.addWindow(MergingWindowSet.java:210) at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.processElement(WindowOperator.java:336) at org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:208) at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:70) at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:265) at org.apache.flink.runtime.taskmanager.Task.run(Task.java:668) at java.lang.Thread.run(Thread.java:745) {quote} The patch is as follows: {code} @@ -86,9 +86,10 @@ public class ContinuousProcessingTimeTrigger extends Trigger fireTimestamp = ctx.getPartitionedState(stateDesc); - long timestamp = fireTimestamp.get(); - ctx.deleteProcessingTimeTimer(timestamp); - fireTimestamp.clear(); + if (fireTimestamp.get() != null) { + ctx.deleteProcessingTimeTimer(fireTimestamp.get()); + fireTimestamp.clear(); + } } {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346)