zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905682558


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployer.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.scheduler;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Default implementation of {@link ExecutionDeployer}. */
+public class DefaultExecutionDeployer implements ExecutionDeployer {
+
+    private final Logger log;
+
+    private final ExecutionSlotAllocator executionSlotAllocator;
+
+    private final ExecutionOperations executionOperations;
+
+    private final ExecutionVertexVersioner executionVertexVersioner;
+
+    private final Time partitionRegistrationTimeout;
+
+    private final Function<ExecutionAttemptID, Execution> executionRetriever;
+
+    private final BiConsumer<ExecutionVertexID, AllocationID> 
allocationReservationFunc;
+
+    private final ComponentMainThreadExecutor mainThreadExecutor;
+
+    private DefaultExecutionDeployer(
+            final Logger log,
+            final ExecutionSlotAllocator executionSlotAllocator,
+            final ExecutionOperations executionOperations,
+            final ExecutionVertexVersioner executionVertexVersioner,
+            final Time partitionRegistrationTimeout,
+            final Function<ExecutionAttemptID, Execution> executionRetriever,
+            final BiConsumer<ExecutionVertexID, AllocationID> 
allocationReservationFunc,
+            final ComponentMainThreadExecutor mainThreadExecutor) {
+
+        this.log = checkNotNull(log);
+        this.executionSlotAllocator = checkNotNull(executionSlotAllocator);
+        this.executionOperations = checkNotNull(executionOperations);
+        this.executionVertexVersioner = checkNotNull(executionVertexVersioner);
+        this.partitionRegistrationTimeout = 
checkNotNull(partitionRegistrationTimeout);
+        this.executionRetriever = checkNotNull(executionRetriever);
+        this.allocationReservationFunc = 
checkNotNull(allocationReservationFunc);
+        this.mainThreadExecutor = checkNotNull(mainThreadExecutor);
+    }
+
+    @Override
+    public void allocateSlotsAndDeploy(
+            final List<Execution> executionsToDeploy,
+            final Map<ExecutionVertexID, ExecutionVertexVersion> 
requiredVersionByVertex) {
+        validateExecutionStates(executionsToDeploy);
+
+        transitionToScheduled(executionsToDeploy);
+
+        final List<ExecutionSlotAssignment> slotExecutionVertexAssignments =
+                allocateSlotsFor(executionsToDeploy);
+
+        final List<ExecutionDeploymentHandle> deploymentHandles =
+                createDeploymentHandles(requiredVersionByVertex, 
slotExecutionVertexAssignments);
+
+        waitForAllSlotsAndDeploy(deploymentHandles);
+    }
+
+    private void validateExecutionStates(final Collection<Execution> 
executionsToDeploy) {
+        executionsToDeploy.stream()
+                .forEach(
+                        e ->
+                                checkState(
+                                        e.getState() == ExecutionState.CREATED,
+                                        "Expected execution %s to be in 
CREATED state, was: %s",
+                                        e.getAttemptId(),
+                                        e.getState()));
+    }
+
+    private void transitionToScheduled(final List<Execution> 
executionsToDeploy) {
+        executionsToDeploy.forEach(e -> 
e.transitionState(ExecutionState.SCHEDULED));
+    }
+
+    private List<ExecutionSlotAssignment> allocateSlotsFor(
+            final List<Execution> executionsToDeploy) {
+        final List<ExecutionAttemptID> executionAttemptIds =
+                executionsToDeploy.stream()
+                        .map(Execution::getAttemptId)
+                        .collect(Collectors.toList());
+        return executionSlotAllocator.allocateSlotsFor(executionAttemptIds);
+    }
+
+    private List<ExecutionDeploymentHandle> createDeploymentHandles(
+            final Map<ExecutionVertexID, ExecutionVertexVersion> 
requiredVersionByVertex,
+            final List<ExecutionSlotAssignment> executionSlotAssignments) {
+
+        return executionSlotAssignments.stream()
+                .map(
+                        executionSlotAssignment -> {
+                            final Execution execution =
+                                    
getExecution(executionSlotAssignment.getExecutionAttemptId());
+                            final ExecutionVertexID executionVertexId =
+                                    execution.getVertex().getID();
+                            return new ExecutionDeploymentHandle(
+                                    executionSlotAssignment,
+                                    
requiredVersionByVertex.get(executionVertexId));
+                        })
+                .collect(Collectors.toList());
+    }
+
+    private void waitForAllSlotsAndDeploy(final 
List<ExecutionDeploymentHandle> deploymentHandles) {
+        FutureUtils.assertNoException(
+                
assignAllResourcesAndRegisterProducedPartitions(deploymentHandles)
+                        .handle(deployAll(deploymentHandles)));
+    }
+
+    private CompletableFuture<Void> 
assignAllResourcesAndRegisterProducedPartitions(
+            final List<ExecutionDeploymentHandle> deploymentHandles) {
+        final List<CompletableFuture<Void>> resultFutures = new ArrayList<>();
+        for (ExecutionDeploymentHandle deploymentHandle : deploymentHandles) {
+            final CompletableFuture<Void> resultFuture =
+                    deploymentHandle
+                            .getLogicalSlotFuture()
+                            .handle(assignResource(deploymentHandle))
+                            
.thenCompose(registerProducedPartitions(deploymentHandle))
+                            .handle(
+                                    (ignore, throwable) -> {
+                                        if (throwable != null) {
+                                            handleTaskDeploymentFailure(
+                                                    
deploymentHandle.getExecutionAttemptId(),
+                                                    throwable);
+                                        }
+                                        return null;
+                                    });
+
+            resultFutures.add(resultFuture);
+        }
+        return FutureUtils.waitForAll(resultFutures);
+    }
+
+    private BiFunction<Void, Throwable, Void> deployAll(
+            final List<ExecutionDeploymentHandle> deploymentHandles) {
+        return (ignored, throwable) -> {
+            propagateIfNonNull(throwable);
+            for (final ExecutionDeploymentHandle deploymentHandle : 
deploymentHandles) {
+                final CompletableFuture<LogicalSlot> slotAssigned =
+                        deploymentHandle.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, LogicalSlot> assignResource(
+            final ExecutionDeploymentHandle deploymentHandle) {
+
+        return (logicalSlot, throwable) -> {
+            final ExecutionVertexVersion requiredVertexVersion =
+                    deploymentHandle.getRequiredVertexVersion();
+            final Execution execution = 
getExecution(deploymentHandle.getExecutionAttemptId());
+
+            if (execution == null
+                    || execution.getState() != ExecutionState.SCHEDULED
+                    || 
executionVertexVersioner.isModified(requiredVertexVersion)) {
+                if (throwable == null) {
+                    log.debug(
+                            "Refusing to assign slot to execution {} because 
this deployment was "
+                                    + "superseded by another deployment",
+                            deploymentHandle.getExecutionAttemptId());
+                    releaseSlotIfPresent(logicalSlot);
+                }
+                return null;
+            }
+
+            // throw exception only if the execution version is not outdated.
+            // this ensures that canceling a pending slot request does not fail
+            // a task which is about to cancel.
+            if (throwable != null) {
+                throw new 
CompletionException(maybeWrapWithNoResourceAvailableException(throwable));
+            }
+
+            if (!execution.tryAssignResource(logicalSlot)) {
+                throw new IllegalStateException(
+                        "Could not assign resource "
+                                + logicalSlot
+                                + " to execution "
+                                + execution
+                                + '.');
+            }
+
+            allocationReservationFunc.accept(
+                    execution.getAttemptId().getExecutionVertexId(), 
logicalSlot.getAllocationId());
+
+            return logicalSlot;
+        };
+    }
+
+    private static void releaseSlotIfPresent(@Nullable final LogicalSlot 
logicalSlot) {
+        if (logicalSlot != null) {
+            logicalSlot.releaseSlot(null);
+        }
+    }
+
+    private static Throwable maybeWrapWithNoResourceAvailableException(final 
Throwable failure) {
+        final Throwable strippedThrowable = 
ExceptionUtils.stripCompletionException(failure);
+        if (strippedThrowable instanceof TimeoutException) {
+            return new NoResourceAvailableException(
+                    "Could not allocate the required slot within slot request 
timeout. "
+                            + "Please make sure that the cluster has enough 
resources.",
+                    failure);
+        } else {
+            return failure;
+        }
+    }
+
+    private Function<LogicalSlot, CompletableFuture<Void>> 
registerProducedPartitions(
+            final ExecutionDeploymentHandle deploymentHandle) {
+
+        return logicalSlot -> {
+            // a null logicalSlot means the slot assignment is skipped, in 
which case
+            // the produced partition registration process can be skipped as 
well
+            if (logicalSlot != null) {
+                final Execution execution = 
getExecution(deploymentHandle.getExecutionAttemptId());
+                final CompletableFuture<Void> partitionRegistrationFuture =
+                        
execution.registerProducedPartitions(logicalSlot.getTaskManagerLocation());
+
+                return FutureUtils.orTimeout(
+                        partitionRegistrationFuture,
+                        partitionRegistrationTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS,
+                        mainThreadExecutor);
+            } else {
+                return FutureUtils.completedVoidFuture();
+            }
+        };
+    }
+
+    private BiFunction<Object, Throwable, Void> deployOrHandleError(
+            final ExecutionDeploymentHandle deploymentHandle) {
+
+        return (ignored, throwable) -> {
+            final ExecutionVertexVersion requiredVertexVersion =
+                    deploymentHandle.getRequiredVertexVersion();
+            final Execution execution = 
getExecution(deploymentHandle.getExecutionAttemptId());
+
+            if (execution == null
+                    || execution.getState() != ExecutionState.SCHEDULED
+                    || 
executionVertexVersioner.isModified(requiredVertexVersion)) {
+                if (throwable == null) {
+                    log.debug(
+                            "Refusing to assign slot to execution {} because 
this deployment was "
+                                    + "superseded by another deployment",
+                            deploymentHandle.getExecutionAttemptId());
+                }
+                return null;
+            }
+
+            if (throwable == null) {
+                deployTaskSafe(execution);
+            } else {
+                handleTaskDeploymentFailure(execution.getAttemptId(), 
throwable);
+            }
+            return null;
+        };
+    }
+
+    private void deployTaskSafe(final Execution execution) {
+        try {
+            executionOperations.deploy(execution);
+        } catch (Throwable e) {
+            handleTaskDeploymentFailure(execution.getAttemptId(), e);
+        }
+    }
+
+    private void handleTaskDeploymentFailure(
+            final ExecutionAttemptID executionAttemptId, final Throwable 
error) {
+
+        final Execution execution = getExecution(executionAttemptId);
+        executionOperations.markFailed(execution, error);
+    }
+
+    private Execution getExecution(ExecutionAttemptID executionAttemptId) {

Review Comment:
   Changed the return type to `Optional<Execution>` (Flink code style suggests 
the return type to be `Optional` instead of `Nullable`). A 
`getExecutionorThrow(...)` is added which alway expects the execution to be 
non-null.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployer.java:
##########
@@ -0,0 +1,386 @@
+/*
+ * 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.scheduler;
+
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.clusterframework.types.AllocationID;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmaster.LogicalSlot;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import org.slf4j.Logger;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Default implementation of {@link ExecutionDeployer}. */
+public class DefaultExecutionDeployer implements ExecutionDeployer {
+
+    private final Logger log;
+
+    private final ExecutionSlotAllocator executionSlotAllocator;
+
+    private final ExecutionOperations executionOperations;
+
+    private final ExecutionVertexVersioner executionVertexVersioner;
+
+    private final Time partitionRegistrationTimeout;
+
+    private final Function<ExecutionAttemptID, Execution> executionRetriever;
+
+    private final BiConsumer<ExecutionVertexID, AllocationID> 
allocationReservationFunc;
+
+    private final ComponentMainThreadExecutor mainThreadExecutor;
+
+    private DefaultExecutionDeployer(
+            final Logger log,
+            final ExecutionSlotAllocator executionSlotAllocator,
+            final ExecutionOperations executionOperations,
+            final ExecutionVertexVersioner executionVertexVersioner,
+            final Time partitionRegistrationTimeout,
+            final Function<ExecutionAttemptID, Execution> executionRetriever,
+            final BiConsumer<ExecutionVertexID, AllocationID> 
allocationReservationFunc,
+            final ComponentMainThreadExecutor mainThreadExecutor) {
+
+        this.log = checkNotNull(log);
+        this.executionSlotAllocator = checkNotNull(executionSlotAllocator);
+        this.executionOperations = checkNotNull(executionOperations);
+        this.executionVertexVersioner = checkNotNull(executionVertexVersioner);
+        this.partitionRegistrationTimeout = 
checkNotNull(partitionRegistrationTimeout);
+        this.executionRetriever = checkNotNull(executionRetriever);
+        this.allocationReservationFunc = 
checkNotNull(allocationReservationFunc);
+        this.mainThreadExecutor = checkNotNull(mainThreadExecutor);
+    }
+
+    @Override
+    public void allocateSlotsAndDeploy(
+            final List<Execution> executionsToDeploy,
+            final Map<ExecutionVertexID, ExecutionVertexVersion> 
requiredVersionByVertex) {
+        validateExecutionStates(executionsToDeploy);
+
+        transitionToScheduled(executionsToDeploy);
+
+        final List<ExecutionSlotAssignment> slotExecutionVertexAssignments =
+                allocateSlotsFor(executionsToDeploy);
+
+        final List<ExecutionDeploymentHandle> deploymentHandles =
+                createDeploymentHandles(requiredVersionByVertex, 
slotExecutionVertexAssignments);
+
+        waitForAllSlotsAndDeploy(deploymentHandles);
+    }
+
+    private void validateExecutionStates(final Collection<Execution> 
executionsToDeploy) {
+        executionsToDeploy.stream()
+                .forEach(
+                        e ->
+                                checkState(
+                                        e.getState() == ExecutionState.CREATED,
+                                        "Expected execution %s to be in 
CREATED state, was: %s",
+                                        e.getAttemptId(),
+                                        e.getState()));
+    }
+
+    private void transitionToScheduled(final List<Execution> 
executionsToDeploy) {
+        executionsToDeploy.forEach(e -> 
e.transitionState(ExecutionState.SCHEDULED));
+    }
+
+    private List<ExecutionSlotAssignment> allocateSlotsFor(
+            final List<Execution> executionsToDeploy) {
+        final List<ExecutionAttemptID> executionAttemptIds =
+                executionsToDeploy.stream()
+                        .map(Execution::getAttemptId)
+                        .collect(Collectors.toList());
+        return executionSlotAllocator.allocateSlotsFor(executionAttemptIds);
+    }
+
+    private List<ExecutionDeploymentHandle> createDeploymentHandles(
+            final Map<ExecutionVertexID, ExecutionVertexVersion> 
requiredVersionByVertex,
+            final List<ExecutionSlotAssignment> executionSlotAssignments) {
+
+        return executionSlotAssignments.stream()
+                .map(
+                        executionSlotAssignment -> {
+                            final Execution execution =
+                                    
getExecution(executionSlotAssignment.getExecutionAttemptId());
+                            final ExecutionVertexID executionVertexId =
+                                    execution.getVertex().getID();
+                            return new ExecutionDeploymentHandle(
+                                    executionSlotAssignment,
+                                    
requiredVersionByVertex.get(executionVertexId));
+                        })
+                .collect(Collectors.toList());
+    }
+
+    private void waitForAllSlotsAndDeploy(final 
List<ExecutionDeploymentHandle> deploymentHandles) {
+        FutureUtils.assertNoException(
+                
assignAllResourcesAndRegisterProducedPartitions(deploymentHandles)
+                        .handle(deployAll(deploymentHandles)));
+    }
+
+    private CompletableFuture<Void> 
assignAllResourcesAndRegisterProducedPartitions(
+            final List<ExecutionDeploymentHandle> deploymentHandles) {
+        final List<CompletableFuture<Void>> resultFutures = new ArrayList<>();
+        for (ExecutionDeploymentHandle deploymentHandle : deploymentHandles) {
+            final CompletableFuture<Void> resultFuture =
+                    deploymentHandle
+                            .getLogicalSlotFuture()
+                            .handle(assignResource(deploymentHandle))
+                            
.thenCompose(registerProducedPartitions(deploymentHandle))
+                            .handle(
+                                    (ignore, throwable) -> {
+                                        if (throwable != null) {
+                                            handleTaskDeploymentFailure(
+                                                    
deploymentHandle.getExecutionAttemptId(),
+                                                    throwable);
+                                        }
+                                        return null;
+                                    });
+
+            resultFutures.add(resultFuture);
+        }
+        return FutureUtils.waitForAll(resultFutures);
+    }
+
+    private BiFunction<Void, Throwable, Void> deployAll(
+            final List<ExecutionDeploymentHandle> deploymentHandles) {
+        return (ignored, throwable) -> {
+            propagateIfNonNull(throwable);
+            for (final ExecutionDeploymentHandle deploymentHandle : 
deploymentHandles) {
+                final CompletableFuture<LogicalSlot> slotAssigned =
+                        deploymentHandle.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, LogicalSlot> assignResource(
+            final ExecutionDeploymentHandle deploymentHandle) {
+
+        return (logicalSlot, throwable) -> {
+            final ExecutionVertexVersion requiredVertexVersion =
+                    deploymentHandle.getRequiredVertexVersion();
+            final Execution execution = 
getExecution(deploymentHandle.getExecutionAttemptId());
+
+            if (execution == null
+                    || execution.getState() != ExecutionState.SCHEDULED
+                    || 
executionVertexVersioner.isModified(requiredVertexVersion)) {
+                if (throwable == null) {
+                    log.debug(
+                            "Refusing to assign slot to execution {} because 
this deployment was "
+                                    + "superseded by another deployment",
+                            deploymentHandle.getExecutionAttemptId());
+                    releaseSlotIfPresent(logicalSlot);
+                }
+                return null;
+            }
+
+            // throw exception only if the execution version is not outdated.
+            // this ensures that canceling a pending slot request does not fail
+            // a task which is about to cancel.
+            if (throwable != null) {
+                throw new 
CompletionException(maybeWrapWithNoResourceAvailableException(throwable));
+            }
+
+            if (!execution.tryAssignResource(logicalSlot)) {
+                throw new IllegalStateException(
+                        "Could not assign resource "
+                                + logicalSlot
+                                + " to execution "
+                                + execution
+                                + '.');
+            }
+
+            allocationReservationFunc.accept(
+                    execution.getAttemptId().getExecutionVertexId(), 
logicalSlot.getAllocationId());
+
+            return logicalSlot;
+        };
+    }
+
+    private static void releaseSlotIfPresent(@Nullable final LogicalSlot 
logicalSlot) {
+        if (logicalSlot != null) {
+            logicalSlot.releaseSlot(null);
+        }
+    }
+
+    private static Throwable maybeWrapWithNoResourceAvailableException(final 
Throwable failure) {
+        final Throwable strippedThrowable = 
ExceptionUtils.stripCompletionException(failure);
+        if (strippedThrowable instanceof TimeoutException) {
+            return new NoResourceAvailableException(
+                    "Could not allocate the required slot within slot request 
timeout. "
+                            + "Please make sure that the cluster has enough 
resources.",
+                    failure);
+        } else {
+            return failure;
+        }
+    }
+
+    private Function<LogicalSlot, CompletableFuture<Void>> 
registerProducedPartitions(
+            final ExecutionDeploymentHandle deploymentHandle) {
+
+        return logicalSlot -> {
+            // a null logicalSlot means the slot assignment is skipped, in 
which case
+            // the produced partition registration process can be skipped as 
well
+            if (logicalSlot != null) {
+                final Execution execution = 
getExecution(deploymentHandle.getExecutionAttemptId());
+                final CompletableFuture<Void> partitionRegistrationFuture =
+                        
execution.registerProducedPartitions(logicalSlot.getTaskManagerLocation());
+
+                return FutureUtils.orTimeout(
+                        partitionRegistrationFuture,
+                        partitionRegistrationTimeout.toMilliseconds(),
+                        TimeUnit.MILLISECONDS,
+                        mainThreadExecutor);
+            } else {
+                return FutureUtils.completedVoidFuture();
+            }
+        };
+    }
+
+    private BiFunction<Object, Throwable, Void> deployOrHandleError(
+            final ExecutionDeploymentHandle deploymentHandle) {
+
+        return (ignored, throwable) -> {
+            final ExecutionVertexVersion requiredVertexVersion =
+                    deploymentHandle.getRequiredVertexVersion();
+            final Execution execution = 
getExecution(deploymentHandle.getExecutionAttemptId());
+
+            if (execution == null
+                    || execution.getState() != ExecutionState.SCHEDULED
+                    || 
executionVertexVersioner.isModified(requiredVertexVersion)) {
+                if (throwable == null) {
+                    log.debug(
+                            "Refusing to assign slot to execution {} because 
this deployment was "
+                                    + "superseded by another deployment",
+                            deploymentHandle.getExecutionAttemptId());
+                }
+                return null;
+            }
+
+            if (throwable == null) {
+                deployTaskSafe(execution);
+            } else {
+                handleTaskDeploymentFailure(execution.getAttemptId(), 
throwable);
+            }
+            return null;
+        };
+    }
+
+    private void deployTaskSafe(final Execution execution) {
+        try {
+            executionOperations.deploy(execution);
+        } catch (Throwable e) {
+            handleTaskDeploymentFailure(execution.getAttemptId(), e);
+        }
+    }
+
+    private void handleTaskDeploymentFailure(
+            final ExecutionAttemptID executionAttemptId, final Throwable 
error) {
+
+        final Execution execution = getExecution(executionAttemptId);
+        executionOperations.markFailed(execution, error);
+    }
+
+    private Execution getExecution(ExecutionAttemptID executionAttemptId) {

Review Comment:
   Changed the return type to `Optional<Execution>` (Flink code style suggests 
the return type to be `Optional` instead of `Nullable`). A 
`getExecutionOrThrow(...)` is added which alway expects the execution to be 
non-null.



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

To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to