flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-4862) NPE on EventTimeSessionWindows with ContinuousEventTimeTrigger
Date Fri, 21 Oct 2016 04:50:59 GMT

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

ASF GitHub Bot commented on FLINK-4862:
---------------------------------------

GitHub user manuzhang opened a pull request:

    https://github.com/apache/flink/pull/2671

    [FLINK-4862] fix Timer register in ContinuousEventTimeTrigger

    Thanks for contributing to Apache Flink. Before you open your pull request, please take
the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request.
For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your
changes.
    
    - [ ] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [ ] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [ ] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/manuzhang/flink fix_merge_window

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2671.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2671
    
----
commit b2370946357044250511e25fce5078812ad22c82
Author: manuzhang <owenzhang1990@gmail.com>
Date:   2016-10-20T07:06:01Z

    [FLINK-4862] fix Timer register in ContinuousEventTimeTrigger

----


> 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
>            Reporter: Manu Zhang
>
> 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)

Mime
View raw message