You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/21 11:27:31 UTC

[GitHub] [flink] zhuzhurk opened a new pull request, #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

zhuzhurk opened a new pull request, #20039:
URL: https://github.com/apache/flink/pull/20039

   ## What is the purpose of the change
   
   Currently, the DefaultScheduler(base of AdaptiveBatchScheduler) can only perform ExecutionVertex level deployment. However, in this case, the scheduler is actually deploying the current execution attempt of the ExecutionVertex.
   
   Therefore, we need to rework the DefaultScheduler to directly deploy executions.
   
   ## Brief change log
   
     - *Introduced ExecutionDeployer which is responsible for deploying executions*
     - *Implemented DefaultExecutionDeployer*
     - *Change DefaultScheduler to use ExecutionDeployer to deploy tasks*
   
   ## Verifying this change
   
     - *Added unit tests DefaultExecutionDeployerTest*
     - *The existing DefaultSchedulerTest and all other IT cases can help to verify the changes*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (**yes** / no / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] reswqa commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r903456529


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+@ExtendWith(TestLoggerExtension.class)
+public class DefaultExecutionDeployerTest {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    void setUp() {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    void tearDown() {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {

Review Comment:
   All test cases are best not to be public.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+@ExtendWith(TestLoggerExtension.class)
+public class DefaultExecutionDeployerTest {

Review Comment:
   In JUnit 5, test classes, test methods, and lifecycle methods are not required to be public, but they must not be private. And the [JUnit5 user guide ](https://junit.org/junit5/docs/snapshot/user-guide/#writing-tests-classes-and-methods)recommends omitting the public modifier for them unless there is a technical reason – for example, when a test class is extended by a test class in another package.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+@ExtendWith(TestLoggerExtension.class)
+public class DefaultExecutionDeployerTest {

Review Comment:
   ```suggestion
   class DefaultExecutionDeployerTest {
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+@ExtendWith(TestLoggerExtension.class)
+public class DefaultExecutionDeployerTest {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    void setUp() {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    void tearDown() {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {

Review Comment:
   ```suggestion
       void testDeployTasks() throws Exception {
   ```



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r903493252


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+@ExtendWith(TestLoggerExtension.class)
+public class DefaultExecutionDeployerTest {

Review Comment:
   Thanks for the suggestion and the reference!



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+@ExtendWith(TestLoggerExtension.class)
+public class DefaultExecutionDeployerTest {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    void setUp() {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    void tearDown() {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {

Review Comment:
   Ok.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r903414036


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.timeoutPendingRequest(attemptId);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).contains(attemptId);

Review Comment:
   Ok.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {

Review Comment:
   Ok.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905684724


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);

Review Comment:
   Done.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905682843


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

Review Comment:
   Done.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java:
##########
@@ -524,8 +525,13 @@ private SlotSharingExecutionSlotAllocator getAllocator() {
             return allocator;
         }
 
-        private List<SlotExecutionVertexAssignment> allocateSlotsFor(ExecutionVertexID... ids) {
-            return allocator.allocateSlotsFor(Arrays.asList(ids));
+        private List<ExecutionSlotAssignment> allocateSlotsFor(ExecutionVertexID... ids) {
+            return allocator.allocateSlotsFor(
+                    Arrays.asList(ids).stream()

Review Comment:
   Done.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionOperations.java:
##########
@@ -20,16 +20,34 @@
 package org.apache.flink.runtime.scheduler;
 
 import org.apache.flink.runtime.JobException;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.Execution;
 
 import java.util.concurrent.CompletableFuture;
 
-/** Operations on the {@link ExecutionVertex}. */
-public interface ExecutionVertexOperations {
+/** Operations on the {@link Execution}. */
+public interface ExecutionOperations {
 
-    void deploy(ExecutionVertex executionVertex) throws JobException;
+    /**
+     * Deploy the execution.
+     *
+     * @param execution to deploy.
+     * @throws JobException

Review Comment:
   Description is added.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] wanglijie95 commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
wanglijie95 commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905694491


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.timeoutPendingRequest(attemptId);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).contains(attemptId);
+    }
+
+    @Test
+    public void testSkipDeploymentIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionOperations.getDeployedVertices()).hasSize(0);
+    }
+
+    @Test
+    public void testReleaseSlotIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionSlotAllocator.getReturnedSlots()).hasSize(1);
+    }
+
+    @Test
+    public void testDeployOnlyIfVertexIsCreated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        // deploy once to transition the tasks out from CREATED state
+        deployTasks(executionDeployer, executionGraph);
+
+        // The deploying of a non-CREATED vertex will result in IllegalStateException
+        try {
+            deployTasks(executionDeployer, executionGraph);
+            fail("IllegalStateException should happen");
+        } catch (IllegalStateException e) {
+            // expected exception
+        }
+    }
+
+    @Test
+    public void testDeploymentWaitForProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setAutoCompleteRegistration(false);
+
+        final List<ResultPartitionID> trackedPartitions = new ArrayList<>();
+        partitionTracker.setStartTrackingPartitionsConsumer(
+                (resourceID, resultPartitionDeploymentDescriptor) ->
+                        trackedPartitions.add(
+                                resultPartitionDeploymentDescriptor
+                                        .getShuffleDescriptor()
+                                        .getResultPartitionID()));
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(trackedPartitions).hasSize(0);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        shuffleMaster.completeAllPendingRegistrations();
+        assertThat(trackedPartitions).hasSize(1);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+    }
+
+    @Test
+    public void testFailedProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setAutoCompleteRegistration(false);
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(0);
+
+        shuffleMaster.failAllPendingRegistrations();
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(1);
+    }
+
+    @Test
+    public void testDirectExceptionOnProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setThrowExceptionalOnRegistration(true);
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(1);
+    }
+
+    @Test

Review Comment:
   OK



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java:
##########
@@ -94,6 +96,34 @@ class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
         this.sharedSlots = new IdentityHashMap<>();
     }
 
+    @Override
+    public List<ExecutionSlotAssignment> allocateSlotsFor(
+            List<ExecutionAttemptID> executionAttemptIds) {
+
+        final Map<ExecutionVertexID, ExecutionAttemptID> vertexIdToExecutionId = new HashMap<>();
+        executionAttemptIds.forEach(
+                executionId ->
+                        vertexIdToExecutionId.put(executionId.getExecutionVertexId(), executionId));
+
+        checkState(
+                vertexIdToExecutionId.size() == executionAttemptIds.size(),
+                "SlotSharingExecutionSlotAllocator does not support one execution vertex to have multiple concurrent executions");
+
+        final List<ExecutionVertexID> vertexIds =

Review Comment:
   OK



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r903416184


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

Review Comment:
   You are right.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905685423


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.timeoutPendingRequest(attemptId);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).contains(attemptId);
+    }
+
+    @Test
+    public void testSkipDeploymentIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionOperations.getDeployedVertices()).hasSize(0);
+    }
+
+    @Test
+    public void testReleaseSlotIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionSlotAllocator.getReturnedSlots()).hasSize(1);
+    }
+
+    @Test
+    public void testDeployOnlyIfVertexIsCreated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        // deploy once to transition the tasks out from CREATED state
+        deployTasks(executionDeployer, executionGraph);
+
+        // The deploying of a non-CREATED vertex will result in IllegalStateException
+        try {
+            deployTasks(executionDeployer, executionGraph);
+            fail("IllegalStateException should happen");
+        } catch (IllegalStateException e) {
+            // expected exception
+        }
+    }
+
+    @Test
+    public void testDeploymentWaitForProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setAutoCompleteRegistration(false);
+
+        final List<ResultPartitionID> trackedPartitions = new ArrayList<>();
+        partitionTracker.setStartTrackingPartitionsConsumer(
+                (resourceID, resultPartitionDeploymentDescriptor) ->
+                        trackedPartitions.add(
+                                resultPartitionDeploymentDescriptor
+                                        .getShuffleDescriptor()
+                                        .getResultPartitionID()));
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(trackedPartitions).hasSize(0);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        shuffleMaster.completeAllPendingRegistrations();
+        assertThat(trackedPartitions).hasSize(1);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+    }
+
+    @Test
+    public void testFailedProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setAutoCompleteRegistration(false);
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(0);
+
+        shuffleMaster.failAllPendingRegistrations();
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(1);
+    }
+
+    @Test
+    public void testDirectExceptionOnProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setThrowExceptionalOnRegistration(true);
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(1);
+    }
+
+    @Test

Review Comment:
   Test timeout should be avoided. See "Avoid timeouts in JUnit tests" in https://flink.apache.org/contributing/code-style-and-quality-common.html#testing.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905687273


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionDeployer.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/** This deployer is responsible for deploying executions. */
+interface ExecutionDeployer {
+
+    /**
+     * Allocate slots and deploy executions.
+     *
+     * @param executionsToDeploy executions to deploy
+     * @param requiredVersionByVertex required versions of the execution vertices. If the actual
+     *     version does not match, the deployment of the execution will be rejected.
+     */
+    void allocateSlotsAndDeploy(
+            final List<Execution> executionsToDeploy,
+            final Map<ExecutionVertexID, ExecutionVertexVersion> requiredVersionByVertex);
+
+    /** Factory to instantiate the {@link ExecutionDeployer}. */
+    interface Factory {
+
+        /**
+         * Instantiate an {@link ExecutionDeployer} with the given params.
+         *
+         * @param log the logger
+         * @param executionSlotAllocator the allocator to allocate slots
+         * @param executionOperations the operations of executions
+         * @param executionVertexVersioner the versioner which records the versions of execution
+         *     vertices
+         * @param partitionRegistrationTimeout timeout of partition registration
+         * @param executionRetriever retriever to get executions
+         * @param allocationReservationFunc function to reserve allocations for local recovery

Review Comment:
   Done.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905684231


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java:
##########
@@ -117,8 +147,7 @@ class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
      *
      * @param executionVertexIds Execution vertices to allocate slots for
      */
-    @Override
-    public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+    private List<SlotExecutionVertexAssignment> allocateSlotsForVertices(

Review Comment:
   Agreed.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r903415505


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {

Review Comment:
   True. With Junit5 the before/after methods are no longer required to be public.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);

Review Comment:
   Ok.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.timeoutPendingRequest(attemptId);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).contains(attemptId);
+    }
+
+    @Test
+    public void testSkipDeploymentIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionOperations.getDeployedVertices()).hasSize(0);
+    }
+
+    @Test
+    public void testReleaseSlotIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionSlotAllocator.getReturnedSlots()).hasSize(1);
+    }
+
+    @Test
+    public void testDeployOnlyIfVertexIsCreated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        // deploy once to transition the tasks out from CREATED state
+        deployTasks(executionDeployer, executionGraph);
+
+        // The deploying of a non-CREATED vertex will result in IllegalStateException
+        try {
+            deployTasks(executionDeployer, executionGraph);
+            fail("IllegalStateException should happen");
+        } catch (IllegalStateException e) {
+            // expected exception
+        }

Review Comment:
   Ok.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] wanglijie95 commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
wanglijie95 commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r904826870


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocatorTest.java:
##########
@@ -524,8 +525,13 @@ private SlotSharingExecutionSlotAllocator getAllocator() {
             return allocator;
         }
 
-        private List<SlotExecutionVertexAssignment> allocateSlotsFor(ExecutionVertexID... ids) {
-            return allocator.allocateSlotsFor(Arrays.asList(ids));
+        private List<ExecutionSlotAssignment> allocateSlotsFor(ExecutionVertexID... ids) {
+            return allocator.allocateSlotsFor(
+                    Arrays.asList(ids).stream()

Review Comment:
   `Arrays.asList(ids).stream()` -> `Arrays.stream(ids)`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionOperations.java:
##########
@@ -20,16 +20,34 @@
 package org.apache.flink.runtime.scheduler;
 
 import org.apache.flink.runtime.JobException;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.Execution;
 
 import java.util.concurrent.CompletableFuture;
 
-/** Operations on the {@link ExecutionVertex}. */
-public interface ExecutionVertexOperations {
+/** Operations on the {@link Execution}. */
+public interface ExecutionOperations {
 
-    void deploy(ExecutionVertex executionVertex) throws JobException;
+    /**
+     * Deploy the execution.
+     *
+     * @param execution to deploy.
+     * @throws JobException

Review Comment:
   This line can be removed if no description



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java:
##########
@@ -117,8 +147,7 @@ class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
      *
      * @param executionVertexIds Execution vertices to allocate slots for
      */
-    @Override
-    public List<SlotExecutionVertexAssignment> allocateSlotsFor(
+    private List<SlotExecutionVertexAssignment> allocateSlotsForVertices(

Review Comment:
   Currently, the `SlotExecutionVertexAssignment` is only used in `SlotSharingExecutionSlotAllocator`, maybe let it as an inner class of `SlotSharingExecutionSlotAllocator` .



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

Review Comment:
   `slotExecutionVertexAssignments` -> `executionSlotAssignments`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java:
##########
@@ -94,6 +96,34 @@ class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
         this.sharedSlots = new IdentityHashMap<>();
     }
 
+    @Override
+    public List<ExecutionSlotAssignment> allocateSlotsFor(
+            List<ExecutionAttemptID> executionAttemptIds) {
+
+        final Map<ExecutionVertexID, ExecutionAttemptID> vertexIdToExecutionId = new HashMap<>();
+        executionAttemptIds.forEach(
+                executionId ->
+                        vertexIdToExecutionId.put(executionId.getExecutionVertexId(), executionId));
+
+        checkState(
+                vertexIdToExecutionId.size() == executionAttemptIds.size(),
+                "SlotSharingExecutionSlotAllocator does not support one execution vertex to have multiple concurrent executions");
+
+        final List<ExecutionVertexID> vertexIds =

Review Comment:
   `vertexIds = new ArrayList<>(vertexIdToExecutionId.keySet())`



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionDeployer.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/** This deployer is responsible for deploying executions. */
+interface ExecutionDeployer {
+
+    /**
+     * Allocate slots and deploy executions.
+     *
+     * @param executionsToDeploy executions to deploy
+     * @param requiredVersionByVertex required versions of the execution vertices. If the actual
+     *     version does not match, the deployment of the execution will be rejected.
+     */
+    void allocateSlotsAndDeploy(
+            final List<Execution> executionsToDeploy,
+            final Map<ExecutionVertexID, ExecutionVertexVersion> requiredVersionByVertex);
+
+    /** Factory to instantiate the {@link ExecutionDeployer}. */
+    interface Factory {
+
+        /**
+         * Instantiate an {@link ExecutionDeployer} with the given params.
+         *
+         * @param log the logger
+         * @param executionSlotAllocator the allocator to allocate slots
+         * @param executionOperations the operations of executions
+         * @param executionVertexVersioner the versioner which records the versions of execution
+         *     vertices
+         * @param partitionRegistrationTimeout timeout of partition registration
+         * @param executionRetriever retriever to get executions
+         * @param allocationReservationFunc function to reserve allocations for local recovery

Review Comment:
   Add more detailed comments for this param



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);

Review Comment:
   `getPendingRequests().keySet()` -> `getPendingRequests()`



##########
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:
   Can the return value be `null` ? If yes, maybe add `@Nullable`  and add checks.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.timeoutPendingRequest(attemptId);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).contains(attemptId);
+    }
+
+    @Test
+    public void testSkipDeploymentIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionOperations.getDeployedVertices()).hasSize(0);
+    }
+
+    @Test
+    public void testReleaseSlotIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionSlotAllocator.getReturnedSlots()).hasSize(1);
+    }
+
+    @Test
+    public void testDeployOnlyIfVertexIsCreated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        // deploy once to transition the tasks out from CREATED state
+        deployTasks(executionDeployer, executionGraph);
+
+        // The deploying of a non-CREATED vertex will result in IllegalStateException
+        try {
+            deployTasks(executionDeployer, executionGraph);
+            fail("IllegalStateException should happen");
+        } catch (IllegalStateException e) {
+            // expected exception
+        }
+    }
+
+    @Test
+    public void testDeploymentWaitForProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setAutoCompleteRegistration(false);
+
+        final List<ResultPartitionID> trackedPartitions = new ArrayList<>();
+        partitionTracker.setStartTrackingPartitionsConsumer(
+                (resourceID, resultPartitionDeploymentDescriptor) ->
+                        trackedPartitions.add(
+                                resultPartitionDeploymentDescriptor
+                                        .getShuffleDescriptor()
+                                        .getResultPartitionID()));
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(trackedPartitions).hasSize(0);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        shuffleMaster.completeAllPendingRegistrations();
+        assertThat(trackedPartitions).hasSize(1);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(2);
+    }
+
+    @Test
+    public void testFailedProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setAutoCompleteRegistration(false);
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(0);
+
+        shuffleMaster.failAllPendingRegistrations();
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(1);
+    }
+
+    @Test
+    public void testDirectExceptionOnProducedPartitionRegistration() throws Exception {
+        shuffleMaster.setThrowExceptionalOnRegistration(true);
+
+        final JobGraph jobGraph = nonParallelSourceSinkJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).hasSize(1);
+    }
+
+    @Test

Review Comment:
   Maybe add a Timeout for this test?



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905685825


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SlotSharingExecutionSlotAllocator.java:
##########
@@ -94,6 +96,34 @@ class SlotSharingExecutionSlotAllocator implements ExecutionSlotAllocator {
         this.sharedSlots = new IdentityHashMap<>();
     }
 
+    @Override
+    public List<ExecutionSlotAssignment> allocateSlotsFor(
+            List<ExecutionAttemptID> executionAttemptIds) {
+
+        final Map<ExecutionVertexID, ExecutionAttemptID> vertexIdToExecutionId = new HashMap<>();
+        executionAttemptIds.forEach(
+                executionId ->
+                        vertexIdToExecutionId.put(executionId.getExecutionVertexId(), executionId));
+
+        checkState(
+                vertexIdToExecutionId.size() == executionAttemptIds.size(),
+                "SlotSharingExecutionSlotAllocator does not support one execution vertex to have multiple concurrent executions");
+
+        final List<ExecutionVertexID> vertexIds =

Review Comment:
   We cannot do this because we need to respect the given order.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhoulii commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhoulii commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r903336973


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

Review Comment:
   Maybe _Iterable#forEach_ is more concise.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r905687352


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionDeployer.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/** This deployer is responsible for deploying executions. */
+interface ExecutionDeployer {
+
+    /**
+     * Allocate slots and deploy executions.
+     *
+     * @param executionsToDeploy executions to deploy
+     * @param requiredVersionByVertex required versions of the execution vertices. If the actual
+     *     version does not match, the deployment of the execution will be rejected.
+     */
+    void allocateSlotsAndDeploy(
+            final List<Execution> executionsToDeploy,
+            final Map<ExecutionVertexID, ExecutionVertexVersion> requiredVersionByVertex);
+
+    /** Factory to instantiate the {@link ExecutionDeployer}. */
+    interface Factory {
+
+        /**
+         * Instantiate an {@link ExecutionDeployer} with the given params.
+         *
+         * @param log the logger
+         * @param executionSlotAllocator the allocator to allocate slots
+         * @param executionOperations the operations of executions
+         * @param executionVertexVersioner the versioner which records the versions of execution

Review Comment:
   Ok.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] flinkbot commented on pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
flinkbot commented on PR #20039:
URL: https://github.com/apache/flink/pull/20039#issuecomment-1161622492

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "479a459532e70b429886a4059647335f51dfd3f7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "479a459532e70b429886a4059647335f51dfd3f7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 479a459532e70b429886a4059647335f51dfd3f7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhoulii commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhoulii commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r903335396


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

Review Comment:
   ```suggestion
           executionsToDeploy.forEach(
   ```



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] reswqa commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
reswqa commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r902880992


##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {

Review Comment:
   ```suggestion
       void setUp() throws Exception {
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.timeoutPendingRequest(attemptId);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).contains(attemptId);

Review Comment:
   ```suggestion
           assertThat(testExecutionOperations.getFailedExecutions()).containsExactly(attemptId);
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.completePendingRequest(attemptId);
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);
+
+        testExecutionSlotAllocator.completePendingRequests();
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(4);
+    }
+
+    @Test
+    public void testDeploymentFailures() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionOperations.enableFailDeploy();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getFailedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testSlotAllocationTimeout() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(2);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionSlotAllocator.getPendingRequests().keySet()).hasSize(2);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+        testExecutionSlotAllocator.timeoutPendingRequest(attemptId);
+
+        assertThat(testExecutionOperations.getFailedExecutions()).contains(attemptId);
+    }
+
+    @Test
+    public void testSkipDeploymentIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionOperations.getDeployedVertices()).hasSize(0);
+    }
+
+    @Test
+    public void testReleaseSlotIfVertexVersionOutdated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+        deployTasks(executionDeployer, executionGraph);
+
+        final ExecutionAttemptID attemptId = getAnyExecution(executionGraph).getAttemptId();
+
+        executionVertexVersioner.recordModification(attemptId.getExecutionVertexId());
+        testExecutionSlotAllocator.completePendingRequests();
+
+        assertThat(testExecutionSlotAllocator.getReturnedSlots()).hasSize(1);
+    }
+
+    @Test
+    public void testDeployOnlyIfVertexIsCreated() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        // deploy once to transition the tasks out from CREATED state
+        deployTasks(executionDeployer, executionGraph);
+
+        // The deploying of a non-CREATED vertex will result in IllegalStateException
+        try {
+            deployTasks(executionDeployer, executionGraph);
+            fail("IllegalStateException should happen");
+        } catch (IllegalStateException e) {
+            // expected exception
+        }

Review Comment:
   ```suggestion
           assertThatThrownBy(() -> deployTasks(executionDeployer, executionGraph))
                   .as("IllegalStateException should happen")
                   .isInstanceOf(IllegalStateException.class);
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {

Review Comment:
   Junit5 recommends that test class and methods use default access modifiers as much as possible



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {

Review Comment:
   ```suggestion
   @ExtendWith(TestLoggerExtension.class)
   class DefaultExecutionDeployerTest {
   ```



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);

Review Comment:
   It is better to use `isEmpty` to assert an empty collection, which is more intuitive to read.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionDeployerTest.java:
##########
@@ -0,0 +1,388 @@
+/*
+ * 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.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.executiongraph.TestingDefaultExecutionGraphBuilder;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.TestingJobMasterPartitionTracker;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobGraphTestUtils;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+import org.apache.flink.runtime.shuffle.TestingShuffleMaster;
+import org.apache.flink.runtime.testtasks.NoOpInvokable;
+import org.apache.flink.util.ExecutorUtils;
+import org.apache.flink.util.IterableUtils;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Fail.fail;
+
+/** Tests for {@link DefaultExecutionDeployer}. */
+public class DefaultExecutionDeployerTest extends TestLogger {
+
+    private ScheduledExecutorService executor;
+    private ComponentMainThreadExecutor mainThreadExecutor;
+    private TestExecutionOperationsDecorator testExecutionOperations;
+    private ExecutionVertexVersioner executionVertexVersioner;
+    private TestExecutionSlotAllocator testExecutionSlotAllocator;
+    private TestingShuffleMaster shuffleMaster;
+    private TestingJobMasterPartitionTracker partitionTracker;
+    private Time partitionRegistrationTimeout;
+
+    @BeforeEach
+    public void setUp() throws Exception {
+        executor = Executors.newSingleThreadScheduledExecutor();
+        mainThreadExecutor = ComponentMainThreadExecutorServiceAdapter.forMainThread();
+        testExecutionOperations =
+                new TestExecutionOperationsDecorator(
+                        new ExecutionOperations() {
+                            @Override
+                            public void deploy(Execution execution) {}
+
+                            @Override
+                            public CompletableFuture<?> cancel(Execution execution) {
+                                return null;
+                            }
+
+                            @Override
+                            public void markFailed(Execution execution, Throwable cause) {}
+                        });
+        executionVertexVersioner = new ExecutionVertexVersioner();
+        testExecutionSlotAllocator = new TestExecutionSlotAllocator();
+        shuffleMaster = new TestingShuffleMaster();
+        partitionTracker = new TestingJobMasterPartitionTracker();
+        partitionRegistrationTimeout = Time.milliseconds(5000);
+    }
+
+    @AfterEach
+    public void tearDown() throws Exception {
+        if (executor != null) {
+            ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, executor);
+        }
+    }
+
+    @Test
+    public void testDeployTasks() throws Exception {
+        final JobGraph jobGraph = singleNonParallelJobVertexJobGraph();
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions())
+                .contains(getAnyExecution(executionGraph).getAttemptId());
+    }
+
+    @Test
+    public void testDeployTasksOnlyIfAllSlotRequestsAreFulfilled() throws Exception {
+        final JobGraph jobGraph = singleJobVertexJobGraph(4);
+        final ExecutionGraph executionGraph = createExecutionGraph(jobGraph);
+        final ExecutionDeployer executionDeployer = createExecutionDeployer(executionGraph);
+
+        testExecutionSlotAllocator.disableAutoCompletePendingRequests();
+
+        deployTasks(executionDeployer, executionGraph);
+
+        assertThat(testExecutionOperations.getDeployedExecutions()).hasSize(0);

Review Comment:
   ```suggestion
           assertThat(testExecutionOperations.getDeployedExecutions()).isEmpty();
   ```



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk commented on pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk commented on PR #20039:
URL: https://github.com/apache/flink/pull/20039#issuecomment-1165282963

   Thanks for the reviewing! @reswqa @zhoulii @wanglijie95 
   Merging.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] zhuzhurk closed pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
zhuzhurk closed pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions
URL: https://github.com/apache/flink/pull/20039


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] wanglijie95 commented on a diff in pull request #20039: [FLINK-28133][runtime] Rework DefaultScheduler to directly deploy executions

Posted by GitBox <gi...@apache.org>.
wanglijie95 commented on code in PR #20039:
URL: https://github.com/apache/flink/pull/20039#discussion_r904857176


##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionDeployer.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * 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.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID;
+
+import org.slf4j.Logger;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+
+/** This deployer is responsible for deploying executions. */
+interface ExecutionDeployer {
+
+    /**
+     * Allocate slots and deploy executions.
+     *
+     * @param executionsToDeploy executions to deploy
+     * @param requiredVersionByVertex required versions of the execution vertices. If the actual
+     *     version does not match, the deployment of the execution will be rejected.
+     */
+    void allocateSlotsAndDeploy(
+            final List<Execution> executionsToDeploy,
+            final Map<ExecutionVertexID, ExecutionVertexVersion> requiredVersionByVertex);
+
+    /** Factory to instantiate the {@link ExecutionDeployer}. */
+    interface Factory {
+
+        /**
+         * Instantiate an {@link ExecutionDeployer} with the given params.
+         *
+         * @param log the logger
+         * @param executionSlotAllocator the allocator to allocate slots
+         * @param executionOperations the operations of executions
+         * @param executionVertexVersioner the versioner which records the versions of execution

Review Comment:
   Maybe add comment to describe how the`ExecutionVertexVersion` affects deploying`Execution(s)`.



-- 
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-unsubscribe@flink.apache.org

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