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

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java
 ##########
 @@ -75,10 +137,281 @@ public DefaultScheduler(
                        slotRequestTimeout,
                        shuffleMaster,
                        partitionTracker);
+
+               this.restartBackoffTimeStrategy = restartBackoffTimeStrategy;
+               this.slotRequestTimeout = slotRequestTimeout;
+               this.slotProvider = slotProvider;
+               this.delayExecutor = delayExecutor;
+               this.userCodeLoader = userCodeLoader;
+               this.schedulingStrategyFactory = 
checkNotNull(schedulingStrategyFactory);
+               this.failoverStrategyFactory = 
checkNotNull(failoverStrategyFactory);
+               this.executionVertexOperations = 
checkNotNull(executionVertexOperations);
+               this.executionVertexVersioner = executionVertexVersioner;
+               this.conditionalFutureHandlerFactory = new 
ConditionalFutureHandlerFactory(executionVertexVersioner);
+       }
+
+       // 
------------------------------------------------------------------------
+       // SchedulerNG
+       // 
------------------------------------------------------------------------
+
+       @Override
+       public void startSchedulingInternal() {
+               initializeScheduling();
+               schedulingStrategy.startScheduling();
+       }
+
+       private void initializeScheduling() {
+               executionFailureHandler = new 
ExecutionFailureHandler(failoverStrategyFactory.create(getFailoverTopology()), 
restartBackoffTimeStrategy);
+               schedulingStrategy = 
schedulingStrategyFactory.createInstance(this, getSchedulingTopology(), 
getJobGraph());
+               executionSlotAllocator = new 
DefaultExecutionSlotAllocator(slotProvider, getInputsLocationsRetriever(), 
slotRequestTimeout);
+               setTaskFailureListener(new 
UpdateTaskExecutionStateInDefaultSchedulerListener(this, 
getJobGraph().getJobID()));
+               prepareExecutionGraphForScheduling();
+       }
+
+       @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);
+                       }
+                       return null;
+               };
+       }
+
+       private CompletableFuture<?> cancelTasksAsync(final 
Set<ExecutionVertexID> verticesToRestart) {
+               final List<CompletableFuture<?>> cancelFutures = 
verticesToRestart.stream()
+                       .map(this::cancelExecutionVertex)
+                       .collect(Collectors.toList());
+
+               return FutureUtils.combineAll(cancelFutures);
        }
 
+       private CompletableFuture<?> cancelExecutionVertex(final 
ExecutionVertexID executionVertexId) {
+               return 
executionVertexOperations.cancel(getExecutionVertex(executionVertexId));
+       }
+
+       @Override
+       public void scheduleOrUpdateConsumers(final ResultPartitionID 
partitionId) {
+               final Optional<ExecutionVertexID> producerVertexId = 
getExecutionVertexId(partitionId.getProducerId());
+               if (producerVertexId.isPresent()) {
+                       updateConsumers(partitionId);
+                       
schedulingStrategy.onPartitionConsumable(producerVertexId.get(), partitionId);
+               }
+       }
+
+       // 
------------------------------------------------------------------------
+       // SchedulerOperations
+       // 
------------------------------------------------------------------------
+
        @Override
-       protected void startSchedulingInternal() {
-               throw new UnsupportedOperationException();
+       public void allocateSlotsAndDeploy(final 
Collection<ExecutionVertexDeploymentOption> executionVertexDeploymentOptions) {
+               final Map<ExecutionVertexID, ExecutionVertexDeploymentOption> 
deploymentOptionsByVertex = 
groupDeploymentOptionsByVertexId(executionVertexDeploymentOptions);
+               final Set<ExecutionVertexID> verticesToDeploy = 
deploymentOptionsByVertex.keySet();
+               final Map<ExecutionVertexID, ExecutionVertexVersion> 
requiredVersionByVertex = 
executionVertexVersioner.recordVertexModifications(verticesToDeploy);
+
+               prepareToDeployVertices(verticesToDeploy);
+
+               final Collection<SlotExecutionVertexAssignment> 
slotExecutionVertexAssignments = 
allocateSlots(executionVertexDeploymentOptions);
+
+               final Collection<DeploymentHandle> deploymentHandles = 
createDeploymentHandles(
+                       requiredVersionByVertex,
+                       deploymentOptionsByVertex,
+                       slotExecutionVertexAssignments);
+
+               if (isDeployIndividually()) {
+                       deployIndividually(deploymentHandles);
+               } else {
+                       waitForAllSlotsAndDeploy(deploymentHandles);
+               }
+       }
+
+       private static Map<ExecutionVertexID, ExecutionVertexDeploymentOption> 
groupDeploymentOptionsByVertexId(
+                       final Collection<ExecutionVertexDeploymentOption> 
executionVertexDeploymentOptions) {
+               return 
executionVertexDeploymentOptions.stream().collect(Collectors.toMap(
+                               
ExecutionVertexDeploymentOption::getExecutionVertexId,
+                               Function.identity()));
+       }
+
+       private void prepareToDeployVertices(final Set<ExecutionVertexID> 
verticesToDeploy) {
 
 Review comment:
   Me neither. Marked this conversation as resolved.

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