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_r326169037
########## 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); + } + } + + // ------------------------------------------------------------------------ + // SchedulerOperations + // ------------------------------------------------------------------------ + + @Override + 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) { + cancelSlotAssignments(verticesToDeploy); + resetForNewExecution(verticesToDeploy); + transitionToScheduled(verticesToDeploy); + } + + private void cancelSlotAssignments(final Collection<ExecutionVertexID> vertices) { + vertices.forEach(executionVertexId -> executionSlotAllocator.cancel(executionVertexId)); + } + + private Collection<SlotExecutionVertexAssignment> allocateSlots(final Collection<ExecutionVertexDeploymentOption> executionVertexDeploymentOptions) { + return executionSlotAllocator.allocateSlotsFor(executionVertexDeploymentOptions + .stream() + .map(ExecutionVertexDeploymentOption::getExecutionVertexId) + .map(this::getExecutionVertex) + .map(ExecutionVertexSchedulingRequirementsMapper::from) + .collect(Collectors.toList())); + } + + private static Collection<DeploymentHandle> createDeploymentHandles( + final Map<ExecutionVertexID, ExecutionVertexVersion> requiredVersionByVertex, + final Map<ExecutionVertexID, ExecutionVertexDeploymentOption> deploymentOptionsByVertex, + final Collection<SlotExecutionVertexAssignment> slotExecutionVertexAssignments) { + + return slotExecutionVertexAssignments + .stream() + .map(slotExecutionVertexAssignment -> { + final ExecutionVertexID executionVertexId = slotExecutionVertexAssignment.getExecutionVertexId(); + return new DeploymentHandle( + requiredVersionByVertex.get(executionVertexId), + deploymentOptionsByVertex.get(executionVertexId), + slotExecutionVertexAssignment); + }) + .collect(Collectors.toList()); + } + + private boolean isDeployIndividually() { + return schedulingStrategy instanceof LazyFromSourcesSchedulingStrategy; + } + + private void deployIndividually(final Collection<DeploymentHandle> deploymentHandles) { + for (final DeploymentHandle deploymentHandle : deploymentHandles) { + FutureUtils.assertNoException( + deploymentHandle + .getSlotExecutionVertexAssignment() + .getLogicalSlotFuture() + .handle(assignResourceOrHandleError(deploymentHandle)) + .handle(deployOrHandleError(deploymentHandle))); + } + } + + private void waitForAllSlotsAndDeploy(final Collection<DeploymentHandle> deploymentHandles) { + FutureUtils.assertNoException( + assignAllResources(deploymentHandles).handle(deployAll(deploymentHandles))); + } + + private CompletableFuture<Void> assignAllResources(final Collection<DeploymentHandle> deploymentHandles) { + final List<CompletableFuture<Void>> slotAssignedFutures = new ArrayList<>(); + for (DeploymentHandle deploymentHandle : deploymentHandles) { + final CompletableFuture<Void> slotAssigned = deploymentHandle + .getSlotExecutionVertexAssignment() + .getLogicalSlotFuture() + .handle(assignResourceOrHandleError(deploymentHandle)); + slotAssignedFutures.add(slotAssigned); + } + return FutureUtils.waitForAll(slotAssignedFutures); + } + + private BiFunction<Void, Throwable, Void> deployAll(final Collection<DeploymentHandle> deploymentHandles) { + return (ignored, throwable) -> { + propagateIfNonNull(throwable); + for (final DeploymentHandle deploymentHandle : deploymentHandles) { + final SlotExecutionVertexAssignment slotExecutionVertexAssignment = deploymentHandle.getSlotExecutionVertexAssignment(); + final CompletableFuture<LogicalSlot> slotAssigned = slotExecutionVertexAssignment.getLogicalSlotFuture(); + checkState(slotAssigned.isDone()); + + FutureUtils.assertNoException( + slotAssigned.handle(deployOrHandleError(deploymentHandle))); + } + return null; + }; + } + + private static void propagateIfNonNull(final Throwable throwable) { + if (throwable != null) { + throw new CompletionException(throwable); + } + } + + private BiFunction<LogicalSlot, Throwable, Void> assignResourceOrHandleError(final DeploymentHandle deploymentHandle) { + final ExecutionVertexVersion requiredVertexVersion = deploymentHandle.getRequiredVertexVersion(); + final ExecutionVertexID executionVertexId = deploymentHandle.getExecutionVertexId(); + + return (logicalSlot, throwable) -> { + if (executionVertexVersioner.isModified(requiredVertexVersion)) { + log.debug("Refusing to assign slot to execution vertex {} because this deployment was " + + "superseded by another deployment", executionVertexId); + stopDeployment(deploymentHandle); + return null; + } + + if (throwable == null) { + final ExecutionVertex executionVertex = getExecutionVertex(executionVertexId); + final boolean sendScheduleOrUpdateConsumerMessage = deploymentHandle.getDeploymentOption().sendScheduleOrUpdateConsumerMessage(); + executionVertex + .getCurrentExecutionAttempt() + .registerProducedPartitions(logicalSlot.getTaskManagerLocation(), sendScheduleOrUpdateConsumerMessage); + executionVertex.tryAssignResource(logicalSlot); + } else { + handleTaskFailure(executionVertexId, throwable); + } + return null; + }; + } + + private BiFunction<Object, Throwable, Void> deployOrHandleError(final DeploymentHandle deploymentHandle) { + final ExecutionVertexVersion requiredVertexVersion = deploymentHandle.getRequiredVertexVersion(); + final ExecutionVertexID executionVertexId = requiredVertexVersion.getExecutionVertexId(); + + return (ignored, throwable) -> { + if (executionVertexVersioner.isModified(requiredVertexVersion)) { + log.debug("Refusing to deploy execution vertex {} because this deployment was " + + "superseded by another deployment", executionVertexId); + stopDeployment(deploymentHandle); + return null; + } + + if (throwable == null) { + deployTaskSafe(executionVertexId); + } else { + handleTaskFailure(executionVertexId, throwable); + } + return null; + }; + } + + private void stopDeployment(final DeploymentHandle deploymentHandle) { + cancelExecutionVertex(deploymentHandle.getExecutionVertexId()); + deploymentHandle.getLogicalSlot().releaseSlot(null); + } + + private void deployTaskSafe(final ExecutionVertexID executionVertexId) { + try { + final ExecutionVertex executionVertex = getExecutionVertex(executionVertexId); + executionVertexOperations.deploy(executionVertex); + } catch (Throwable e) { + handleTaskFailure(executionVertexId, e); Review comment: If we move the error handling to the back of the chain as proposed here: https://github.com/tillrohrmann/flink/commit/546040cb7106ca371a70a6a29f28a1b08a4da2fc, then we don't need to catch the exception here. ---------------------------------------------------------------- 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