[ https://issues.apache.org/jira/browse/BEAM-5687?focusedWorklogId=152749&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-152749 ]
ASF GitHub Bot logged work on BEAM-5687: ---------------------------------------- Author: ASF GitHub Bot Created on: 09/Oct/18 16:15 Start Date: 09/Oct/18 16:15 Worklog Time Spent: 10m Work Description: mxm opened a new pull request #6617: [BEAM-5687] Fix checkpointing of FlinkRunner for portable pipelines URL: https://github.com/apache/beam/pull/6617 ### [BEAM-5687] Fix checkpointing of FlinkRunner for portable pipelines This provides the input WindowValue Coder to ExecutableStageDoFnOperator which ensures that the buffered elements can be checkpointed correctly. ### [BEAM-3727] Do not shutdown Impulse sources to enable checkpointing Flink's checkpointing won't work properly after sources have finished. They need to be up and running for as long as checkpoints should be taken. This was already the case for the non-portable UnboundedSourceWrapper but it needs to be extended also for Impulse transforms. ### [BEAM-3727] Allow sources to shutdown when checkpointing is disabled CC @tweise @angoenka Post-Commit Tests Status (on master branch) ------------------------------------------------------------------------------------------------ Lang | SDK | Apex | Dataflow | Flink | Gearpump | Samza | Spark --- | --- | --- | --- | --- | --- | --- | --- Go | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Go_GradleBuild/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Go_GradleBuild/lastCompletedBuild/) | --- | --- | --- | --- | --- | --- Java | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_GradleBuild/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Apex_Gradle/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Gradle/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Gradle/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Gearpump_Gradle/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza_Gradle/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark_Gradle/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark_Gradle/lastCompletedBuild/) Python | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_Verify/lastCompletedBuild/) | --- | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/) </br> [![Build Status](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon)](https://builds.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/) | --- | --- | --- ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 152749) Time Spent: 10m Remaining Estimate: 0h > Checkpointing in portable pipelines does not work > ------------------------------------------------- > > Key: BEAM-5687 > URL: https://issues.apache.org/jira/browse/BEAM-5687 > Project: Beam > Issue Type: Bug > Components: runner-flink > Reporter: Maximilian Michels > Assignee: Maximilian Michels > Priority: Major > Labels: portability-flink > Fix For: 2.9.0 > > Time Spent: 10m > Remaining Estimate: 0h > > Checkpoints fail: > {noformat} > AsynchronousException{java.lang.Exception: Could not materialize checkpoint 2 > for operator Source: Custom Source -> 9TestInput.None/beam:env:docker:v1:0 -> > ToKeyedWorkItem (1/1).} > at > org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointExceptionHandler.tryHandleCheckpointException(StreamTask.java:1154) > at > org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.handleExecutionException(StreamTask.java:948) > at > org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:885) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.lang.Exception: Could not materialize checkpoint 2 for > operator Source: Custom Source -> 9TestInput.None/beam:env:docker:v1:0 -> > ToKeyedWorkItem (1/1). > at > org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.handleExecutionException(StreamTask.java:943) > ... 6 more > Caused by: java.util.concurrent.ExecutionException: > java.lang.NullPointerException > at java.util.concurrent.FutureTask.report(FutureTask.java:122) > at java.util.concurrent.FutureTask.get(FutureTask.java:192) > at > org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:53) > at > org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:53) > at > org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:854) > ... 5 more > Caused by: java.lang.NullPointerException > at > org.apache.beam.runners.flink.translation.types.CoderTypeSerializer$CoderTypeSerializerConfigSnapshot.<init>(CoderTypeSerializer.java:162) > at > org.apache.beam.runners.flink.translation.types.CoderTypeSerializer.snapshotConfiguration(CoderTypeSerializer.java:136) > at > org.apache.flink.runtime.state.RegisteredOperatorBackendStateMetaInfo.snapshot(RegisteredOperatorBackendStateMetaInfo.java:93) > at > org.apache.flink.runtime.state.DefaultOperatorStateBackend$1.performOperation(DefaultOperatorStateBackend.java:394) > at > org.apache.flink.runtime.state.DefaultOperatorStateBackend$1.performOperation(DefaultOperatorStateBackend.java:352) > at > org.apache.flink.runtime.io.async.AbstractAsyncCallableWithResources.call(AbstractAsyncCallableWithResources.java:75) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:50) > ... 7 more > {noformat} -- This message was sent by Atlassian JIRA (v7.6.3#76005)