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

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

Github user StefanRRichter commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3074#discussion_r95589363
  
    --- Diff: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java
 ---
    @@ -280,4 +368,50 @@ public void restoreState(Serializable state) throws 
Exception {
                        fail("should never be called");
                }
        }
    +
    +   private static class TestSourceOperator implements 
SourceFunction<Object>, CheckpointedFunction {
    +           private static final long serialVersionUID = 1L;
    +
    +           @Override
    +           public void run(SourceContext<Object> ctx) throws Exception {
    +                   fail("should never be called");
    +           }
    +
    +           @Override
    +           public void cancel() {}
    +
    +
    +           @Override
    +           public void snapshotState(FunctionSnapshotContext context) 
throws Exception {
    +                   fail("should never be called");
    +           }
    +
    +           @Override
    +           public void initializeState(FunctionInitializationContext 
context) throws Exception {
    +                   
((StateInitializationContext)context).getRawOperatorStateInputs().iterator().next().getStream().read();
    +           }
    +   }
    +
    +   private static class TestSourceKeyed implements SourceFunction<Object>, 
CheckpointedFunction {
    +           private static final long serialVersionUID = 1L;
    +
    +           @Override
    +           public void run(SourceContext<Object> ctx) throws Exception {
    +                   fail("should never be called");
    +           }
    +
    +           @Override
    +           public void cancel() {}
    +
    +
    +           @Override
    +           public void snapshotState(FunctionSnapshotContext context) 
throws Exception {
    +                   fail("should never be called");
    +           }
    +
    +           @Override
    +           public void initializeState(FunctionInitializationContext 
context) throws Exception {
    +                   
((StateInitializationContext)context).getRawKeyedStateInputs().iterator().next().getStream().read();
    --- End diff --
    
    Agreed. Will simplify the test accordingly.


> Explicit restore method in Snapshotable
> ---------------------------------------
>
>                 Key: FLINK-5421
>                 URL: https://issues.apache.org/jira/browse/FLINK-5421
>             Project: Flink
>          Issue Type: Improvement
>            Reporter: Stefan Richter
>            Assignee: Stefan Richter
>
> We should introduce an explicit {{restore(...)}} method to match the 
> {{snapshot(...)}} method in this interface.
> Currently, restore happens implicit in backends, i.e. when state handles are 
> provided, backends execute restore logic in their constructors. This 
> behaviour makes it hard for backends to participate in the task's lifecycle 
> through {{CloseableRegistry}}, because we can only register backend objects 
> after they have been constructed. As a result, for example, all restore 
> operations that happen in the constructor are not responsive to cancelation.
> When we introduce an explicit restore, we can first create a backend object, 
> then register it, and only then run restore.



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

Reply via email to