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_r299754065
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java
 ##########
 @@ -0,0 +1,316 @@
+/*
+ * 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.failover;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.checkpoint.CheckpointFailureReason;
+import org.apache.flink.runtime.concurrent.FutureUtils;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.GlobalModVersionMismatch;
+import org.apache.flink.runtime.executiongraph.SchedulingUtils;
+import 
org.apache.flink.runtime.executiongraph.failover.adapter.DefaultFailoverTopology;
+import 
org.apache.flink.runtime.executiongraph.failover.flip1.RestartPipelinedRegionStrategy;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
+import org.apache.flink.runtime.scheduler.ExecutionVertexVersion;
+import org.apache.flink.runtime.scheduler.ExecutionVertexVersioner;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * This failover strategy uses flip1.RestartPipelinedRegionStrategy to make 
task failover decisions.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNG extends FailoverStrategy {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(AdaptedRestartPipelinedRegionStrategyNG.class);
+
+       /** The execution graph on which this FailoverStrategy works. */
+       private final ExecutionGraph executionGraph;
+
+       /** The versioner helps to maintain execution vertex versions. */
+       private final ExecutionVertexVersioner executionVertexVersioner;
+
+       /** The underlying new generation region failover strategy. */
+       private RestartPipelinedRegionStrategy restartPipelinedRegionStrategy;
+
+       public AdaptedRestartPipelinedRegionStrategyNG(final ExecutionGraph 
executionGraph) {
+               this.executionGraph = checkNotNull(executionGraph);
+               this.executionVertexVersioner = new ExecutionVertexVersioner();
+       }
+
+       @Override
+       public void onTaskFailure(final Execution taskExecution, final 
Throwable cause) {
+               if (!executionGraph.getRestartStrategy().canRestart()) {
+                       // delegate the failure to a global fail that will 
check the restart strategy and not restart
+                       LOG.info("Fail to pass the restart strategy validation 
in region failover. Fallback to fail global.");
+                       failGlobal(cause);
+                       return;
+               }
+
+               if 
(!isLocalFailoverValid(executionGraph.getGlobalModVersion())) {
+                       LOG.info("Skip current region failover as a global 
failover is ongoing.");
+                       return;
+               }
+
+               final ExecutionVertexID vertexID = 
getExecutionVertexID(taskExecution.getVertex());
+
+               final Set<ExecutionVertexID> tasksToRestart = 
restartPipelinedRegionStrategy.getTasksNeedingRestart(vertexID, cause);
+               restartTasks(tasksToRestart);
+       }
+
+       @VisibleForTesting
+       protected void restartTasks(final Set<ExecutionVertexID> 
verticesToRestart) {
+               final long globalModVersion = 
executionGraph.getGlobalModVersion();
+               final Set<ExecutionVertexVersion> vertexVersions = new 
HashSet<>(
+                       
executionVertexVersioner.recordVertexModifications(verticesToRestart).values());
+
+               FutureUtils.assertNoException(
+                       cancelTasks(verticesToRestart)
+                               
.thenRunAsync(resetAndRescheduleTasks(globalModVersion, vertexVersions), 
executionGraph.getJobMasterMainThreadExecutor())
 
 Review comment:
   The purpose to use async call here is to break infinite "region 
failover->schedule->region failover->..." call stack chain to avoid 
stackoverflow issue, which may rarely happen though(only when all 
failover/scheduling futures get completed at once, like task cancellation, 
resource allocation, etc).
   Once we have introduced the restart strategy and restart delay, we can make 
here sync again, since the scheduled restarting can break this chain too.

----------------------------------------------------------------
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