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

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

zentol commented on issue #7013: [FLINK-5601][Checkpointing] Watermark 
checkpointing
URL: https://github.com/apache/flink/pull/7013#issuecomment-435927023
 
 
   Related test-failure:
   ```
   Tests run: 12, Failures: 0, Errors: 3, Skipped: 0, Time elapsed: 10.549 sec 
<<< FAILURE! - in org.apache.flink.test.streaming.runtime.TimestampITCase
   
testTimestampExtractorWithAutoInterval(org.apache.flink.test.streaming.runtime.TimestampITCase)
  Time elapsed: 0.093 sec  <<< ERROR!
   org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
        at 
org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146)
        at 
org.apache.flink.runtime.minicluster.MiniCluster.executeJobBlocking(MiniCluster.java:647)
        at 
org.apache.flink.streaming.util.TestStreamEnvironment.execute(TestStreamEnvironment.java:79)
        at 
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1510)
        at 
org.apache.flink.test.streaming.runtime.TimestampITCase.testTimestampExtractorWithAutoInterval(TimestampITCase.java:346)
   Caused by: java.lang.NullPointerException: null
        at 
org.apache.flink.test.streaming.runtime.TimestampITCase$CustomOperator.processWatermark(TimestampITCase.java:713)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$ChainingOutput.emitWatermark(OperatorChain.java:500)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.emitWatermark(AbstractStreamOperator.java:702)
        at 
org.apache.flink.streaming.runtime.operators.TimestampsAndPeriodicWatermarksOperator.initializeState(TimestampsAndPeriodicWatermarksOperator.java:137)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:278)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:704)
        at java.lang.Thread.run(Thread.java:748)
   
   
testTimestampExtractorWithDecreasingCustomWatermarkEmit(org.apache.flink.test.streaming.runtime.TimestampITCase)
  Time elapsed: 0.092 sec  <<< ERROR!
   org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
        at 
org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146)
        at 
org.apache.flink.runtime.minicluster.MiniCluster.executeJobBlocking(MiniCluster.java:647)
        at 
org.apache.flink.streaming.util.TestStreamEnvironment.execute(TestStreamEnvironment.java:79)
        at 
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1510)
        at 
org.apache.flink.test.streaming.runtime.TimestampITCase.testTimestampExtractorWithDecreasingCustomWatermarkEmit(TimestampITCase.java:469)
   Caused by: java.lang.NullPointerException: null
        at 
org.apache.flink.test.streaming.runtime.TimestampITCase$CustomOperator.processWatermark(TimestampITCase.java:713)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$ChainingOutput.emitWatermark(OperatorChain.java:500)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.emitWatermark(AbstractStreamOperator.java:702)
        at 
org.apache.flink.streaming.runtime.operators.TimestampsAndPunctuatedWatermarksOperator.initializeState(TimestampsAndPunctuatedWatermarksOperator.java:115)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:278)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:704)
        at java.lang.Thread.run(Thread.java:748)
   
   
testTimestampExtractorWithCustomWatermarkEmit(org.apache.flink.test.streaming.runtime.TimestampITCase)
  Time elapsed: 0.069 sec  <<< ERROR!
   org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
        at 
org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146)
        at 
org.apache.flink.runtime.minicluster.MiniCluster.executeJobBlocking(MiniCluster.java:647)
        at 
org.apache.flink.streaming.util.TestStreamEnvironment.execute(TestStreamEnvironment.java:79)
        at 
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1510)
        at 
org.apache.flink.test.streaming.runtime.TimestampITCase.testTimestampExtractorWithCustomWatermarkEmit(TimestampITCase.java:408)
   Caused by: java.lang.NullPointerException: null
        at 
org.apache.flink.test.streaming.runtime.TimestampITCase$CustomOperator.processWatermark(TimestampITCase.java:713)
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain$ChainingOutput.emitWatermark(OperatorChain.java:500)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator$CountingOutput.emitWatermark(AbstractStreamOperator.java:702)
        at 
org.apache.flink.streaming.runtime.operators.TimestampsAndPunctuatedWatermarksOperator.initializeState(TimestampsAndPunctuatedWatermarksOperator.java:115)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:278)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:738)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:289)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:704)
        at java.lang.Thread.run(Thread.java:748)
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Window operator does not checkpoint watermarks
> ----------------------------------------------
>
>                 Key: FLINK-5601
>                 URL: https://issues.apache.org/jira/browse/FLINK-5601
>             Project: Flink
>          Issue Type: Improvement
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.5.0, 1.6.0, 1.7.0
>            Reporter: Ufuk Celebi
>            Assignee: Jiayi Liao
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 1.8.0
>
>
> During release testing [[email protected]] and I noticed that 
> watermarks are not checkpointed in the window operator.
> This can lead to non determinism when restoring checkpoints. I was running an 
> adjusted {{SessionWindowITCase}} via Kafka for testing migration and 
> rescaling and ran into failures, because the data generator required 
> determinisitic behaviour.
> What happened was that on restore it could happen that late elements were not 
> dropped, because the watermarks needed to be re-established after restore 
> first.
> [~aljoscha] Do you know whether there is a special reason for explicitly not 
> checkpointing watermarks?



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to