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

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/strategy/LazyFromSourcesSchedulingStrategy.java
 ##########
 @@ -0,0 +1,227 @@
+/*
+ * 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.api.common.InputDependencyConstraint;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+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.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+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 DeploymentOption deploymentOption = new 
DeploymentOption(true);
+
+       private final JobGraph jobGraph;
+
+       private final Map<IntermediateDataSetID, SchedulingIntermediateDataSet> 
intermediateDataSets;
+
+       public LazyFromSourcesSchedulingStrategy(
+               SchedulerOperations schedulerOperations,
+               SchedulingTopology schedulingTopology,
+               JobGraph jobGraph) {
+               this.schedulerOperations = checkNotNull(schedulerOperations);
+               this.schedulingTopology = checkNotNull(schedulingTopology);
+               this.jobGraph = checkNotNull(jobGraph);
+               this.intermediateDataSets = new HashMap<>();
+       }
+
+       @Override
+       public void startScheduling() {
+               List<ExecutionVertexDeploymentOption> 
executionVertexDeploymentOptions = new LinkedList<>();
+               for (SchedulingVertex schedulingVertex : 
schedulingTopology.getVertices()) {
+                       if 
(jobGraph.findVertexByID(schedulingVertex.getJobVertexId()).isInputVertex()) {
+                               // schedule vertices without consumed result 
partition
+                               executionVertexDeploymentOptions.add(
+                                       new 
ExecutionVertexDeploymentOption(schedulingVertex.getId(), deploymentOption));
+                       }
+
+                       Collection<SchedulingResultPartition> partitions = 
schedulingVertex.getProducedResultPartitions();
+                       if (partitions != null) {
+                               for (SchedulingResultPartition srp : 
partitions) {
+                                       SchedulingIntermediateDataSet sid = 
intermediateDataSets.computeIfAbsent(srp.getResultId(),
+                                               (key) -> new 
SchedulingIntermediateDataSet());
+                                       sid.increaseProducerCnt();
+                               }
+                       }
+               }
+               if (!executionVertexDeploymentOptions.isEmpty()) {
+                       
schedulerOperations.allocateSlotsAndDeploy(executionVertexDeploymentOptions);
+               }
+       }
+
+       @Override
+       public void restartTasks(Set<ExecutionVertexID> verticesNeedingRestart) 
{
 
 Review comment:
   > I thought the verticesNeedingRestart of restartTasks are well chosen for 
directly scheduling vertices before.
   
   The vertices are well chosen but in some cases all vertices need to be 
restarted.
   
   > the latter schedules vertices with [...] ANY/ALL input result partition is 
ready in verticesNeedingRestart
   
   That's correct but if a vertex has 0 input result partitions, _all_ of the 
vertex's input partitions are _trivially_ ready. For example, the code below 
[prints `true` twice](https://repl.it/repls/AquamarinePeacefulPublishing):
   ```
   public static void main(String[] args) {
        System.out.println(Collections.emptyList().stream().allMatch(o -> 
false));
        System.out.println(Collections.emptyList().stream().allMatch(o -> 
true));
   }
   ```
   
   Maybe this helps to simplify the code and use virtually the same code path 
for both methods, or at least reuse some of the code.

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