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

Robert Metzger commented on FLINK-5229:
---------------------------------------

I'm seeing the following exceptions in my 1.2.0 RC1 code

{code}
2017-01-25 12:47:48,775 WARN  
org.apache.flink.streaming.runtime.tasks.StreamTask           - Could not 
properly clean up the async checkpoint runnable.
java.lang.Exception: Could not properly cancel managed operator state future.
        at 
org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:99)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.cleanup(StreamTask.java:992)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.close(StreamTask.java:979)
        at org.apache.flink.util.IOUtils.closeQuietly(IOUtils.java:224)
        at 
org.apache.flink.util.AbstractCloseableRegistry.close(AbstractCloseableRegistry.java:92)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.cancel(StreamTask.java:363)
        at 
org.apache.flink.runtime.taskmanager.Task$TaskCanceler.run(Task.java:1384)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.NullPointerException
        at 
org.apache.flink.runtime.state.StateUtil.discardStateFuture(StateUtil.java:81)
        at 
org.apache.flink.streaming.api.operators.OperatorSnapshotResult.cancel(OperatorSnapshotResult.java:97)
        ... 7 more
{code}

I'm asking here because its code that has been touched while addressing this 
JIRA.

How critical is this?

> Cleanup StreamTaskStates if a checkpoint operation of a subsequent operator 
> fails 
> ----------------------------------------------------------------------------------
>
>                 Key: FLINK-5229
>                 URL: https://issues.apache.org/jira/browse/FLINK-5229
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing, TaskManager
>    Affects Versions: 1.2.0, 1.1.3
>            Reporter: Till Rohrmann
>            Assignee: Till Rohrmann
>             Fix For: 1.2.0, 1.1.4, 1.3.0
>
>
> Due to chaining, a {{StreamTask}} needs to checkpoint multiple operators. If 
> the first operators succeed in creating a checkpoint but a subsequent 
> operator in the chain fails, the {{StreamTask}} has to clean up the already 
> completed checkpoints. Otherwise we might end up with orphaned state data.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to