GJL commented on a change in pull request #9663: [WIP][FLINK-12433][runtime] 
Implement DefaultScheduler stub
URL: https://github.com/apache/flink/pull/9663#discussion_r327069936
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
 ##########
 @@ -75,10 +128,293 @@ public DefaultScheduler(
                        slotRequestTimeout,
                        shuffleMaster,
                        partitionTracker);
+
+               this.log = log;
+
+               this.delayExecutor = checkNotNull(delayExecutor);
+               this.userCodeLoader = checkNotNull(userCodeLoader);
+               this.executionVertexOperations = 
checkNotNull(executionVertexOperations);
+               this.executionVertexVersioner = 
checkNotNull(executionVertexVersioner);
+
+               this.executionFailureHandler = new 
ExecutionFailureHandler(failoverStrategyFactory.create(getFailoverTopology()), 
restartBackoffTimeStrategy);
+               this.schedulingStrategy = 
schedulingStrategyFactory.createInstance(this, getSchedulingTopology(), 
getJobGraph());
+               this.executionSlotAllocator = new 
DefaultExecutionSlotAllocator(slotProvider, getInputsLocationsRetriever(), 
slotRequestTimeout);
+               setTaskFailureListener(new 
UpdateSchedulerNgOnInternalTaskFailuresListener(this, 
getJobGraph().getJobID()));
+       }
+
+       // 
------------------------------------------------------------------------
+       // SchedulerNG
+       // 
------------------------------------------------------------------------
+
+       @Override
+       protected void startSchedulingInternal() {
+               prepareExecutionGraphForScheduling();
+               schedulingStrategy.startScheduling();
+       }
+
+       @Override
+       public boolean updateTaskExecutionState(final TaskExecutionState 
taskExecutionState) {
+               final Optional<ExecutionVertexID> executionVertexIdOptional = 
getExecutionVertexId(taskExecutionState.getID());
+               if (executionVertexIdOptional.isPresent()) {
+                       final ExecutionVertexID executionVertexId = 
executionVertexIdOptional.get();
+                       updateState(taskExecutionState);
+                       
schedulingStrategy.onExecutionStateChange(executionVertexId, 
taskExecutionState.getExecutionState());
+                       maybeHandleTaskFailure(taskExecutionState, 
executionVertexId);
+                       return true;
+               }
+
+               return false;
+       }
+
+       private void maybeHandleTaskFailure(final TaskExecutionState 
taskExecutionState, final ExecutionVertexID executionVertexId) {
+               if (taskExecutionState.getExecutionState() == 
ExecutionState.FAILED) {
+                       final Throwable error = 
taskExecutionState.getError(userCodeLoader);
+                       handleTaskFailure(executionVertexId, error);
+               }
+       }
+
+       private void handleTaskFailure(final ExecutionVertexID 
executionVertexId, final Throwable error) {
+               final FailureHandlingResult failureHandlingResult = 
executionFailureHandler.getFailureHandlingResult(executionVertexId, error);
+               maybeRestartTasks(failureHandlingResult);
+       }
+
+       private void maybeRestartTasks(final FailureHandlingResult 
failureHandlingResult) {
+               if (failureHandlingResult.canRestart()) {
+                       restartTasksWithDelay(failureHandlingResult);
+               } else {
+                       failJob(failureHandlingResult.getError());
+               }
+       }
+
+       private void restartTasksWithDelay(final FailureHandlingResult 
failureHandlingResult) {
+               final Set<ExecutionVertexID> verticesToRestart = 
failureHandlingResult.getVerticesToRestart();
+
+               final Set<ExecutionVertexVersion> executionVertexVersions =
+                       new 
HashSet<>(executionVertexVersioner.recordVertexModifications(verticesToRestart).values());
+
+               final CompletableFuture<?> cancelFuture = 
cancelTasksAsync(verticesToRestart);
+
+               delayExecutor.schedule(
+                       () -> FutureUtils.assertNoException(
+                               
cancelFuture.handleAsync(restartTasksOrHandleError(executionVertexVersions), 
getMainThreadExecutor())),
+                       failureHandlingResult.getRestartDelayMS(),
+                       TimeUnit.MILLISECONDS);
+       }
+
+       private BiFunction<Object, Throwable, Void> 
restartTasksOrHandleError(final Set<ExecutionVertexVersion> 
executionVertexVersions) {
+               return (Object ignored, Throwable throwable) -> {
+
+                       if (throwable == null) {
+                               final Set<ExecutionVertexID> verticesToRestart 
= 
executionVertexVersioner.getUnmodifiedExecutionVertices(executionVertexVersions);
+                               
schedulingStrategy.restartTasks(verticesToRestart);
+                       } else {
+                               failJob(throwable);
 
 Review comment:
   Yes, the throwable here is from the failed task cancellation. For reference, 
here is what happens in legacy code when the task cancellation fails during 
failover: 
https://github.com/apache/flink/blob/a80bbf187b8a7318db3a8b117e7281eaa174c647/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/failover/FailoverRegion.java#L147-L148
   
   From what I can see is that the execution transitions to failed and tasks 
are restarted if the restart strategy allows this.

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to