zhuzhurk commented on a change in pull request #9663: [WIP][FLINK-12433][runtime] Implement DefaultScheduler stub URL: https://github.com/apache/flink/pull/9663#discussion_r326540913
########## 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); + } + 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 startScheduling() { - throw new UnsupportedOperationException(); + public void scheduleOrUpdateConsumers(final ResultPartitionID partitionId) { + final Optional<ExecutionVertexID> producerVertexId = getExecutionVertexId(partitionId.getProducerId()); + if (producerVertexId.isPresent()) { + updateConsumers(partitionId); + schedulingStrategy.onPartitionConsumable(producerVertexId.get(), partitionId); Review comment: A partition is consumable in 2 cases: 1. a pipelined partition has data produced (including notified finished from TM side) 2. a blocking partition is finished and all other partitions in the its parent IntermediateResult is finished Here we only cover the first case, though it is not problematic at the moment since the `LazyFromSourcesSchedulingStrategy` is doing scheduling based on the FINISHED task state rather than consumable partition state. Future scheduling strategies, however, may find the needed state updates of partitions are missing. Besides, I think the philosophy of `lazy_from_sources` scheduling is to schedule a task when its inputs are ready, naming the needed result partitions are consumable. It actually does not directly care whether a task is finished, though a finished task will lead its partitions to be consumable. The right place to catch the partition state state changes is in `Execution#scheduleOrUpdateConsumers` currently. I think we should reconsider how we pass the partition state updates and how to better implement `LazyFromSourcesSchedulingStrategy`. And sorry for not recognizing this potential issue in 1.9. It can be follow up work after we have made `DefaultScheduler` in a working 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