zhuzhurk commented on a change in pull request #8922: [FLINK-12876][runtime] 
Add an adapter of region failover NG for legacy scheduler
URL: https://github.com/apache/flink/pull/8922#discussion_r298552634
 
 

 ##########
 File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest.java
 ##########
 @@ -0,0 +1,282 @@
+/*
+ * 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.runtime.executiongraph;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.blob.VoidBlobWriter;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ManuallyTriggeredScheduledExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import 
org.apache.flink.runtime.executiongraph.AdaptedRestartPipelinedRegionStrategyNGFailoverTest.TestAdaptedRestartPipelinedRegionStrategyNG;
+import 
org.apache.flink.runtime.executiongraph.failover.AdaptedRestartPipelinedRegionStrategyNG;
+import org.apache.flink.runtime.executiongraph.failover.FailoverStrategy;
+import org.apache.flink.runtime.executiongraph.utils.SimpleSlotProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.testingUtils.TestingUtils;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link AdaptedRestartPipelinedRegionStrategyNG} failover handling 
when concurrent failovers happen.
+ * There can be local+local and local+global concurrent failovers.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNGConcurrentFailoverTest 
extends TestLogger {
+
+       private static final JobID TEST_JOB_ID = new JobID();
+
+       private static final int DEFAULT_PARALLELISM = 2;
+
+       private ManuallyTriggeredScheduledExecutor manualMainThreadExecutor;
+
+       private ComponentMainThreadExecutor componentMainThreadExecutor;
+
+       private TestRestartStrategy manuallyTriggeredRestartStrategy;
+
+       @Before
+       public void setUp() {
+               manualMainThreadExecutor = new 
ManuallyTriggeredScheduledExecutor();
+               componentMainThreadExecutor = new 
ScheduledExecutorToComponentMainThreadExecutorAdapter(manualMainThreadExecutor, 
Thread.currentThread());
+               manuallyTriggeredRestartStrategy = 
TestRestartStrategy.manuallyTriggered();
+       }
+
+       /**
+        * Tests that 2 concurrent region failovers can lead to a properly 
vertex state.
+        * <pre>
+        *     (v11) -+-> (v21)
+        *            x
+        *     (v12) -+-> (v22)
+        *
+        *            ^
+        *            |
+        *       (blocking)
+        * </pre>
+        */
+       @Test
+       public void testConcurrentRegionFailovers() throws Exception {
+
+               // the logic in this test is as follows:
+               //  - start a job
+               //  - cause {ev11} failure and delay the local recovery action 
via the manual executor
+               //  - cause {ev12} failure and delay the local recovery action 
via the manual executor
+               //  - resume local recovery actions
+               //  - validate that each task is restarted only once
+
+               final ExecutionGraph eg = createExecutionGraph();
+
+               final TestAdaptedRestartPipelinedRegionStrategyNG 
failoverStrategy =
+                       (TestAdaptedRestartPipelinedRegionStrategyNG) 
eg.getFailoverStrategy();
+               failoverStrategy.setBlockerFuture(new CompletableFuture<>());
+
+               final Iterator<ExecutionVertex> vertexIterator = 
eg.getAllExecutionVertices().iterator();
+               final ExecutionVertex ev11 = vertexIterator.next();
+               final ExecutionVertex ev12 = vertexIterator.next();
+               final ExecutionVertex ev21 = vertexIterator.next();
+               final ExecutionVertex ev22 = vertexIterator.next();
+
+               // start job scheduling
+               eg.scheduleForExecution();
+               manualMainThreadExecutor.triggerAll();
+
+               // fail ev11 to trigger region failover of {ev11}, {ev21}, 
{ev22}
+               ev11.getCurrentExecutionAttempt().fail(new Exception("task 
failure 1"));
+               manualMainThreadExecutor.triggerAll();
+               assertEquals(ExecutionState.FAILED, ev11.getExecutionState());
+               assertEquals(ExecutionState.DEPLOYING, 
ev12.getExecutionState());
+               assertEquals(ExecutionState.CANCELED, ev21.getExecutionState());
+               assertEquals(ExecutionState.CANCELED, ev22.getExecutionState());
+
+               // fail ev12 to trigger region failover of {ev12}, {ev21}, 
{ev22}
+               ev12.getCurrentExecutionAttempt().fail(new Exception("task 
failure 2"));
+               manualMainThreadExecutor.triggerAll();
+               assertEquals(ExecutionState.FAILED, ev11.getExecutionState());
+               assertEquals(ExecutionState.FAILED, ev12.getExecutionState());
+               assertEquals(ExecutionState.CANCELED, ev21.getExecutionState());
+               assertEquals(ExecutionState.CANCELED, ev22.getExecutionState());
+
+               // complete region failover blocker to trigger region failover 
recovery
+               failoverStrategy.getBlockerFuture().complete(null);
+               manualMainThreadExecutor.triggerAll();
+
+               // verify that all tasks are recovered and no task is restarted 
more than once
+               assertEquals(ExecutionState.DEPLOYING, 
ev11.getExecutionState());
+               assertEquals(ExecutionState.DEPLOYING, 
ev12.getExecutionState());
+               assertEquals(ExecutionState.CREATED, ev21.getExecutionState());
+               assertEquals(ExecutionState.CREATED, ev22.getExecutionState());
+               assertEquals(1, 
ev11.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev12.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev21.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev22.getCurrentExecutionAttempt().getAttemptNumber());
+       }
+
+       /**
+        * Tests that a global failover will take precedence over local 
failovers.
+        * <pre>
+        *     (v11) -+-> (v21)
+        *            x
+        *     (v12) -+-> (v22)
+        *
+        *            ^
+        *            |
+        *       (blocking)
+        * </pre>
+        */
+       @Test
+       public void testRegionFailoverInterruptedByGlobalFailover() throws 
Exception {
+
+               // the logic in this test is as follows:
+               //  - start a job
+               //  - cause a task failure and delay the local recovery action 
via the manual executor
+               //  - cause a global failure
+               //  - resume in local recovery action
+               //  - validate that the local recovery does not restart tasks
+
+               final ExecutionGraph eg = createExecutionGraph();
+
+               final TestAdaptedRestartPipelinedRegionStrategyNG 
failoverStrategy =
+                       (TestAdaptedRestartPipelinedRegionStrategyNG) 
eg.getFailoverStrategy();
+               failoverStrategy.setBlockerFuture(new CompletableFuture<>());
+
+               final Iterator<ExecutionVertex> vertexIterator = 
eg.getAllExecutionVertices().iterator();
+               final ExecutionVertex ev11 = vertexIterator.next();
+               final ExecutionVertex ev12 = vertexIterator.next();
+               final ExecutionVertex ev21 = vertexIterator.next();
+               final ExecutionVertex ev22 = vertexIterator.next();
+
+               // start job scheduling
+               eg.scheduleForExecution();
+               manualMainThreadExecutor.triggerAll();
+
+               // fail ev11 to trigger region failover of {ev11}, {ev21}, 
{ev22}
+               ev11.getCurrentExecutionAttempt().fail(new Exception("task 
failure"));
+               manualMainThreadExecutor.triggerAll();
+               assertEquals(JobStatus.RUNNING, eg.getState());
+               assertEquals(ExecutionState.FAILED, ev11.getExecutionState());
+               assertEquals(ExecutionState.DEPLOYING, 
ev12.getExecutionState());
+               assertEquals(ExecutionState.CANCELED, ev21.getExecutionState());
+               assertEquals(ExecutionState.CANCELED, ev22.getExecutionState());
+
+               // trigger global failover cancelling and immediately recovery
+               eg.failGlobal(new Exception("Test global failure"));
+               ev12.getCurrentExecutionAttempt().completeCancelling();
+               manuallyTriggeredRestartStrategy.triggerNextAction();
+               manualMainThreadExecutor.triggerAll();
+
+               // verify the job state and vertex attempt number
+               assertEquals(2, eg.getGlobalModVersion());
+               assertEquals(1, 
ev11.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev12.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev21.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev22.getCurrentExecutionAttempt().getAttemptNumber());
+
+               // complete region failover blocker to trigger region failover
+               failoverStrategy.getBlockerFuture().complete(null);
+               manualMainThreadExecutor.triggerAll();
+
+               // verify that no task is restarted by region failover
+               assertEquals(ExecutionState.DEPLOYING, 
ev11.getExecutionState());
+               assertEquals(ExecutionState.DEPLOYING, 
ev12.getExecutionState());
+               assertEquals(ExecutionState.CREATED, ev21.getExecutionState());
+               assertEquals(ExecutionState.CREATED, ev22.getExecutionState());
+               assertEquals(1, 
ev11.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev12.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev21.getCurrentExecutionAttempt().getAttemptNumber());
+               assertEquals(1, 
ev22.getCurrentExecutionAttempt().getAttemptNumber());
+       }
+
+       @Test
+       public void testSkipFailoverIfExecutionStateIsNotRunning() throws 
Exception {
+               final ExecutionGraph executionGraph = createExecutionGraph();
+
+               final Iterator<ExecutionVertex> vertexIterator = 
executionGraph.getAllExecutionVertices().iterator();
+               final ExecutionVertex firstVertex = vertexIterator.next();
+
+               executionGraph.cancel();
+
+               final FailoverStrategy failoverStrategy = 
executionGraph.getFailoverStrategy();
+               
failoverStrategy.onTaskFailure(firstVertex.getCurrentExecutionAttempt(), new 
Exception("Test Exception"));
+
 
 Review comment:
   The vertex directly transitions to `CANCELED ` because it was no scheduled 
yet and the graph gets cancelled. It does not prove whether the local failover 
is skipped. (Even though a local failover is not skipped, the recovery action 
will be queued in main thread executor and not able to change the vertex state 
out from CANCELED).
   
   If we want to verify "skip local failover if the job status is not Running", 
how about trigger task failure directly without graph.cancel()? In this case, 
the local failover will be skipped since the graph is in CREATED state, and the 
vertex will remain in `CREATED` state.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to