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

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

GitHub user pnowojski opened a pull request:

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

    [FLINK-7623][tests] Add tests to make sure operator is never restored when 
using new operator id

    ## What is the purpose of the change
    
    This PR adds tests coverage for correct behaviour of 
`ManagedInitializationContext#isRestored` flag - if application is restarted 
and a some operator has a new `uid`, it should return false. This bug was fixed 
by #4353. 
    
    ## Brief change log
    
    Please check commit messages for change log
    
    ## Verifying this change
    
    This PR adds `RestoreStreamTaskTest` and is not changing any productional 
code.

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

    $ git pull https://github.com/pnowojski/flink f7623

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

    https://github.com/apache/flink/pull/4851.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 #4851
    
----
commit a99d384fcd5b021b71dc703d0e9e8063bd72f89e
Author: Piotr Nowojski <piotr.nowoj...@gmail.com>
Date:   2017-10-18T13:02:02Z

    [hotfix][streaming] Fix formatting in OperatorChain

commit 5a2972d5f02521166c70ad68ad3fac0df9fad2e8
Author: Piotr Nowojski <piotr.nowoj...@gmail.com>
Date:   2017-10-18T14:01:38Z

    [hotfix][tests] Add easier way to chain operator in StreamTaskTestHarness

commit 26cabd7762458633be891b746d06595a873033b4
Author: Piotr Nowojski <piotr.nowoj...@gmail.com>
Date:   2017-10-18T15:18:19Z

    [hotfix][tests] Extract AcknowledgeStreamMockEnvironment

commit 2d6b45a55b8c63df4635a2d5506eab4b5ab590c3
Author: Piotr Nowojski <piotr.nowoj...@gmail.com>
Date:   2017-10-18T13:01:37Z

    [FLINK-7623][tests] Add tests to make sure operator is never restored when 
using new operator id

----


> Detecting whether an operator is restored doesn't work with chained state
> -------------------------------------------------------------------------
>
>                 Key: FLINK-7623
>                 URL: https://issues.apache.org/jira/browse/FLINK-7623
>             Project: Flink
>          Issue Type: Bug
>          Components: DataStream API, State Backends, Checkpointing
>    Affects Versions: 1.4.0, 1.3.2
>            Reporter: Aljoscha Krettek
>            Assignee: Piotr Nowojski
>            Priority: Blocker
>             Fix For: 1.4.0, 1.3.3
>
>         Attachments: StreamingJob.java
>
>
> Originally reported on the ML: 
> https://lists.apache.org/thread.html/22a2cf83de3107aa81a03a921325a191c29df8aa8676798fcd497199@%3Cuser.flink.apache.org%3E
> If we have a chain of operators where multiple of them have operator state, 
> detection of the {{context.isRestored()}} flag (of {{CheckpointedFunction}}) 
> does not work correctly. It's best exemplified using this minimal example 
> where both the source and the flatMap have state:
> {code}
> final StreamExecutionEnvironment env = 
> StreamExecutionEnvironment.getExecutionEnvironment();
> env
>               .addSource(new MaSource()).uid("source-1")
>               .flatMap(new MaFlatMap()).uid("flatMap-1");
> env.execute("testing");
> {code}
> If I do a savepoint with these UIDs, then change "source-1" to "source-2" and 
> restore from the savepoint {{context.isRestored()}} still reports {{true}} 
> for the source.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to