Whether a Flink task should fail in case of a checkpoint error or not can
be configured via the CheckpointConfig which you can access via the
StreamExecutionEnvironment. You have to call
`CheckpointConfig#setFailOnCheckpointingErrors(false)` to deactivate the
default behaviour where the task always fails in case of a checkpoint error.

Cheers,
Till

On Tue, Jul 10, 2018 at 10:50 AM Vishal Santoshi <[email protected]>
wrote:

> That makes sense, what does not make sense is that the pipeline restarted.
> I would have imagined that an aborted chk point would not abort the
> pipeline.
>
> On Tue, Jul 10, 2018 at 3:16 AM, Till Rohrmann <[email protected]>
> wrote:
>
>> Hi Vishal,
>>
>> it looks as if the flushing of the checkpoint data to HDFS failed due to
>> some expired lease on the checkpoint file. Therefore, Flink aborted the
>> checkpoint `chk-125` and removed it. This is the normal behaviour if Flink
>> cannot complete a checkpoint. As you can see, afterwards, the checkpoints
>> are again successful.
>>
>> Cheers,
>> Till
>>
>> On Mon, Jul 9, 2018 at 7:15 PM Vishal Santoshi <[email protected]>
>> wrote:
>>
>>> drwxr-xr-x   - root hadoop          0 2018-07-09 12:33
>>> /flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-123
>>> drwxr-xr-x   - root hadoop          0 2018-07-09 12:35
>>> /flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-124
>>> drwxr-xr-x   - root hadoop          0 2018-07-09 12:51
>>> /flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-126
>>> drwxr-xr-x   - root hadoop          0 2018-07-09 12:53
>>> /flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-127
>>> drwxr-xr-x   - root hadoop          0 2018-07-09 12:55
>>> /flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-128
>>>
>>> See the missing chk-125
>>>
>>> So I see the above checkpoints for a job. at the  2018-07-09, 12:38:43
>>>  this exception was thrown
>>>
>>>
>>> the  chk-125 is missing from hdfs and the job complains about it
>>> Caused by:
>>> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException):
>>> No lease on
>>> /flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-125/e9d6886c-e693-4827-97bc-dd3fd526b64e
>>> (inode 1987098987): File does not exist. Holder
>>> DFSClient_NONMAPREDUCE_1527557459_11240 does not have any open files.
>>>
>>> At about the same time
>>>
>>> ID: 125Failure Time: 12:38:23Cause: Checkpoint expired before
>>> completing..
>>>
>>>
>>> Is this some race condition. A checkpoint had to be taken and , that was
>>> was chk-125, it took longer than the configure time ( 1 minute ).  It
>>> aborted the pipe. Should it have ? It actually did not even create the 
>>> chk-125
>>> but then refers to it and aborts the pipe.
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>>
>>> This is the full exception.
>>>
>>> AsynchronousException{java.lang.Exception: Could not materialize checkpoint 
>>> 125 for operator 360 minute interval -> 360 minutes to TimeSeries.Entry.2 
>>> (5/6).}
>>>     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 125 for 
>>> operator 360 minute interval -> 360 minutes to TimeSeries.Entry.2 (5/6).
>>>     at 
>>> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.handleExecutionException(StreamTask.java:943)
>>>     ... 6 more
>>> Caused by: java.util.concurrent.ExecutionException: java.io.IOException: 
>>> Could not flush and close the file system output stream to 
>>> hdfs://nn-crunchy.bf2.tumblr.net:8020/flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-125/e9d6886c-e693-4827-97bc-dd3fd526b64e
>>>  in order to obtain the stream state handle
>>>     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:47)
>>>     at 
>>> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:854)
>>>     ... 5 more
>>> Caused by: java.io.IOException: Could not flush and close the file system 
>>> output stream to 
>>> hdfs://nn-crunchy.bf2.tumblr.net:8020/flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-125/e9d6886c-e693-4827-97bc-dd3fd526b64e
>>>  in order to obtain the stream state handle
>>>     at 
>>> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.closeAndGetHandle(FsCheckpointStreamFactory.java:325)
>>>     at 
>>> org.apache.flink.runtime.state.CheckpointStreamWithResultProvider$PrimaryStreamOnly.closeAndFinalizeCheckpointStreamResult(CheckpointStreamWithResultProvider.java:77)
>>>     at 
>>> org.apache.flink.runtime.state.heap.HeapKeyedStateBackend$HeapSnapshotStrategy$1.performOperation(HeapKeyedStateBackend.java:705)
>>>     at 
>>> org.apache.flink.runtime.state.heap.HeapKeyedStateBackend$HeapSnapshotStrategy$1.performOperation(HeapKeyedStateBackend.java:641)
>>>     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)
>>>
>>> Caused by: 
>>> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException):
>>>  No lease on 
>>> /flink/kpi_unique/392d0436e53f3ef5e494ba3cc63428bf/chk-125/e9d6886c-e693-4827-97bc-dd3fd526b64e
>>>  (inode 1987098987): File does not exist. Holder 
>>> DFSClient_NONMAPREDUCE_1527557459_11240 does not have any open files.
>>>
>>>
>>>
>>>
>

Reply via email to