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 31D51200BBA for ; Fri, 21 Oct 2016 15:17:00 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 30769160AE0; Fri, 21 Oct 2016 13:17:00 +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 775CA160AE8 for ; Fri, 21 Oct 2016 15:16:59 +0200 (CEST) Received: (qmail 97451 invoked by uid 500); 21 Oct 2016 13:16:58 -0000 Mailing-List: contact issues-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 issues@flink.apache.org Received: (qmail 97421 invoked by uid 99); 21 Oct 2016 13:16:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 21 Oct 2016 13:16:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 7079E2C0D53 for ; Fri, 21 Oct 2016 13:16:58 +0000 (UTC) Date: Fri, 21 Oct 2016 13:16:58 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-4862) NPE on EventTimeSessionWindows with ContinuousEventTimeTrigger MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 21 Oct 2016 13:17:00 -0000 [ https://issues.apache.org/jira/browse/FLINK-4862?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15595033#comment-15595033 ] ASF GitHub Bot commented on FLINK-4862: --------------------------------------- Github user mxm commented on a diff in the pull request: https://github.com/apache/flink/pull/2671#discussion_r84464167 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousEventTimeTrigger.java --- @@ -88,9 +96,13 @@ public TriggerResult onProcessingTime(long time, W window, TriggerContext ctx) t @Override public void clear(W window, TriggerContext ctx) throws Exception { ReducingState fireTimestamp = ctx.getPartitionedState(stateDesc); - long timestamp = fireTimestamp.get(); - ctx.deleteEventTimeTimer(timestamp); - fireTimestamp.clear(); + Long timestamp = fireTimestamp.get(); + if (timestamp != null) { + ctx.deleteEventTimeTimer(timestamp); + fireTimestamp.clear(); + } else if (cachedFireTimestamp != null){ + ctx.deleteEventTimeTimer(cachedFireTimestamp); + } --- End diff -- The above `else if` block is not correct because there is only one instance of the trigger which is reused for each Window. Hence the abstraction using the state descriptor to retrieve the appropriate state. > NPE on EventTimeSessionWindows with ContinuousEventTimeTrigger > -------------------------------------------------------------- > > Key: FLINK-4862 > URL: https://issues.apache.org/jira/browse/FLINK-4862 > Project: Flink > Issue Type: Bug > Components: Streaming, Windowing Operators > Affects Versions: 1.2.0, 1.1.3 > Reporter: Manu Zhang > Assignee: Manu Zhang > Fix For: 1.2.0, 1.1.4 > > > h3. what's the error ? > The following NPE error is thrown when EventTimeSessionWindows with ContinuousEventTimeTrigger is used. > {code} > Caused by: java.lang.NullPointerException > at org.apache.flink.streaming.api.windowing.triggers.ContinuousEventTimeTrigger.clear(ContinuousEventTimeTrigger.java:91) > at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$Context.clear(WindowOperator.java:768) > at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$2.merge(WindowOperator.java:310) > at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$2.merge(WindowOperator.java:297) > at org.apache.flink.streaming.runtime.operators.windowing.MergingWindowSet.addWindow(MergingWindowSet.java:196) > at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.processElement(WindowOperator.java:297) > at org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:183) > at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:66) > at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:271) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:609) > at java.lang.Thread.run(Thread.java:745) > {code} > h3. how to reproduce ? > use {{ContinuousEventTimeTrigger}} instead of the default {{EventTimeTrigger}} in [SessionWindowing | https://github.com/apache/flink/blob/master/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/windowing/SessionWindowing.java#L84] example. > h3. what's the cause ? > When two session windows are being merged, the states of the two {{ContinuousEventTimeTrigger}} are merged as well and the new namespace is the merged window. Later when the context tries to delete {{Timer}} from the old trigger and looks up the timestamp by the old namespace, null value is returned. -- This message was sent by Atlassian JIRA (v6.3.4#6332)