[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31130185
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
 ---
@@ -40,9 +44,18 @@
 
private static final Logger LOG = 
LoggerFactory.getLogger(ResultPartitionManager.class);
 
+   /**
+* Table of running/finished ResultPartitions with corresponding IDs
+*/
public final TableExecutionAttemptID, IntermediateResultPartitionID, 
ResultPartition
registeredPartitions = HashBasedTable.create();
 
+   /**
+* Cached ResultPartitions which are used to resume/recover from
--- End diff --

Let's add a comment that the LinkedHashMap implements a LRU policy.


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31136509
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
 ---
@@ -264,21 +266,21 @@ public void attachJobGraph(ListAbstractJobVertex 
topologiallySorted) throws Jo
LOG.debug(String.format(Attaching %d topologically 
sorted vertices to existing job graph with %d 
--- End diff --

We have to check that the parallelism of both vertices are the same, 
because the runtime results are already partitioned. I think a good error msg 
is the best solution at this point.


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31136136
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
 ---
@@ -119,10 +153,50 @@ public void shutdown() {
}
}
 
+   /**
+* Registers and pins a cached ResultPartition that holds the data for 
an IntermediateResultPartition.
+* @param partitionID The IntermediateResultPartitionID to find a 
corresponding ResultPartition for.
+* @param numConsumers The number of consumers that want to access the 
ResultPartition
+* @return true if the registering/pinning succeeded, false otherwise.
+*/
+   public boolean pinCachedResultPartition(IntermediateResultPartitionID 
partitionID, int numConsumers) {
+   synchronized (cachedResultPartitions) {
+   ResultPartition resultPartition = 
cachedResultPartitions.get(partitionID);
+   if (resultPartition != null) {
+   try {
+   // update its least recently used value
+   
updateIntermediateResultPartitionCache(resultPartition);
+
+   synchronized (registeredPartitions) {
+   if 
(!registeredPartitions.containsValue(resultPartition)) {
+   LOG.debug(Registered 
previously cached ResultPartition {}., resultPartition);
+   
registerResultPartition(resultPartition);
+   }
+   }
+
+   for (int i = 0; i  numConsumers; i++) {
+   resultPartition.pin();
--- End diff --

Pinning a result partition, increases the pending references count by the 
total number of subpartitions. We need to adjust it to just increment it for 
each call.


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31130488
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
 ---
@@ -40,9 +44,18 @@
 
private static final Logger LOG = 
LoggerFactory.getLogger(ResultPartitionManager.class);
 
+   /**
+* Table of running/finished ResultPartitions with corresponding IDs
+*/
public final TableExecutionAttemptID, IntermediateResultPartitionID, 
ResultPartition
registeredPartitions = HashBasedTable.create();
 
+   /**
+* Cached ResultPartitions which are used to resume/recover from
--- End diff --

There are two unused classes LRUCache and LRUCacheMap, which can be 
removed. I think this solution is fine. :)


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31131135
  
--- Diff: 
flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
 ---
@@ -334,6 +335,14 @@ extends Actor with ActorLogMessages with 
ActorSynchronousLogging {
   }
 }
 
+  /**
+   * Pin a ResultPartition corresponding to an 
IntermediateResultPartition
+   */
+  case LockResultPartition(partitionID, numConsumers) =
+val partitionManager: ResultPartitionManager = 
this.network.getPartitionManager
--- End diff --

@StephanEwen after a disconnect from the JM, the TM will clear the network 
stack components and the partition manager might be null. Is it possible that a 
msg is received after the disconnect, e.g. do we have to check for null here?


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31131305
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
 ---
@@ -137,9 +211,46 @@ void onConsumedPartition(ResultPartition partition) {
 
// Release the partition if it was successfully removed
if (partition == previous) {
-   partition.release();
+   // move to cache if cachable
+   updateIntermediateResultPartitionCache(partition);
 
-   LOG.debug(Released {}., partition);
+   LOG.debug(Cached {}., partition);
--- End diff --

Let's adjust this log message to reflect that a partition might also be 
released after the update call.


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31134168
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
 ---
@@ -119,10 +153,50 @@ public void shutdown() {
}
}
 
+   /**
+* Registers and pins a cached ResultPartition that holds the data for 
an IntermediateResultPartition.
+* @param partitionID The IntermediateResultPartitionID to find a 
corresponding ResultPartition for.
+* @param numConsumers The number of consumers that want to access the 
ResultPartition
+* @return true if the registering/pinning succeeded, false otherwise.
+*/
+   public boolean pinCachedResultPartition(IntermediateResultPartitionID 
partitionID, int numConsumers) {
+   synchronized (cachedResultPartitions) {
+   ResultPartition resultPartition = 
cachedResultPartitions.get(partitionID);
+   if (resultPartition != null) {
+   try {
+   // update its least recently used value
+   
updateIntermediateResultPartitionCache(resultPartition);
+
+   synchronized (registeredPartitions) {
+   if 
(!registeredPartitions.containsValue(resultPartition)) {
+   LOG.debug(Registered 
previously cached ResultPartition {}., resultPartition);
+   
registerResultPartition(resultPartition);
--- End diff --

I think there is a race between pinning and releasing. It could happen that 
a pinned result is removed by a concurrent release (in the onConsumed callback).


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31137592
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionManager.java
 ---
@@ -137,9 +211,46 @@ void onConsumedPartition(ResultPartition partition) {
 
// Release the partition if it was successfully removed
if (partition == previous) {
-   partition.release();
+   // move to cache if cachable
+   updateIntermediateResultPartitionCache(partition);
 
-   LOG.debug(Released {}., partition);
+   LOG.debug(Cached {}., partition);
+   }
+   }
+
+   /**
+* Triggered by @link{NetworkBufferPool} when network buffers should be 
freed
+* @param requiredBuffers The number of buffers that should be cleared.
+*/
+   public boolean releaseLeastRecentlyUsedCachedPartitions (int 
requiredBuffers) {
+   synchronized (cachedResultPartitions) {
+   // make a list of ResultPartitions to release
+   ListResultPartition toBeReleased = new 
ArrayListResultPartition();
+   int numBuffersToBeFreed = 0;
+
+   // traverse from least recently used cached 
ResultPartition
+   for (Map.EntryIntermediateResultPartitionID, 
ResultPartition entry : cachedResultPartitions.entrySet()) {
+   ResultPartition cachedResult = entry.getValue();
+
+   synchronized (registeredPartitions) {
+   if 
(!registeredPartitions.containsValue(cachedResult)) {
+   if (numBuffersToBeFreed  
requiredBuffers) {
+   
toBeReleased.add(cachedResult);
+   numBuffersToBeFreed += 
cachedResult.getTotalNumberOfBuffers();
--- End diff --

Sorry for the bad naming, but this is the total number of produced buffers. 
We need the number of buffers, which are currently in memory.


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31138771
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultSubpartition.java
 ---
@@ -71,6 +71,11 @@ protected void onConsumedSubpartition() {
 
abstract public void finish() throws IOException;
 
+   /**
--- End diff --

Sorry for the missing comment. This acutally discards the result partition. 


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-05-27 Thread uce
Github user uce commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r31138954
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartition.java
 ---
@@ -130,16 +131,12 @@ public void release() throws IOException {
spillWriter.closeAndDelete();
}
 
-   // Get the view...
-   view = readView;
-   readView = null;
-
isReleased = true;
}
 
// Release the view outside of the synchronized block
-   if (view != null) {
-   view.notifySubpartitionConsumed();
+   if (!readView.isEmpty()) {
--- End diff --

Because this is discarding the whole partition, we should do this for all 
elements of the queue.


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-04-30 Thread tillrohrmann
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 DequeTaskRequirement taskRequirements = new 
ArrayDequeTaskRequirement();
+
+   private final MapIntermediateResultPartitionID, Boolean 
visitedPartitions = new HashMapIntermediateResultPartitionID, Boolean();
+
+   private ScheduleAction scheduleAction;
+   private Runnable postBacktrackingHook;
+   
+   public Backtracking(CollectionExecutionJobVertex 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 DequeIntermediateResultPartition pendingInputs 
= new ArrayDequeIntermediateResultPartition();
+   private final int numInputs;
+
+   private int availableInputs = 0;
+
+   public TaskRequirement(ExecutionVertex executionVertex) {
+   this.executionVertex = executionVertex;
+   

[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-04-30 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r29426706
  
--- 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 DequeTaskRequirement taskRequirements = new 
ArrayDequeTaskRequirement();
+
+   private final MapIntermediateResultPartitionID, Boolean 
visitedPartitions = new HashMapIntermediateResultPartitionID, Boolean();
+
+   private ScheduleAction scheduleAction;
+   private Runnable postBacktrackingHook;
+   
+   public Backtracking(CollectionExecutionJobVertex 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 DequeIntermediateResultPartition pendingInputs 
= new ArrayDequeIntermediateResultPartition();
+   private final int numInputs;
+
+   private int availableInputs = 0;
+
+   public TaskRequirement(ExecutionVertex executionVertex) {
+   this.executionVertex = executionVertex;
+   

[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-04-30 Thread mxm
Github user mxm commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r29432849
  
--- 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 DequeTaskRequirement taskRequirements = new 
ArrayDequeTaskRequirement();
+
+   private final MapIntermediateResultPartitionID, Boolean 
visitedPartitions = new HashMapIntermediateResultPartitionID, Boolean();
+
+   private ScheduleAction scheduleAction;
+   private Runnable postBacktrackingHook;
+   
+   public Backtracking(CollectionExecutionJobVertex 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 DequeIntermediateResultPartition pendingInputs 
= new ArrayDequeIntermediateResultPartition();
+   private final int numInputs;
+
+   private int availableInputs = 0;
+
+   public TaskRequirement(ExecutionVertex executionVertex) {
+   this.executionVertex = executionVertex;
+   this.pendingInputs.addAll(executionVertex.getInputs());

[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-04-30 Thread mxm
Github user mxm commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r29431402
  
--- 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 DequeTaskRequirement taskRequirements = new 
ArrayDequeTaskRequirement();
+
+   private final MapIntermediateResultPartitionID, Boolean 
visitedPartitions = new HashMapIntermediateResultPartitionID, Boolean();
+
+   private ScheduleAction scheduleAction;
+   private Runnable postBacktrackingHook;
+   
+   public Backtracking(CollectionExecutionJobVertex 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 DequeIntermediateResultPartition pendingInputs 
= new ArrayDequeIntermediateResultPartition();
+   private final int numInputs;
+
+   private int availableInputs = 0;
+
+   public TaskRequirement(ExecutionVertex executionVertex) {
+   this.executionVertex = executionVertex;
+   this.pendingInputs.addAll(executionVertex.getInputs());

[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-04-30 Thread mxm
Github user mxm commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r29431391
  
--- 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 DequeTaskRequirement taskRequirements = new 
ArrayDequeTaskRequirement();
+
+   private final MapIntermediateResultPartitionID, Boolean 
visitedPartitions = new HashMapIntermediateResultPartitionID, Boolean();
+
+   private ScheduleAction scheduleAction;
+   private Runnable postBacktrackingHook;
+   
+   public Backtracking(CollectionExecutionJobVertex 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 DequeIntermediateResultPartition pendingInputs 
= new ArrayDequeIntermediateResultPartition();
+   private final int numInputs;
+
+   private int availableInputs = 0;
+
+   public TaskRequirement(ExecutionVertex executionVertex) {
+   this.executionVertex = executionVertex;
+   this.pendingInputs.addAll(executionVertex.getInputs());

[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-04-30 Thread mxm
Github user mxm commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r29431413
  
--- Diff: 
flink-runtime/src/main/scala/org/apache/flink/runtime/jobmanager/JobManager.scala
 ---
@@ -475,11 +479,21 @@ class JobManager(val flinkConfiguration: 
Configuration,
   throw new JobSubmissionException(jobId, The given job is empty)
 }
 
-// see if there already exists an ExecutionGraph for the 
corresponding job ID
-executionGraph = currentJobs.getOrElseUpdate(jobGraph.getJobID,
-  (new ExecutionGraph(jobGraph.getJobID, jobGraph.getName,
-jobGraph.getJobConfiguration, timeout, 
jobGraph.getUserJarBlobKeys, userCodeLoader),
-JobInfo(sender(), System.currentTimeMillis(._1
+executionGraph = currentJob match {
+  case Some((graph, _)) if !graph.getState.isTerminalState =
+  throw new Exception(Job still running)
--- End diff --

I hadn't thought about attaching vertices to a running execution graph. 
That could makes sense. Especially, in the case of streaming.


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


[GitHub] flink pull request: [scheduling] implement backtracking of interme...

2015-04-30 Thread tillrohrmann
Github user tillrohrmann commented on a diff in the pull request:

https://github.com/apache/flink/pull/640#discussion_r29432295
  
--- 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 DequeTaskRequirement taskRequirements = new 
ArrayDequeTaskRequirement();
+
+   private final MapIntermediateResultPartitionID, Boolean 
visitedPartitions = new HashMapIntermediateResultPartitionID, Boolean();
+
+   private ScheduleAction scheduleAction;
+   private Runnable postBacktrackingHook;
+   
+   public Backtracking(CollectionExecutionJobVertex 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 DequeIntermediateResultPartition pendingInputs 
= new ArrayDequeIntermediateResultPartition();
+   private final int numInputs;
+
+   private int availableInputs = 0;
+
+   public TaskRequirement(ExecutionVertex executionVertex) {
+   this.executionVertex = executionVertex;
+