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

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

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

    https://github.com/apache/flink/pull/4851#discussion_r145921058
  
    --- Diff: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/AcknowledgeStreamMockEnvironment.java
 ---
    @@ -0,0 +1,65 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.streaming.runtime.tasks;
    +
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.testutils.OneShotLatch;
    +import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
    +
    +/**
    + * Stream environment that allows to wait for checkpoint acknowledgement.
    + */
    +class AcknowledgeStreamMockEnvironment extends StreamMockEnvironment {
    --- End diff --
    
    `RocksDBAsyncSnapshotTest` does something very similar in anonymous class


> 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