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

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/AdaptedRestartPipelinedRegionStrategyNG.java
 ##########
 @@ -0,0 +1,295 @@
+/*
+ * 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.failover.adapter.DefaultFailoverTopology;
+import org.apache.flink.runtime.executiongraph.failover.flip1.FailoverRegion;
+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.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.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This failover strategy uses flip1.RestartPipelinedRegionStrategy to make 
task failover decisions.
+ */
+public class AdaptedRestartPipelinedRegionStrategyNG extends FailoverStrategy {
+
+       /** The log object used for debugging. */
+       private static final Logger LOG = 
LoggerFactory.getLogger(AdaptedRestartPipelinedRegionStrategyNG.class);
+
+       /** The execution graph on which this FailoverStrategy works. */
+       private final ExecutionGraph executionGraph;
+
+       /** The underlying new generation region failover strategy. */
+       private RestartPipelinedRegionStrategy restartPipelinedRegionStrategy;
+
+       /** The versioner helps to maintain execution vertex versions. */
+       private final ExecutionVertexVersioner executionVertexVersioner;
+
+       public AdaptedRestartPipelinedRegionStrategyNG(final ExecutionGraph 
executionGraph) {
+               this.executionGraph = checkNotNull(executionGraph);
+               this.executionVertexVersioner = new ExecutionVertexVersioner();
+       }
+
+       @Override
+       public void onTaskFailure(final Execution taskExecution, final 
Throwable cause) {
+               // skip the failover if global restart strategy suppresses 
restarts
+               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.");
+                       executionGraph.failGlobal(cause);
+                       return;
+               }
+
+               // skip local failover if is in global failover
+               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);
+
+               // restart tasks at once
+               restartTasks(tasksToRestart);
+       }
+
+       private void restartTasks(final Set<ExecutionVertexID> 
verticesToRestart) {
+               // record current versions
+               final long globalModVersion = 
executionGraph.getGlobalModVersion();
+               final Set<ExecutionVertexVersion> vertexVersions = new 
HashSet<>(
+                       
executionVertexVersioner.recordVertexModifications(verticesToRestart).values());
+
+               // cancel tasks involved in this task failure
+               cancelTasks(verticesToRestart)
+                       .thenAccept(
+                               (Object ignored) -> {
+                                       // skip local failover if is in global 
failover
+                                       if 
(!isLocalFailoverValid(globalModVersion)) {
+                                               LOG.info("Skip current region 
failover as a global failover is ongoing.");
+                                               return;
+                                       }
+
+                                       // found out vertices which are still 
valid to restart.
+                                       // some vertices involved in this 
failover may be modified if another region
+                                       // failover happens during the 
cancellation stage of this failover.
+                                       // Will ignore the modified vertices as 
the other failover will deal with them.
+                                       final Set<ExecutionVertex> 
unmodifiedVertices = executionVertexVersioner
+                                               
.getUnmodifiedExecutionVertices(vertexVersions)
+                                               .stream()
+                                               .map(this::getExecutionVertex)
+                                               .collect(Collectors.toSet());
+
+                                       try {
+                                               LOG.info("Finally restart {} 
tasks to recover from task failure.", unmodifiedVertices.size());
+
+                                               // reset tasks to CREATED state 
and reload state
+                                               resetTasks(unmodifiedVertices, 
globalModVersion);
+
+                                               // re-schedule tasks
+                                               
rescheduleTasks(unmodifiedVertices, globalModVersion);
+                                       } catch (GlobalModVersionMismatch e) {
+                                               // happens when a global 
recovery happens concurrently to the regional recovery
+                                               // just stop this local failover
+                                       } catch (Exception e) {
+                                               throw new 
CompletionException(e);
+                                       }
+                               })
+                       .whenComplete(
+                               (Object ignored, Throwable t) -> {
+                                       // fail globally if any error happens
+                                       if (t != null) {
+                                               LOG.info("Unexpected error 
happens in region failover. Fail globally.", t);
+                                               failGlobal(t);
+                                       }
+                               });
+       }
+
+       @VisibleForTesting
+       protected CompletableFuture<?> cancelTasks(final Set<ExecutionVertexID> 
vertices) {
+               final List<CompletableFuture<?>> cancelFutures = 
vertices.stream()
+                       .map(this::cancelExecutionVertex)
+                       .collect(Collectors.toList());
+
+               return FutureUtils.combineAll(cancelFutures);
+       }
+
+       private void resetTasks(final Set<ExecutionVertex> vertices, final long 
globalModVersion) throws Exception {
+               final Set<CoLocationGroup> colGroups = new HashSet<>();
+               final long restartTimestamp = System.currentTimeMillis();
+
+               for (ExecutionVertex ev : vertices) {
+                       CoLocationGroup cgroup = 
ev.getJobVertex().getCoLocationGroup();
+                       if (cgroup != null && !colGroups.contains(cgroup)){
+                               cgroup.resetConstraints();
+                               colGroups.add(cgroup);
+                       }
+
+                       ev.resetForNewExecution(restartTimestamp, 
globalModVersion);
+               }
+
+               // if there is checkpointed state, reload it into the executions
+               if (executionGraph.getCheckpointCoordinator() != null) {
+                       // abort pending checkpoints to
+                       // i) enable new checkpoint triggering without waiting 
for last checkpoint expired.
+                       // ii) ensure the EXACTLY_ONCE semantics if needed.
+                       
executionGraph.getCheckpointCoordinator().abortPendingCheckpoints(
+                               new 
CheckpointException(CheckpointFailureReason.JOB_FAILOVER_REGION));
+
+                       final Map<JobVertexID, ExecutionJobVertex> 
involvedExecutionJobVertices =
+                               getInvolvedExecutionJobVertices(vertices);
+                       
executionGraph.getCheckpointCoordinator().restoreLatestCheckpointedState(
+                               involvedExecutionJobVertices, false, true);
+               }
+       }
+
+       private void rescheduleTasks(final Set<ExecutionVertex> vertices, final 
long globalModVersion) throws Exception {
+               final CompletableFuture<Void> newSchedulingFuture;
+               switch (executionGraph.getScheduleMode()) {
+
+                       case LAZY_FROM_SOURCES:
+                               newSchedulingFuture = 
AdaptedSchedulingUtils.scheduleLazy(vertices, executionGraph);
+                               break;
+
+                       case EAGER:
+                               newSchedulingFuture = 
AdaptedSchedulingUtils.scheduleEager(vertices, executionGraph);
+                               break;
+
+                       default:
+                               throw new JobException("Schedule mode is 
invalid.");
+               }
+
+               // if no global failover is triggered in the scheduling process,
+               // register a failure handling callback to the scheduling
+               if (isLocalFailoverValid(globalModVersion)) {
 
 Review comment:
   This is to ensure that no global failure happens during the sync part of the 
schedule logic. Usually this should not happen. It's a insurance without much 
cost.

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