GJL commented on a change in pull request #8309: [FLINK-12229] [runtime] 
Implement LazyFromSourcesScheduling Strategy
URL: https://github.com/apache/flink/pull/8309#discussion_r287003247
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java
 ##########
 @@ -0,0 +1,197 @@
+/*
+ * 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.strategy;
+
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.scheduler.DeploymentOption;
+import org.apache.flink.runtime.scheduler.ExecutionVertexDeploymentOption;
+import org.apache.flink.runtime.scheduler.SchedulerOperations;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * {@link SchedulingStrategy} instance for batch job which schedule vertices 
when input data are ready.
+ */
+public class LazyFromSourcesSchedulingStrategy implements SchedulingStrategy {
+
+       private final SchedulerOperations schedulerOperations;
+
+       private final SchedulingTopology schedulingTopology;
+
+       private final Map<ExecutionVertexID, DeploymentOption> 
deploymentOptions;
+
+       private final InputDependencyConstraintChecker inputConstraintChecker;
+
+       public LazyFromSourcesSchedulingStrategy(
+                       SchedulerOperations schedulerOperations,
+                       SchedulingTopology schedulingTopology) {
+               this.schedulerOperations = checkNotNull(schedulerOperations);
+               this.schedulingTopology = checkNotNull(schedulingTopology);
+               this.deploymentOptions = new HashMap<>();
+               this.inputConstraintChecker = new 
InputDependencyConstraintChecker();
+       }
+
+       @Override
+       public void startScheduling() {
+               List<ExecutionVertexDeploymentOption> 
executionVertexDeploymentOptions = new ArrayList<>();
+               final DeploymentOption updateOption = new 
DeploymentOption(true);
+               final DeploymentOption nonUpdateOption = new 
DeploymentOption(false);
+
+               for (SchedulingExecutionVertex schedulingVertex : 
schedulingTopology.getVertices()) {
+                       DeploymentOption option = nonUpdateOption;
+                       for (SchedulingResultPartition srp : 
schedulingVertex.getProducedResultPartitions()) {
+                               if (srp.getPartitionType().isBlocking()) {
+                                       
inputConstraintChecker.addSchedulingResultPartition(srp);
+                               } else {
+                                       option = updateOption;
+                               }
+                       }
+                       deploymentOptions.put(schedulingVertex.getId(), option);
+
+                       if 
(schedulingVertex.getConsumedResultPartitions().isEmpty()) {
+                               // schedule vertices without consumed result 
partition
+                               executionVertexDeploymentOptions.add(
+                                               new 
ExecutionVertexDeploymentOption(schedulingVertex.getId(), option));
+                       }
+               }
+
+               
schedulerOperations.allocateSlotsAndDeploy(executionVertexDeploymentOptions);
+       }
+
+       @Override
+       public void restartTasks(Set<ExecutionVertexID> verticesToRestart) {
+               // increase counter of the dataset first
+               for (ExecutionVertexID executionVertexId : verticesToRestart) {
+                       final SchedulingExecutionVertex schedulingVertex = 
schedulingTopology.getVertex(executionVertexId)
 
 Review comment:
   I think we can add `getVertexOrThrow(ExecutionVertexID)` to the topology. 
Can you create a ticket for that?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to