Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/640#discussion_r29426482
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Backtracking.java
 ---
    @@ -0,0 +1,252 @@
    +/*
    + * 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.executiongraph;
    +
    +import akka.actor.ActorRef;
    +import akka.dispatch.OnComplete;
    +import akka.pattern.Patterns;
    +import com.google.common.base.Preconditions;
    +import org.apache.flink.runtime.akka.AkkaUtils;
    +import org.apache.flink.runtime.execution.ExecutionState;
    +import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
    +import org.apache.flink.runtime.messages.TaskMessages;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import scala.concurrent.Future;
    +
    +import java.util.ArrayDeque;
    +import java.util.Collection;
    +import java.util.Deque;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +
    +/**
    + * Backtracking is a mechanism to schedule only those Execution Vertices 
of an Execution Graph which
    + * do not have an intermediate result available. This is in contrast to 
the simple way of scheduling
    + * a job, where all Execution Vertices are executed starting from the 
source. The Backtracking starts
    + * from the sinks and traverses the Execution Graph to the sources. It 
only reaches the sources if
    + * no intermediate result could be found on the way.
    + *
    + * @see ExecutionGraph
    + * @see ExecutionVertex
    + * @see Execution
    + */
    +public class Backtracking {
    +
    +   private static final Logger LOG = 
LoggerFactory.getLogger(Backtracking.class);
    +
    +   private final Deque<TaskRequirement> taskRequirements = new 
ArrayDeque<TaskRequirement>();
    +
    +   private final Map<IntermediateResultPartitionID, Boolean> 
visitedPartitions = new HashMap<IntermediateResultPartitionID, Boolean>();
    +
    +   private ScheduleAction scheduleAction;
    +   private Runnable postBacktrackingHook;
    +   
    +   public Backtracking(Collection<ExecutionJobVertex> vertices) {
    +           Preconditions.checkNotNull(vertices);
    +
    +           // add all sinks found to the stack
    +           for (ExecutionJobVertex ejv : vertices) {
    +                   if (ejv.getJobVertex().isOutputVertex()) {
    +                           for (ExecutionVertex ev : 
ejv.getTaskVertices()) {
    +                                   if (ev.getExecutionState() == 
ExecutionState.CREATED) {
    +                                           taskRequirements.add(new 
TaskRequirement(ev));
    +                                   }
    +                           }
    +                   }
    +           }
    +
    +           this.scheduleAction = new ScheduleAction() {
    +                   @Override
    +                   public void schedule(ExecutionVertex ev) {}
    +           };
    +
    +           this.postBacktrackingHook = new Runnable() {
    +                   @Override
    +                   public void run() {}
    +           };
    +   }
    +
    +   /**
    +    * Scheduling to be performed when an ExecutionVertex is encountered 
that cannot be resumed
    +    */
    +   public interface ScheduleAction {
    +           void schedule(ExecutionVertex ev);
    +   }
    +
    +   /**
    +    * A TaskRequirement encaplusates an ExecutionVertex and its 
IntermediateResultPartitions which
    +    * are required for execution.
    +    */
    +   private class TaskRequirement {
    +
    +           private final ExecutionVertex executionVertex;
    +           private final Deque<IntermediateResultPartition> pendingInputs 
= new ArrayDeque<IntermediateResultPartition>();
    +           private final int numInputs;
    +
    +           private int availableInputs = 0;
    +
    +           public TaskRequirement(ExecutionVertex executionVertex) {
    +                   this.executionVertex = executionVertex;
    +                   this.pendingInputs.addAll(executionVertex.getInputs());
    +                   this.numInputs = pendingInputs.size();
    +           }
    +
    +           public ExecutionVertex getExecutionVertex() {
    +                   return executionVertex;
    +           }
    +
    +           public boolean pendingRequirements() {
    +                   for (IntermediateResultPartition ir : pendingInputs) {
    +                           Boolean visitedPartition = 
visitedPartitions.get(ir.getPartitionId());
    +                           if (visitedPartition == null) {
    +                                   return true;
    +                           } else {
    +                                   if (visitedPartition) {
    +                                           availableInputs++;
    +                                   }
    +                                   pendingInputs.remove(ir);
    --- End diff --
    
    As far as I know, it's unsafe to modify a collection while iterating over 
it if you don't use the remove method of the iterator directly. 
    
    Quote from JavaDoc: Iterator.remove() "The behavior of an iterator is 
unspecified if the underlying collection is modified while the iteration is in 
progress in any way other than by calling this method."


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to