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

NPE on TumblingEventTimeWindows with ContinuousEventTimeTrigger and allowedLateness

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • 1.2.0
    • 1.2.1, 1.3.0
    • API / DataStream
    • 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

      Attachments

        Issue Links

          Activity

            People

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

              Dates

                Created:
                Updated:
                Resolved: