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

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

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

    https://github.com/apache/flink/pull/4851#discussion_r145917932
  
    --- Diff: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/RestoreStreamTaskTest.java
 ---
    @@ -0,0 +1,325 @@
    +/*
    + * 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.state.ListState;
    +import org.apache.flink.api.common.state.ListStateDescriptor;
    +import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
    +import org.apache.flink.api.common.typeutils.base.LongSerializer;
    +import org.apache.flink.api.common.typeutils.base.StringSerializer;
    +import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
    +import org.apache.flink.runtime.checkpoint.CheckpointOptions;
    +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
    +import org.apache.flink.runtime.checkpoint.StateAssignmentOperation;
    +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
    +import org.apache.flink.runtime.jobgraph.OperatorID;
    +import org.apache.flink.runtime.jobgraph.OperatorInstanceID;
    +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
    +import org.apache.flink.runtime.state.StateInitializationContext;
    +import org.apache.flink.runtime.state.StateSnapshotContext;
    +import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
    +import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
    +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
    +import org.apache.flink.streaming.util.TestHarnessUtil;
    +import org.apache.flink.util.TestLogger;
    +
    +import org.junit.Test;
    +
    +import java.util.Collections;
    +import java.util.Optional;
    +import java.util.concurrent.ConcurrentLinkedQueue;
    +import java.util.concurrent.atomic.AtomicLong;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +/**
    + * Tests ensuring correct behaviour of {@link 
org.apache.flink.runtime.state.ManagedInitializationContext#isRestored}
    + * method.
    + */
    +public class RestoreStreamTaskTest extends TestLogger {
    +   @Test
    +   public void testRestore() throws Exception {
    +           AcknowledgeStreamMockEnvironment environment1 = processRecords(
    +                   new OperatorID(42L, 42L),
    +                   new CounterOperator(),
    +                   new OperatorID(44L, 44L),
    +                   new CounterOperator(),
    +                   Optional.empty());
    +
    +           assertEquals(2, 
environment1.getCheckpointStateHandles().getSubtaskStateMappings().size());
    +
    +           TaskStateSnapshot stateHandles = 
environment1.getCheckpointStateHandles();
    +
    +           try {
    +                   AcknowledgeStreamMockEnvironment environment2 = 
processRecords(
    +                           new OperatorID(42L, 42L),
    +                           new RestoreCounterOperator(),
    +                           new OperatorID(44L, 44L),
    +                           new RestoreCounterOperator(),
    +                           Optional.of(stateHandles));
    +
    +                   assertEquals(2, 
RestoreCounterOperator.RESTORE_COUNTER.get());
    +           }
    +           finally {
    +                   RestoreCounterOperator.RESTORE_COUNTER.getAndSet(0);
    +           }
    +   }
    +
    +   @Test
    +   public void testRestoreWithNewId() throws Exception {
    +           AcknowledgeStreamMockEnvironment environment1 = processRecords(
    +                   new OperatorID(42L, 42L),
    +                   new CounterOperator(),
    +                   new OperatorID(44L, 44L),
    +                   new CounterOperator(),
    +                   Optional.empty());
    +
    +           assertEquals(2, 
environment1.getCheckpointStateHandles().getSubtaskStateMappings().size());
    +
    +           TaskStateSnapshot stateHandles = 
environment1.getCheckpointStateHandles();
    +
    +           try {
    +                   AcknowledgeStreamMockEnvironment environment2 = 
processRecords(
    +                           new OperatorID(4242L, 4242L),
    +                           new RestoreCounterOperator(),
    +                           new OperatorID(44L, 44L),
    +                           new RestoreCounterOperator(),
    +                           Optional.of(stateHandles));
    +
    +                   assertEquals(1, 
RestoreCounterOperator.RESTORE_COUNTER.get());
    +           }
    +           finally {
    +                   RestoreCounterOperator.RESTORE_COUNTER.getAndSet(0);
    +           }
    +   }
    +
    +   @Test
    +   public void testRestoreAfterScaleUp() throws Exception {
    +           OperatorID headOperatorID = new OperatorID(42L, 42L);
    +           OperatorID tailOperatorID = new OperatorID(44L, 44L);
    +
    +           AcknowledgeStreamMockEnvironment environment1 = processRecords(
    +                   headOperatorID,
    +                   new CounterOperator(),
    +                   tailOperatorID,
    +                   new CounterOperator(),
    +                   Optional.empty());
    +
    +           assertEquals(2, 
environment1.getCheckpointStateHandles().getSubtaskStateMappings().size());
    +
    +           // test empty state in case of scale up
    +           OperatorSubtaskState emptyHeadOperatorState = 
StateAssignmentOperation.operatorSubtaskStateFrom(
    +                   new OperatorInstanceID(0, headOperatorID),
    +                   Collections.emptyMap(),
    +                   Collections.emptyMap(),
    +                   Collections.emptyMap(),
    +                   Collections.emptyMap());
    +
    +           TaskStateSnapshot stateHandles = 
environment1.getCheckpointStateHandles();
    +           stateHandles.putSubtaskStateByOperatorID(headOperatorID, 
emptyHeadOperatorState);
    +
    +           try {
    +                   AcknowledgeStreamMockEnvironment environment2 = 
processRecords(
    +                           headOperatorID,
    +                           new RestoreCounterOperator(),
    +                           tailOperatorID,
    +                           new RestoreCounterOperator(),
    +                           Optional.of(stateHandles));
    +
    +                   assertEquals(2, 
RestoreCounterOperator.RESTORE_COUNTER.get());
    +           }
    +           finally {
    +                   RestoreCounterOperator.RESTORE_COUNTER.getAndSet(0);
    +           }
    +   }
    +
    +   @Test
    +   public void testRestoreWithoutState() throws Exception {
    +           OperatorID headOperatorID = new OperatorID(42L, 42L);
    +           OperatorID tailOperatorID = new OperatorID(44L, 44L);
    +
    +           AcknowledgeStreamMockEnvironment environment1 = processRecords(
    +                   headOperatorID,
    +                   new StatelessRestoreCounterOperator(),
    +                   tailOperatorID,
    +                   new CounterOperator(),
    +                   Optional.empty());
    +
    +           assertEquals(2, 
environment1.getCheckpointStateHandles().getSubtaskStateMappings().size());
    +
    +           TaskStateSnapshot stateHandles = 
environment1.getCheckpointStateHandles();
    +
    +           try {
    +                   AcknowledgeStreamMockEnvironment environment2 = 
processRecords(
    +                           headOperatorID,
    +                           new StatelessRestoreCounterOperator(),
    +                           tailOperatorID,
    +                           new RestoreCounterOperator(),
    +                           Optional.of(stateHandles));
    +
    +                   assertEquals(1, 
StatelessRestoreCounterOperator.RESTORE_COUNTER.get());
    --- End diff --
    
    `StatelessRestoreCounterOperator` and `RestoreCounterOperator` are using 
different counters. However that leads to my mistake, because this shows, that 
even stateless operators are getting `isRestore() == true`.


> 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