Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-6001

NPE on TumblingEventTimeWindows with ContinuousEventTimeTrigger and allowedLateness

    Details

    • Type: Bug
    • Status: Closed
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: 1.2.0
    • Fix Version/s: 1.3.0, 1.2.1
    • Component/s: DataStream API, Streaming
    • Labels:
      None

      Description

      I try to isolate the problem in a small and simple reproducer by extracting the data from my real setup.

      I fails with NPE at :

      java.lang.NullPointerException: null
      	at org.apache.flink.streaming.api.windowing.triggers.ContinuousEventTimeTrigger.onEventTime(ContinuousEventTimeTrigger.java:81) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator$Context.onEventTime(WindowOperator.java:721) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.onEventTime(WindowOperator.java:425) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.streaming.api.operators.HeapInternalTimerService.advanceWatermark(HeapInternalTimerService.java:276) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.processWatermark(AbstractStreamOperator.java:858) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.streaming.runtime.io.StreamInputProcessor.processInput(StreamInputProcessor.java:168) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.run(OneInputStreamTask.java:63) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:272) ~[flink-streaming-java_2.11-1.2.0.jar:1.2.0]
      	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:655) ~[flink-runtime_2.11-1.2.0.jar:1.2.0]
      	at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_121]
      

      It fails only with the Thread.sleep. If you uncomment it, it won't fail.
      So, you may have to increase the sleep time depending of your environment.
      I know this is not a very rigourous test, but this is the only way I've found to reproduce it.

      You can find the reproducer here :
      https://github.com/vpernin/flink-window-npe

        Issue Links

          Activity

          Hide
          vpernin Vladislav Pernin added a comment -

          I have simplified the reproducer but it fails less often. Please use the following branch :
          https://github.com/vpernin/flink-window-npe/tree/simpler-but-fails-less-often

          Show
          vpernin Vladislav Pernin added a comment - I have simplified the reproducer but it fails less often. Please use the following branch : https://github.com/vpernin/flink-window-npe/tree/simpler-but-fails-less-often
          Hide
          vpernin Vladislav Pernin added a comment -

          Another reproducer version without a "sleeping map" but a slow source function that try to mimic the reality :
          https://github.com/vpernin/flink-window-npe/tree/slow-serializer

          Show
          vpernin Vladislav Pernin added a comment - Another reproducer version without a "sleeping map" but a slow source function that try to mimic the reality : https://github.com/vpernin/flink-window-npe/tree/slow-serializer
          Hide
          vpernin Vladislav Pernin added a comment -

          Maybe related to FLINK-5713 ? I have to test.

          Show
          vpernin Vladislav Pernin added a comment - Maybe related to FLINK-5713 ? I have to test.
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user aljoscha opened a pull request:

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

          FLINK-6001 Fix ContinuousEventTimeTrigger firing without state

          R: @kl0u

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

          $ git pull https://github.com/aljoscha/flink jira-6001-fix-cont-trigger-npe

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

          https://github.com/apache/flink/pull/3544.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 #3544


          commit 31fd820666d1715c57298c3f4d8a434a96431a1f
          Author: Aljoscha Krettek <aljoscha.krettek@gmail.com>
          Date: 2017-03-13T14:04:01Z

          FLINK-6001 Fix ContinuousEventTimeTrigger firing without state


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user aljoscha opened a pull request: https://github.com/apache/flink/pull/3544 FLINK-6001 Fix ContinuousEventTimeTrigger firing without state R: @kl0u You can merge this pull request into a Git repository by running: $ git pull https://github.com/aljoscha/flink jira-6001-fix-cont-trigger-npe Alternatively you can review and apply these changes as the patch at: https://github.com/apache/flink/pull/3544.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 #3544 commit 31fd820666d1715c57298c3f4d8a434a96431a1f Author: Aljoscha Krettek <aljoscha.krettek@gmail.com> Date: 2017-03-13T14:04:01Z FLINK-6001 Fix ContinuousEventTimeTrigger firing without state
          Hide
          vpernin Vladislav Pernin added a comment -

          Very nice, it works.
          I was reluctant to push a PR myself with your NPE protection without being aware of possible side effects.

          Show
          vpernin Vladislav Pernin added a comment - Very nice, it works. I was reluctant to push a PR myself with your NPE protection without being aware of possible side effects.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user aljoscha closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user aljoscha closed the pull request at: https://github.com/apache/flink/pull/3544
          Hide
          aljoscha Aljoscha Krettek added a comment -

          Fixed on release-1.2 in
          697ede00d044af4eb18a981fd2734fba3e112a30

          Fixed on master in
          0a501e9f7f56baba2905002b74746998458db007

          Show
          aljoscha Aljoscha Krettek added a comment - Fixed on release-1.2 in 697ede00d044af4eb18a981fd2734fba3e112a30 Fixed on master in 0a501e9f7f56baba2905002b74746998458db007

            People

            • Assignee:
              aljoscha Aljoscha Krettek
              Reporter:
              vpernin Vladislav Pernin
            • Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development