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

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

GitHub user asavartsov opened a pull request:

    https://github.com/apache/flink/pull/2062

    [FLINK-4000] Fix for checkpoint state restore at 
MessageAcknowledgingSourceBase

    As says documentation for MessageAcknowledgingSourceBase.restoreState() 
    
    This method is invoked when a function is executed as part of a recovery 
run. Note that restoreState() is called before open().
    
    So current implementation
    
    1. Fails on restoreState with NullPointerException, jobs fail to restart.
    2. Does not restore anything because following open erases all checkpoint 
data immediately.
    3. As consequence, violates exactly once rule because processed but not 
acknowledged list erased.
    
    Proposed change fixes that.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/asavartsov/flink FLINK-4000

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2062.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2062
    
----
commit 58fb7df0b2d1dd68a90f4088fbde10874eb290b6
Author: Alexey Savartsov <asavart...@gmail.com>
Date:   2016-06-01T23:23:53Z

    [FLINK-4000] Fix for checkpoint state restore at 
MessageAcknowledgingSourceBase
    
    As says documentation for MessageAcknowledgingSourceBase.restoreState() 
    
    This method is invoked when a function is executed as part of a recovery 
run. Note that restoreState() is called before open().
    
    So current implementation
    
    1. Fails on restoreState with NullPointerException, jobs fail to restart.
    2. Does not restore anything because following open erases all checkpoint 
data immediately.
    3. As consequence, violates exactly once rule because processed but not 
acknowledged list erased.
    
    Proposed change fixes that.

----


> Exception: Could not restore checkpointed state to operators and functions;  
> during Job Restart (Job restart is triggered due to one of the task manager 
> failure)
> -----------------------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: FLINK-4000
>                 URL: https://issues.apache.org/jira/browse/FLINK-4000
>             Project: Flink
>          Issue Type: Bug
>          Components: DataStream API
>    Affects Versions: 1.0.3
>         Environment: //Fault Tolerance Configuration of the Job
> env.enableCheckpointing(5000);     
> env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE);
> env.getCheckpointConfig().setMaxConcurrentCheckpoints(1);
> env.setRestartStrategy(RestartStrategies.fixedDelayRestart( 3,10000));
>            Reporter: Aride Chettali
>            Assignee: Rekha Joshi
>
> java.lang.Exception: Could not restore checkpointed state to operators and 
> functions
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreState(StreamTask.java:457)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:209)
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:559)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.Exception: Failed to restore state to function: null
>       at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:168)
>       at 
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreState(StreamTask.java:449)
>       ... 3 more
> Caused by: java.lang.NullPointerException
>       at 
> org.apache.flink.streaming.api.functions.source.MessageAcknowledgingSourceBase.restoreState(MessageAcknowledgingSourceBase.java:184)
>       at 
> org.apache.flink.streaming.api.functions.source.MessageAcknowledgingSourceBase.restoreState(MessageAcknowledgingSourceBase.java:80)
>       at 
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.restoreState(AbstractUdfStreamOperator.java:165)
>       ... 4 more



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

Reply via email to