You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2014/09/21 04:12:25 UTC

[01/63] [abbrv] Refactor job graph construction to incremental attachment based

Repository: incubator-flink
Updated Branches:
  refs/heads/master 439ca7ffe -> 91cfbc5aa


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
index 8246d22..65c9857 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
@@ -131,7 +131,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		// --------------------------------------------------------------------------------------------------------------
 
 		// - input -----------------------------------------------------------------------------------------------------
-		InputFormatInputVertex input = JobGraphUtils.createInput(
+		InputFormatVertex input = JobGraphUtils.createInput(
 			new PointInFormat(), inputPath, "Input", jobGraph, numSubTasks);
 		TaskConfig inputConfig = new TaskConfig(input.getConfiguration());
 		{
@@ -214,7 +214,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - output ----------------------------------------------------------------------------------------------------
-		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		{
 			outputConfig.addInputToGroup(0);
@@ -225,10 +225,10 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - fake tail -------------------------------------------------------------------------------------------------
-		SimpleOutputVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks);
+		OutputFormatVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks);
 
 		// - sync ------------------------------------------------------------------------------------------------------
-		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
+		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(maxIterations);
 		syncConfig.setIterationId(ITERATION_ID);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
index 052c7ea..82bd046 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.iterative.nephele;
 
 import java.io.IOException;
@@ -33,12 +32,10 @@ import org.apache.flink.runtime.iterative.io.FakeOutputTask;
 import org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
 import org.apache.flink.runtime.operators.DataSinkTask;
 import org.apache.flink.runtime.operators.DataSourceTask;
 import org.apache.flink.runtime.operators.RegularPactTask;
@@ -56,21 +53,21 @@ public class JobGraphUtils {
 		client.submitJobAndWait();
 	}
 	
-	public static <T extends FileInputFormat<?>> InputFormatInputVertex createInput(T stub, String path, String name, JobGraph graph,
+	public static <T extends FileInputFormat<?>> InputFormatVertex createInput(T stub, String path, String name, JobGraph graph,
 			int degreeOfParallelism)
 	{
 		stub.setFilePath(path);
 		return createInput(new UserCodeObjectWrapper<T>(stub), name, graph, degreeOfParallelism);
 	}
 
-	private static <T extends InputFormat<?,?>> InputFormatInputVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
+	private static <T extends InputFormat<?,?>> InputFormatVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
 			int degreeOfParallelism)
 	{
-		InputFormatInputVertex inputVertex = new InputFormatInputVertex(name, graph);
+		InputFormatVertex inputVertex = new InputFormatVertex(graph, name);
 		
 		inputVertex.setInvokableClass(DataSourceTask.class);
 		
-		inputVertex.setNumberOfSubtasks(degreeOfParallelism);
+		inputVertex.setParallelism(degreeOfParallelism);
 
 		TaskConfig inputConfig = new TaskConfig(inputVertex.getConfiguration());
 		inputConfig.setStubWrapper(stub);
@@ -100,28 +97,28 @@ public class JobGraphUtils {
 		return taskVertex;
 	}
 
-	public static SimpleOutputVertex createSync(JobGraph jobGraph, int degreeOfParallelism) {
-		SimpleOutputVertex sync = new SimpleOutputVertex("BulkIterationSync", jobGraph);
+	public static OutputFormatVertex createSync(JobGraph jobGraph, int degreeOfParallelism) {
+		OutputFormatVertex sync = new OutputFormatVertex(jobGraph, "BulkIterationSync");
 		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
-		sync.setNumberOfSubtasks(1);
+		sync.setParallelism(1);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, degreeOfParallelism);
 		return sync;
 	}
 
-	public static SimpleOutputVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
+	public static OutputFormatVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
-		SimpleOutputVertex outputVertex = new SimpleOutputVertex(name, jobGraph);
+		OutputFormatVertex outputVertex = new OutputFormatVertex(jobGraph, name);
 		outputVertex.setInvokableClass(FakeOutputTask.class);
-		outputVertex.setNumberOfSubtasks(degreeOfParallelism);
+		outputVertex.setParallelism(degreeOfParallelism);
 		return outputVertex;
 	}
 
-	public static OutputFormatOutputVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
+	public static OutputFormatVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
-		OutputFormatOutputVertex sinkVertex = new OutputFormatOutputVertex(name, jobGraph);
+		OutputFormatVertex sinkVertex = new OutputFormatVertex(jobGraph, name);
 		sinkVertex.setInvokableClass(DataSinkTask.class);
-		sinkVertex.setNumberOfSubtasks(degreeOfParallelism);
+		sinkVertex.setParallelism(degreeOfParallelism);
 		return sinkVertex;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
index aea2c2c..a6771ba 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
@@ -267,7 +267,7 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -276,10 +276,10 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
+		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
 			degreeOfParallelism);
 
-		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
index a740cf3..7eacf1b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
@@ -138,7 +138,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		// --------------- the inputs ---------------------
 
 		// page rank input
-		InputFormatInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
+		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
 			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -147,7 +147,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
 
 		// edges as adjacency list
-		InputFormatInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
+		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
 			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -279,7 +279,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the output ---------------------
 
-		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -288,9 +288,10 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", degreeOfParallelism);
+		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
+			degreeOfParallelism);
 
-		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
index d1f4ae0..317963b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
@@ -119,7 +119,7 @@ public class CompensatableDanglingPageRank {
 		// --------------- the inputs ---------------------
 
 		// page rank input
-		InputFormatInputVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(),
+		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(),
 			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -128,7 +128,7 @@ public class CompensatableDanglingPageRank {
 		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
 
 		// edges as adjacency list
-		InputFormatInputVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(),
+		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(),
 			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -248,7 +248,7 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(recSerializer, 0);
@@ -257,10 +257,10 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
+		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
 			degreeOfParallelism);
 
-		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java
deleted file mode 100644
index fdc3375..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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 eu.stratosphere.nephele.jobgraph;
-
-/**
- * An intermediate data set is the data set produced by an operator - either a
- * source or any intermediate operation.
- * 
- * Intermediate data sets may be read by other operators, materialized, or
- * discarded.
- */
-public class IntermediateDataSet {
-	
-	private final IntermediateDataSetID id; 		// the identifier
-	
-	private final AbstractJobVertex producer;		// the operation that produced this data set
-
-	
-	public IntermediateDataSet(AbstractJobVertex producer) {
-		this(new IntermediateDataSetID(), producer);
-	}
-	
-	public IntermediateDataSet(IntermediateDataSetID id, AbstractJobVertex producer) {
-		this.id = id;
-		this.producer = producer;
-	}
-	
-	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java
deleted file mode 100644
index ac12be9..0000000
--- a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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 eu.stratosphere.nephele.jobgraph;
-
-import java.util.UUID;
-
-import eu.stratosphere.nephele.AbstractID;
-
-public class IntermediateDataSetID extends AbstractID {
-
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Creates an new random intermediate data set ID.
-	 */
-	public IntermediateDataSetID() {
-		super();
-	}
-	
-	/**
-	 * Creates a new intermediate data set ID with the bytes of the given ID.
-	 * 
-	 * @param from The ID to create this ID from.
-	 */
-	public IntermediateDataSetID(AbstractID from) {
-		super(from);
-	}
-	
-	/**
-	 * Creates a new intermediate data set ID with the bytes of the given UUID.
-	 * 
-	 * @param from The UUID to create this ID from.
-	 */
-	public IntermediateDataSetID(UUID from) {
-		super(from.getLeastSignificantBits(), from.getMostSignificantBits());
-	}
-}


[40/63] [abbrv] git commit: Finalize ExecutionGraph state machine and calls

Posted by se...@apache.org.
Finalize ExecutionGraph state machine and calls


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/ae139f5a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/ae139f5a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/ae139f5a

Branch: refs/heads/master
Commit: ae139f5ae2199a52e8d7f561f94db51631107d00
Parents: 43e7d0f
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Sep 11 07:18:56 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../org/apache/flink/util/ExceptionUtils.java   |   24 +-
 .../deployment/ChannelDeploymentDescriptor.java |    4 +-
 .../deployment/GateDeploymentDescriptor.java    |   10 +-
 .../event/job/ExecutionStateChangeEvent.java    |   12 +-
 .../flink/runtime/event/job/VertexEvent.java    |   12 +-
 .../runtime/execution/ExecutionAttempt.java     |  100 --
 .../runtime/execution/ExecutionListener.java    |    2 +-
 .../runtime/execution/ExecutionObserver.java    |    2 +-
 .../flink/runtime/execution/ExecutionState.java |   60 +-
 .../runtime/execution/ExecutionState2.java      |   38 -
 .../librarycache/LibraryCacheManager.java       |   12 +-
 .../executiongraph/AllVerticesIterator.java     |    6 +-
 .../flink/runtime/executiongraph/Execution.java |  606 ++++++++
 .../executiongraph/ExecutionAttempt.java        |  111 --
 .../runtime/executiongraph/ExecutionEdge.java   |   74 +
 .../runtime/executiongraph/ExecutionEdge2.java  |   74 -
 .../runtime/executiongraph/ExecutionGraph.java  | 1387 +++---------------
 .../executiongraph/ExecutionJobVertex.java      |   95 +-
 .../runtime/executiongraph/ExecutionState.java  |   34 -
 .../runtime/executiongraph/ExecutionVertex.java | 1230 ++++------------
 .../executiongraph/ExecutionVertex2.java        |  710 ---------
 .../IntermediateResultPartition.java            |   22 +-
 .../flink/runtime/instance/AllocatedSlot.java   |   27 +-
 .../apache/flink/runtime/instance/Instance.java |   26 +-
 .../instance/InstanceConnectionInfo.java        |   13 +-
 .../runtime/io/network/ChannelManager.java      |    4 +
 .../runtime/io/network/RemoteReceiver.java      |   20 +-
 .../runtime/io/network/api/MutableReader.java   |   10 -
 .../concurrent/SolutionSetUpdateBarrier.java    |    4 +-
 .../SolutionSetUpdateBarrierBroker.java         |    6 +-
 .../runtime/jobmanager/EventCollector.java      |    4 +-
 .../flink/runtime/jobmanager/JobManager.java    |   10 +-
 .../jobmanager/scheduler/DefaultScheduler.java  |   23 +-
 .../jobmanager/scheduler/ScheduledUnit.java     |   22 +-
 .../scheduler/SlotSharingGroupAssignment.java   |    9 +-
 .../jobmanager/web/JobmanagerInfoServlet.java   |   61 +-
 .../runtime/jobmanager/web/JsonFactory.java     |   18 +-
 .../hash/AbstractMutableHashTable.java          |    9 +-
 .../operators/sort/UnilateralSortMerger.java    |   23 +-
 .../profiling/impl/EnvironmentListenerImpl.java |    4 +-
 .../profiling/impl/JobProfilingData.java        |   10 +-
 .../runtime/protocols/JobManagerProtocol.java   |   13 +-
 .../protocols/TaskOperationProtocol.java        |    3 +-
 .../apache/flink/runtime/taskmanager/Task.java  |   61 +-
 .../runtime/taskmanager/TaskExecutionState.java |   88 +-
 .../flink/runtime/taskmanager/TaskManager.java  |   25 +-
 .../taskmanager/TaskOperationResult.java        |   62 +-
 .../flink/runtime/client/JobResultTest.java     |    1 -
 .../flink/runtime/event/job/EventsTest.java     |   20 +-
 .../flink/runtime/event/task/TaskEventTest.java |    2 -
 .../ExecutionGraphConstructionTest.java         |   81 +-
 .../ExecutionGraphDeploymentTest.java           |  150 +-
 .../executiongraph/ExecutionGraphTestUtils.java |   71 +-
 .../ExecutionStateProgressTest.java             |   79 +
 .../ExecutionVertexCancelTest.java              |  221 ++-
 .../ExecutionVertexDeploymentTest.java          |  203 ++-
 .../ExecutionVertexSchedulingTest.java          |  141 ++
 .../executiongraph/PointwisePatternTest.java    |   30 +-
 .../apache/flink/runtime/fs/LineReaderTest.java |    2 -
 .../runtime/instance/AllocatedSlotTest.java     |    8 +-
 .../instance/LocalInstanceManagerTest.java      |    5 +-
 .../io/disk/iomanager/IOManagerITCase.java      |    6 +-
 .../runtime/jobmanager/JobManagerITCase.java    |    5 +-
 .../runtime/jobmanager/JobManagerTest.java      |   23 -
 .../scheduler/SchedulerIsolatedTasksTest.java   |   32 +-
 .../scheduler/SchedulerTestUtils.java           |   50 +-
 .../runtime/operators/hash/HashTableITCase.java |    3 +-
 .../CombiningUnilateralSortMergerITCase.java    |    3 +-
 .../taskmanager/TaskExecutionStateTest.java     |   33 +-
 .../runtime/taskmanager/TaskManagerTest.java    |  139 +-
 .../flink/runtime/taskmanager/TaskTest.java     |   50 +-
 71 files changed, 2681 insertions(+), 3857 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
index fd4b375..047b9af 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
@@ -30,17 +30,29 @@ import java.io.StringWriter;
 public class ExceptionUtils {
 
 	/**
-	 * Makes a string representation of the exception's stack trace.
+	 * Makes a string representation of the exception's stack trace, or "(null)", if the
+	 * exception is null.
+	 * 
+	 * This method makes a best effort and never fails.
 	 * 
 	 * @param e The exception to stringify.
 	 * @return A string with exception name and call stack.
 	 */
 	public static String stringifyException(final Throwable e) {
-		final StringWriter stm = new StringWriter();
-		final PrintWriter wrt = new PrintWriter(stm);
-		e.printStackTrace(wrt);
-		wrt.close();
-		return stm.toString();
+		if (e == null) {
+			return "(null)";
+		}
+		
+		try {
+			StringWriter stm = new StringWriter();
+			PrintWriter wrt = new PrintWriter(stm);
+			e.printStackTrace(wrt);
+			wrt.close();
+			return stm.toString();
+		}
+		catch (Throwable t) {
+			return e.getClass().getName() + " (error while printing stack trace)";
+		}
 	}
 	
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
index b4a38f2..6cd18c3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
@@ -23,7 +23,7 @@ import java.io.IOException;
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.ExecutionEdge2;
+import org.apache.flink.runtime.executiongraph.ExecutionEdge;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 
 /**
@@ -95,7 +95,7 @@ public final class ChannelDeploymentDescriptor implements IOReadableWritable {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public static ChannelDeploymentDescriptor fromExecutionEdge(ExecutionEdge2 edge) {
+	public static ChannelDeploymentDescriptor fromExecutionEdge(ExecutionEdge edge) {
 		return new ChannelDeploymentDescriptor(edge.getOutputChannelId(), edge.getInputChannelId());
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
index e4a447f..71a19d3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
@@ -25,7 +25,7 @@ import java.util.List;
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.ExecutionEdge2;
+import org.apache.flink.runtime.executiongraph.ExecutionEdge;
 
 /**
  * A gate deployment descriptor contains the deployment descriptors for the channels associated with that gate.
@@ -83,17 +83,17 @@ public final class GateDeploymentDescriptor implements IOReadableWritable {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public static GateDeploymentDescriptor fromEdges(List<ExecutionEdge2> edges) {
+	public static GateDeploymentDescriptor fromEdges(List<ExecutionEdge> edges) {
 		List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(edges.size());
-		for (ExecutionEdge2 edge : edges) {
+		for (ExecutionEdge edge : edges) {
 			channels.add(ChannelDeploymentDescriptor.fromExecutionEdge(edge));
 		}
 		return new GateDeploymentDescriptor(channels);
 	}
 	
-	public static GateDeploymentDescriptor fromEdges(ExecutionEdge2[] edges) {
+	public static GateDeploymentDescriptor fromEdges(ExecutionEdge[] edges) {
 		List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(edges.length);
-		for (ExecutionEdge2 edge : edges) {
+		for (ExecutionEdge edge : edges) {
 			channels.add(ChannelDeploymentDescriptor.fromExecutionEdge(edge));
 		}
 		return new GateDeploymentDescriptor(channels);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
index f233d49..15d2fe6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
@@ -41,7 +41,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 	
 	private ExecutionAttemptID executionAttemptId;
 
-	private ExecutionState2 newExecutionState;
+	private ExecutionState newExecutionState;
 
 	/**
 	 * Constructs a new vertex event object.
@@ -54,7 +54,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 	 *        the new execution state of the vertex this event refers to
 	 */
 	public ExecutionStateChangeEvent(long timestamp, JobVertexID vertexId, int subtask,
-			ExecutionAttemptID executionAttemptId, ExecutionState2 newExecutionState)
+			ExecutionAttemptID executionAttemptId, ExecutionState newExecutionState)
 	{
 		super(timestamp);
 		
@@ -79,7 +79,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 
 		this.vertexId = new JobVertexID();
 		this.executionAttemptId = new ExecutionAttemptID();
-		this.newExecutionState = ExecutionState2.CREATED;
+		this.newExecutionState = ExecutionState.CREATED;
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -106,7 +106,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 	 * 
 	 * @return the new execution state of the vertex this event refers to
 	 */
-	public ExecutionState2 getNewExecutionState() {
+	public ExecutionState getNewExecutionState() {
 		return this.newExecutionState;
 	}
 
@@ -118,7 +118,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 		this.vertexId.read(in);
 		this.executionAttemptId.read(in);
 		this.subtask = in.readInt();
-		this.newExecutionState = ExecutionState2.values()[in.readInt()];
+		this.newExecutionState = ExecutionState.values()[in.readInt()];
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
index c697aa4..a935ba9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
@@ -22,7 +22,7 @@ import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.util.StringUtils;
@@ -53,7 +53,7 @@ public class VertexEvent extends AbstractEvent {
 	private ExecutionAttemptID executionAttemptId;
 
 	/** The current execution state of the subtask this event belongs to. */
-	private ExecutionState2 currentExecutionState;
+	private ExecutionState currentExecutionState;
 
 	/** An optional more detailed description of the event. */
 	private String description;
@@ -78,7 +78,7 @@ public class VertexEvent extends AbstractEvent {
 	 */
 	public VertexEvent(long timestamp, JobVertexID jobVertexID, String jobVertexName,
 			int totalNumberOfSubtasks, int indexOfSubtask, ExecutionAttemptID executionAttemptId,
-			ExecutionState2 currentExecutionState, String description)
+			ExecutionState currentExecutionState, String description)
 	{
 		super(timestamp);
 		
@@ -106,7 +106,7 @@ public class VertexEvent extends AbstractEvent {
 		this.totalNumberOfSubtasks = -1;
 		this.indexOfSubtask = -1;
 		this.executionAttemptId = new ExecutionAttemptID();
-		this.currentExecutionState = ExecutionState2.CREATED;
+		this.currentExecutionState = ExecutionState.CREATED;
 	}
 
 	/**
@@ -152,7 +152,7 @@ public class VertexEvent extends AbstractEvent {
 	 * 
 	 * @return the current execution state of the subtask this event belongs to
 	 */
-	public ExecutionState2 getCurrentExecutionState() {
+	public ExecutionState getCurrentExecutionState() {
 		return currentExecutionState;
 	}
 
@@ -181,7 +181,7 @@ public class VertexEvent extends AbstractEvent {
 		this.executionAttemptId.read(in);
 		this.totalNumberOfSubtasks = in.readInt();
 		this.indexOfSubtask = in.readInt();
-		this.currentExecutionState = ExecutionState2.values()[in.readInt()];
+		this.currentExecutionState = ExecutionState.values()[in.readInt()];
 		this.jobVertexName = StringUtils.readNullableString(in);
 		this.description = StringUtils.readNullableString(in);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java
deleted file mode 100644
index 9b39851..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/**
- * 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.execution;
-
-import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-
-/**
- * An attempt to execute a task for a {@link ExecutionVertex2}.
- */
-public class ExecutionAttempt implements java.io.Serializable {
-	
-	private static final long serialVersionUID = 1L;
-	
-
-	private final JobVertexID vertexId;
-	
-	private final int subtaskIndex;
-	
-	private final ExecutionAttemptID executionId;
-	
-	private final int attempt;
-
-	// --------------------------------------------------------------------------------------------
-	
-	public ExecutionAttempt(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, int attempt) {
-		if (vertexId == null || executionId == null || subtaskIndex < 0 || attempt < 1) {
-			throw new IllegalArgumentException();
-		}
-		
-		this.vertexId = vertexId;
-		this.subtaskIndex = subtaskIndex;
-		this.executionId = executionId;
-		this.attempt = attempt;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public JobVertexID getVertexId() {
-		return vertexId;
-	}
-	
-	public int getSubtaskIndex() {
-		return subtaskIndex;
-	}
-	
-	public ExecutionAttemptID getExecutionId() {
-		return executionId;
-	}
-	
-	public int getAttempt() {
-		return attempt;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public int hashCode() {
-		return vertexId.hashCode() +
-				executionId.hashCode() +
-				31 * subtaskIndex +
-				17 * attempt;
-	}
-	
-	@Override
-	public boolean equals(Object obj) {
-		if (obj instanceof ExecutionAttempt) {
-			ExecutionAttempt other = (ExecutionAttempt) obj;
-			return this.executionId.equals(other.executionId) &&
-					this.vertexId.equals(other.vertexId) &&
-					this.subtaskIndex == other.subtaskIndex &&
-					this.attempt == other.attempt;
-		} else {
-			return false;
-		}
-	}
-	
-	@Override
-	public String toString() {
-		return String.format("ExecutionAttempt (vertex=%s, subtask=%d, executionAttemptId=%s, attempt=%d)",
-				vertexId, subtaskIndex, executionId, attempt);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
index b08c847..74f94cf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
@@ -29,5 +29,5 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 public interface ExecutionListener {
 
 	void executionStateChanged(JobID jobID, JobVertexID vertexId, int subtask, ExecutionAttemptID executionId,
-			ExecutionState2 newExecutionState, String optionalMessage);
+			ExecutionState newExecutionState, String optionalMessage);
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
index 20a6180..78ed4d9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
@@ -28,7 +28,7 @@ public interface ExecutionObserver {
 	 * @param optionalMessage
 	 *        an optional message providing further information on the state change
 	 */
-	void executionStateChanged(ExecutionState2 newExecutionState, String optionalMessage);
+	void executionStateChanged(ExecutionState newExecutionState, String optionalMessage);
 
 	/**
 	 * Returns whether the task has been canceled.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
index 36a8672..6ad936e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState.java
@@ -16,67 +16,23 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.execution;
 
-/**
- * This enumerations includes all possible states during a task's lifetime.
- * 
- */
 public enum ExecutionState {
 
-	/**
-	 * The task has been created, but is not yet submitted to a scheduler.
-	 */
 	CREATED,
-
-	/**
-	 * The task has been accepted by the scheduler, the resource for the task has been requested
-	 */
+	
 	SCHEDULED,
-
-	/**
-	 * The task has been assigned a resource to run, but is not yet read to by deployed.
-	 */
-	ASSIGNED,
-
-	/**
-	 * The task has been announced ready to run by the scheduler, but is not yet running.
-	 */
-	READY,
-
-	/**
-	 * The task is currently deployed to the assigned to task manager.
-	 */
-	STARTING,
-
-	/**
-	 * The task is currently running.
-	 */
+	
+	DEPLOYING,
+	
 	RUNNING,
-
-	/**
-	 * The task has already finished, but not all of its results have been consumed yet.
-	 */
-	FINISHING,
-
-	/**
-	 * The task finished, all of its results have been consumed.
-	 */
+	
 	FINISHED,
-
-	/**
-	 * The task has been requested to be canceled, but is not yet terminated.
-	 */
+	
 	CANCELING,
-
-	/**
-	 * The task has been canceled due to a user request or the error of a connected task.
-	 */
+	
 	CANCELED,
-
-	/**
-	 * The task has been aborted due to a failure during execution.
-	 */
+	
 	FAILED
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java
deleted file mode 100644
index c2b2070..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * 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.execution;
-
-public enum ExecutionState2 {
-
-	CREATED,
-	
-	SCHEDULED,
-	
-	DEPLOYING,
-	
-	RUNNING,
-	
-	FINISHED,
-	
-	CANCELING,
-	
-	CANCELED,
-	
-	FAILED
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
index 1076ede..cbcd368 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
@@ -364,12 +364,14 @@ public final class LibraryCacheManager {
 
 		// Use spin lock here
 		while (this.lockMap.putIfAbsent(id, LOCK_OBJECT) != null);
-
-		if (decrementReferenceCounter(id) == 0) {
-			this.libraryManagerEntries.remove(id);
+		try {
+			if (decrementReferenceCounter(id) == 0) {
+				this.libraryManagerEntries.remove(id);
+			}
+		}
+		finally {
+			this.lockMap.remove(id);
 		}
-
-		this.lockMap.remove(id);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java
index 84781cb..4bf36f2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java
@@ -21,11 +21,11 @@ package org.apache.flink.runtime.executiongraph;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
-class AllVerticesIterator implements Iterator<ExecutionVertex2> {
+class AllVerticesIterator implements Iterator<ExecutionVertex> {
 
 	private final Iterator<ExecutionJobVertex> jobVertices;
 	
-	private ExecutionVertex2[] currVertices;
+	private ExecutionVertex[] currVertices;
 	
 	private int currPos;
 	
@@ -56,7 +56,7 @@ class AllVerticesIterator implements Iterator<ExecutionVertex2> {
 	}
 	
 	@Override
-	public ExecutionVertex2 next() {
+	public ExecutionVertex next() {
 		if (hasNext()) {
 			return currVertices[currPos++];
 		} else {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
new file mode 100644
index 0000000..8cfc7fd
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -0,0 +1,606 @@
+/**
+ * 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 static org.apache.flink.runtime.execution.ExecutionState.*;
+
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
+import org.apache.flink.util.ExceptionUtils;
+import org.slf4j.Logger;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A single execution of a vertex. While an {@link ExecutionVertex} can be executed multiple times (for recovery,
+ * or other re-computation), this class tracks the state of a single execution of that vertex and the resources.
+ * 
+ * NOTE ABOUT THE DESIGN RATIONAL:
+ * 
+ * In several points of the code, we need to deal with possible concurrent state changes and actions.
+ * For example, while the call to deploy a task (send it to the TaskManager) happens, the task gets cancelled.
+ * 
+ * We could lock the entire portion of the code (decision to deploy, deploy, set state to running) such that
+ * it is guaranteed that any "cancel command" will only pick up after deployment is done and that the "cancel
+ * command" call will never overtake the deploying call.
+ * 
+ * This blocks the threads big time, because the remote calls may take long. Depending of their locking behavior, it
+ * may even result in distributed deadlocks (unless carefully avoided). We therefore use atomic state updates and
+ * occasional double-checking to ensure that the state after a completed call is as expected, and trigger correcting
+ * actions if it is not. Many actions are also idempotent (like canceling).
+ */
+public class Execution {
+
+	private static final AtomicReferenceFieldUpdater<Execution, ExecutionState> STATE_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(Execution.class, ExecutionState.class, "state");
+	
+	private static final Logger LOG = ExecutionGraph.LOG;
+	
+	private static final int NUM_CANCEL_CALL_TRIES = 3;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private final ExecutionVertex vertex;
+	
+	private final ExecutionAttemptID attemptId;
+	
+	private final long[] stateTimestamps;
+	
+	private final int attemptNumber;
+	
+	
+	private volatile ExecutionState state = CREATED;
+	
+	private volatile AllocatedSlot assignedResource;  // once assigned, never changes
+	
+	private volatile Throwable failureCause;          // once assigned, never changes
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public Execution(ExecutionVertex vertex, int attemptNumber, long startTimestamp) {
+		Preconditions.checkNotNull(vertex);
+		Preconditions.checkArgument(attemptNumber >= 0);
+		
+		this.vertex = vertex;
+		this.attemptId = new ExecutionAttemptID();
+		this.attemptNumber = attemptNumber;
+		
+		this.stateTimestamps = new long[ExecutionState.values().length];
+		markTimestamp(ExecutionState.CREATED, startTimestamp);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//   Properties
+	// --------------------------------------------------------------------------------------------
+	
+	public ExecutionVertex getVertex() {
+		return vertex;
+	}
+	
+	public ExecutionAttemptID getAttemptId() {
+		return attemptId;
+	}
+
+	public int getAttemptNumber() {
+		return attemptNumber;
+	}
+	
+	public ExecutionState getState() {
+		return state;
+	}
+	
+	public AllocatedSlot getAssignedResource() {
+		return assignedResource;
+	}
+	
+	public Throwable getFailureCause() {
+		return failureCause;
+	}
+	
+	public long getStateTimestamp(ExecutionState state) {
+		return this.stateTimestamps[state.ordinal()];
+	}
+	
+	public boolean isFinished() {
+		return state == FINISHED || state == FAILED || state == CANCELED;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Actions
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * NOTE: This method only throws exceptions if it is in an illegal state to be scheduled, or if the tasks needs
+	 *       to be scheduled immediately and no resource is available. If the task is accepted by the schedule, any
+	 *       error sets the vertex state to failed and triggers the recovery logic.
+	 * 
+	 * @param scheduler
+	 * 
+	 * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only state that permits scheduling.
+	 * @throws NoResourceAvailableException Thrown is no queued scheduling is allowed and no resources are currently available.
+	 */
+	public void scheduleForExecution(DefaultScheduler scheduler, boolean queued) throws NoResourceAvailableException {
+		if (scheduler == null) {
+			throw new NullPointerException();
+		}
+		
+		if (transitionState(CREATED, SCHEDULED)) {
+			
+			// record that we were scheduled
+			vertex.notifyStateTransition(attemptId, SCHEDULED, null);
+			
+			ScheduledUnit toSchedule = new ScheduledUnit(this, vertex.getJobVertex().getSlotSharingGroup());
+		
+			// IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned
+			//     in all cases where the deployment failed. we use many try {} finally {} clauses to assure that
+			if (queued) {
+				SlotAllocationFuture future = scheduler.scheduleQueued(toSchedule);
+				
+				future.setFutureAction(new SlotAllocationFutureAction() {
+					@Override
+					public void slotAllocated(AllocatedSlot slot) {
+						try {
+							deployToSlot(slot);
+						}
+						catch (Throwable t) {
+							try {
+								slot.releaseSlot();
+							} finally {
+								markFailed(t);
+							}
+						}
+					}
+				});
+			}
+			else {
+				AllocatedSlot slot = scheduler.scheduleImmediately(toSchedule);
+				try {
+					deployToSlot(slot);
+				}
+				catch (Throwable t) {
+					try {
+						slot.releaseSlot();
+					} finally {
+						markFailed(t);
+					}
+				}
+			}
+		}
+		else if (this.state == CANCELED) {
+			// this can occur very rarely through heavy races. if the task was canceled, we do not
+			// schedule it
+			return;
+		}
+		else {
+			throw new IllegalStateException("The vertex must be in CREATED state to be scheduled.");
+		}
+	}
+	
+	public void deployToSlot(final AllocatedSlot slot) throws JobException {
+		// sanity checks
+		if (slot == null) {
+			throw new NullPointerException();
+		}
+		if (!slot.isAlive()) {
+			throw new JobException("Traget slot for deployment is not alive.");
+		}
+		
+		// make sure exactly one deployment call happens from the correct state
+		// note: the transition from CREATED to DEPLOYING is for testing purposes only
+		ExecutionState previous = this.state;
+		if (previous == SCHEDULED || previous == CREATED) {
+			if (!transitionState(previous, DEPLOYING)) {
+				// race condition, someone else beat us to the deploying call.
+				// this should actually not happen and indicates a race somewhere else
+				throw new IllegalStateException("Cannot deploy task: Concurrent deployment call race.");
+			}
+			
+			vertex.notifyStateTransition(attemptId, DEPLOYING, null);
+		}
+		else {
+			// vertex may have been cancelled, or it was already scheduled
+			throw new IllegalStateException("The vertex must be in CREATED or SCHEDULED state to be deployed. Found state " + previous);
+		}
+		
+		try {
+			// good, we are allowed to deploy
+			if (!slot.setExecutedVertex(this)) {
+				throw new JobException("Could not assign the ExecutionVertex to the slot " + slot);
+			}
+			this.assignedResource = slot;
+			
+			final TaskDeploymentDescriptor deployment = vertex.createDeploymentDescriptor(attemptId, slot);
+			
+			// register this execution at the execution graph, to receive callbacks
+			vertex.getExecutionGraph().registerExecution(this);
+			
+			// we execute the actual deploy call in a concurrent action to prevent this call from blocking for long
+			Runnable deployaction = new Runnable() {
+	
+				@Override
+				public void run() {
+					try {
+						Instance instance = slot.getInstance();
+						instance.checkLibraryAvailability(vertex.getJobId());
+						
+						TaskOperationResult result = instance.getTaskManagerProxy().submitTask(deployment);
+						if (result == null) {
+							markFailed(new Exception("Failed to deploy the task to slot " + slot + ": TaskOperationResult was null"));
+						}
+						else if (!result.getExecutionId().equals(attemptId)) {
+							markFailed(new Exception("Answer execution id does not match the request execution id."));
+						}
+						else if (result.isSuccess()) {
+							switchToRunning();
+						}
+						else {
+							// deployment failed :(
+							markFailed(new Exception("Failed to deploy the task " + getVertexWithAttempt() + " to slot " + slot + ": " + result.getDescription()));
+						}
+					}
+					catch (Throwable t) {
+						// some error occurred. fail the task
+						markFailed(t);
+					}
+				}
+			};
+			
+			vertex.execute(deployaction);
+		}
+		catch (Throwable t) {
+			markFailed(t);
+			ExceptionUtils.rethrow(t);
+		}
+	}
+	
+	
+	public void cancel() {
+		// depending on the previous state, we go directly to cancelled (no cancel call necessary)
+		// -- or to canceling (cancel call needs to be sent to the task manager)
+		
+		// because of several possibly previous states, we need to again loop until we make a
+		// successful atomic state transition
+		while (true) {
+			
+			ExecutionState current = this.state;
+			
+			if (current == CANCELING || current == CANCELED) {
+				// already taken care of, no need to cancel again
+				return;
+			}
+				
+			// these two are the common cases where we need to send a cancel call
+			else if (current == RUNNING || current == DEPLOYING) {
+				// try to transition to canceling, if successful, send the cancel call
+				if (transitionState(current, CANCELING)) {
+					vertex.notifyStateTransition(attemptId, CANCELING, null);
+					sendCancelRpcCall();
+					return;
+				}
+				// else: fall through the loop
+			}
+			
+			else if (current == FINISHED || current == FAILED) {
+				// nothing to do any more. finished failed before it could be cancelled.
+				// in any case, the task is removed from the TaskManager already
+				return;
+			}
+			else if (current == CREATED || current == SCHEDULED) {
+				// from here, we can directly switch to cancelled, because the no task has been deployed
+				if (transitionState(current, CANCELED)) {
+					
+					// we skip the canceling state. set the timestamp, for a consistent appearance
+					markTimestamp(CANCELING, getStateTimestamp(CANCELED));
+					vertex.notifyStateTransition(attemptId, CANCELED, null);
+					return;
+				}
+				// else: fall through the loop
+			}
+			else {
+				throw new IllegalStateException(current.name());
+			}
+		}
+	}
+	
+	/**
+	 * This method fails the vertex due to an external condition. The task will move to state FAILED.
+	 * If the task was in state RUNNING or DEPLOYING before, it will send a cancel call to the TaskManager.
+	 * 
+	 * @param t The exception that caused the task to fail.
+	 */
+	public void fail(Throwable t) {
+		if (processFail(t, false)) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Task " + getVertexWithAttempt() + " was failed.", t);
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//   Callbacks
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * This method marks the task as failed, but will make no attempt to remove task execution from the task manager.
+	 * It is intended for cases where the task is known not to be running, or then the TaskManager reports failure
+	 * (in which case it has already removed the task).
+	 * 
+	 * @param t The exception that caused the task to fail.
+	 */
+	void markFailed(Throwable t) {
+		// the call returns true if it actually made the state transition (was not already failed before, etc)
+		if (processFail(t, true)) {
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Task " + getVertexWithAttempt() + " failed.", t);
+			}
+		}
+	}
+	
+	void markFinished() {
+		// this call usually comes during RUNNING, but may also come while still in deploying (very fast tasks!)
+		while (true) {
+			ExecutionState current = this.state;
+			
+			if (current == RUNNING || current == DEPLOYING) {
+			
+				if (transitionState(current, FINISHED)) {
+					try {
+						vertex.notifyStateTransition(attemptId, FINISHED, null);
+						vertex.executionFinished();
+						return;
+					}
+					finally {
+						vertex.getExecutionGraph().deregisterExecution(this);
+						assignedResource.releaseSlot();
+					}
+				}
+			}
+			else {
+				if (current == CANCELED || current == CANCELING || current == FAILED) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Task FINISHED, but concurrently went to state " + state);
+					}
+					return;
+				}
+				else {
+					// this should not happen, we need to fail this
+					markFailed(new Exception("Vertex received FINISHED message while being in state " + state));
+					return;
+				}
+			}
+		}
+	}
+	
+	void cancelingComplete() {
+		if (transitionState(CANCELING, CANCELED)) {
+			try {
+				vertex.executionCanceled();
+				vertex.notifyStateTransition(attemptId, CANCELED, null);
+			}
+			finally {
+				vertex.getExecutionGraph().deregisterExecution(this);
+				assignedResource.releaseSlot();
+			}
+		}
+		else {
+			ExecutionState actualState = this.state;
+			// failing in the meantime may happen and is no problem.
+			// anything else is a serious problem !!!
+			if (actualState != FAILED) {
+				String message = String.format("Asynchronous race: Found state %s after successful cancel call.", state);
+				LOG.error(message);
+				vertex.getExecutionGraph().fail(new Exception(message));
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Internal Actions
+	// --------------------------------------------------------------------------------------------
+	
+	private boolean processFail(Throwable t, boolean isCallback) {
+		
+		// damn, we failed. This means only that we keep our books and notify our parent JobExecutionVertex
+		// the actual computation on the task manager is cleaned up by the TaskManager that noticed the failure
+		
+		// we may need to loop multiple times (in the presence of concurrent calls) in order to
+		// atomically switch to failed 
+		while (true) {
+			ExecutionState current = this.state;
+			
+			if (current == FAILED) {
+				// already failed. It is enough to remember once that we failed (its sad enough)
+				return false;
+			}
+			
+			if (current == CANCELED || (current == CANCELING && isCallback)) {
+				// we are already aborting or are already aborted
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Ignoring transition of vertex %s to %s while being %s", 
+							getVertexWithAttempt(), FAILED, current));
+				}
+				return false;
+			}
+			
+			if (transitionState(current, FAILED)) {
+				// success (in a manner of speaking)
+				
+				if (!isCallback && (current == RUNNING || current == DEPLOYING)) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Sending out cancel request, to remove task execution from TaskManager.");
+					}
+					
+					try {
+						if (assignedResource != null) {
+							sendCancelRpcCall();
+						}
+					} catch (Throwable tt) {
+						// no reason this should ever happen, but log it to be safe
+						LOG.error("Error triggering cancel call while marking task as failed.", tt);
+					}
+				}
+				
+				try {
+					this.failureCause = t;
+					vertex.executionFailed(t);
+					vertex.notifyStateTransition(attemptId, FAILED, t);
+				}
+				finally {
+					if (assignedResource != null) {
+						assignedResource.releaseSlot();
+					}
+					vertex.getExecutionGraph().deregisterExecution(this);
+				}
+				
+				// leave the loop
+				return true;
+			}
+		}
+	}
+	
+	private void switchToRunning() {
+		
+		// transition state, the common case
+		if (transitionState(DEPLOYING, RUNNING)) {
+			vertex.notifyStateTransition(attemptId, RUNNING, null);
+		}
+		else {
+			// something happened while the call was in progress.
+			// it can mean:
+			//  - canceling, while deployment was in progress. state is now canceling, or canceled, if the response overtook
+			//  - finishing (execution and finished call overtook the deployment answer, which is possible and happens for fast tasks)
+			//  - failed (execution, failure, and failure message overtook the deployment answer)
+			
+			ExecutionState currentState = this.state;
+			
+			if (currentState == FINISHED || currentState == CANCELED) {
+				// do nothing, this is nice, the task was really fast
+			}
+			
+			if (currentState == CANCELING || currentState == FAILED) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Concurrent canceling/failing of %s while deployment was in progress.", getVertexWithAttempt()));
+				}
+				sendCancelRpcCall();
+			}
+			else {
+				String message = String.format("Concurrent unexpected state transition of task %s to %s while deployment was in progress.",
+						getVertexWithAttempt(), currentState);
+				
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(message);
+				}
+				
+				// undo the deployment
+				sendCancelRpcCall();
+				
+				// record the failure
+				markFailed(new Exception(message));
+			}
+		}
+	}
+	
+	private void sendCancelRpcCall() {
+		final AllocatedSlot slot = this.assignedResource;
+		if (slot == null) {
+			throw new IllegalStateException("Cannot cancel when task was not running or deployed.");
+		}
+		
+		Runnable cancelAction = new Runnable() {
+			
+			@Override
+			public void run() {
+				Throwable exception = null;
+				
+				for (int triesLeft = NUM_CANCEL_CALL_TRIES; triesLeft > 0; --triesLeft) {
+					
+					try {
+						// send the call. it may be that the task is not really there (asynchronous / overtaking messages)
+						// in which case it is fine (the deployer catches it)
+						TaskOperationResult result = slot.getInstance().getTaskManagerProxy().cancelTask(attemptId);
+						
+						if (!result.isSuccess()) {
+							// the task was not found, which may be when the task concurrently finishes or fails, or
+							// when the cancel call overtakes the deployment call
+							if (LOG.isDebugEnabled()) {
+								LOG.debug("Cancel task call did not find task. Probably RPC call race.");
+							}
+						}
+						
+						// in any case, we need not call multiple times, so we quit
+						return;
+					}
+					catch (Throwable t) {
+						if (exception == null) {
+							exception = t;
+						}
+						LOG.error("Canceling vertex " + getVertexWithAttempt() + " failed (" + triesLeft + " tries left): " + t.getMessage() , t);
+					}
+				}
+				
+				// dang, utterly unsuccessful - the target node must be down, in which case the tasks are lost anyways
+				fail(new Exception("Task could not be canceled.", exception));
+			}
+		};
+		
+		vertex.execute(cancelAction);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Miscellaneous
+	// --------------------------------------------------------------------------------------------
+	
+	private boolean transitionState(ExecutionState currentState, ExecutionState targetState) {
+		if (STATE_UPDATER.compareAndSet(this, currentState, targetState)) {
+			markTimestamp(targetState);
+			return true;
+		} else {
+			return false;
+		}
+	}
+	
+	private void markTimestamp(ExecutionState state) {
+		markTimestamp(state, System.currentTimeMillis());
+	}
+	
+	private void markTimestamp(ExecutionState state, long timestamp) {
+		this.stateTimestamps[state.ordinal()] = timestamp;
+	}
+	
+	public String getVertexWithAttempt() {
+		return vertex.getSimpleName() + " - execution #" + attemptNumber;
+	}
+	
+	@Override
+	public String toString() {
+		return String.format("Attempt #%d (%s) @ %s - [%s]", attemptNumber, vertex.getSimpleName(),
+				(assignedResource == null ? "(unassigned)" : assignedResource.toString()), state);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java
deleted file mode 100644
index b623d6f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/**
- * 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 java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.flink.runtime.instance.AllocatedSlot;
-
-public class ExecutionAttempt {
-
-	private final AtomicBoolean finished = new AtomicBoolean();
-	
-	private final ExecutionAttemptID attemptId;
-	
-	private final AllocatedSlot assignedResource;
-	
-	private final int attemptNumber;
-	
-	private final long startTimestamp;
-	
-	private volatile long endTimestamp;
-	
-	private volatile Throwable failureCause;
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public ExecutionAttemptID getAttemptId() {
-		return attemptId;
-	}
-	
-	public ExecutionAttempt(ExecutionAttemptID attemptId, AllocatedSlot assignedResource, int attemptNumber, long startTimestamp) {
-		this.attemptId = attemptId;
-		this.assignedResource = assignedResource;
-		this.attemptNumber = attemptNumber;
-		this.startTimestamp = startTimestamp;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-
-	public AllocatedSlot getAssignedResource() {
-		return assignedResource;
-	}
-	
-	public int getAttemptNumber() {
-		return attemptNumber;
-	}
-	
-	public long getStartTimestamp() {
-		return startTimestamp;
-	}
-	
-	public long getEndTimestamp() {
-		return endTimestamp;
-	}
-	
-	public Throwable getFailureCause() {
-		return failureCause;
-	}
-	
-	public boolean isFinished() {
-		return finished.get();
-	}
-	
-	public boolean isFailed() {
-		return finished.get() && failureCause != null;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public boolean finish() {
-		if (finished.compareAndSet(false, true)) {
-			endTimestamp = System.currentTimeMillis();
-			return true;
-		} else {
-			return false;
-		}
-	}
-	
-	public boolean fail(Throwable error) {
-		if (finished.compareAndSet(false, true)) {
-			failureCause = error;
-			return true;
-		} else {
-			return false;
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public String toString() {
-		return String.format("Attempt #%d (%s) @ %s - started %d %s", attemptNumber, attemptId,
-				assignedResource.toString(), startTimestamp, isFinished() ? "finished " + endTimestamp : "[RUNNING]");
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java
new file mode 100644
index 0000000..f001b6f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge.java
@@ -0,0 +1,74 @@
+/**
+ * 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 org.apache.flink.runtime.io.network.channels.ChannelID;
+
+public class ExecutionEdge {
+
+	private final IntermediateResultPartition source;
+	
+	private final ExecutionVertex target;
+	
+	private final int inputNum;
+
+	private final ChannelID inputChannelId;
+	
+	private final ChannelID outputChannelId;
+	
+	
+	public ExecutionEdge(IntermediateResultPartition source, ExecutionVertex target, int inputNum) {
+		this.source = source;
+		this.target = target;
+		this.inputNum = inputNum;
+		
+		this.inputChannelId = new ChannelID();
+		this.outputChannelId = new ChannelID();
+	}
+	
+	public ExecutionEdge(IntermediateResultPartition source, ExecutionVertex target, int inputNum, ChannelID inputChannelId, ChannelID outputChannelId) {
+		this.source = source;
+		this.target = target;
+		this.inputNum = inputNum;
+		
+		this.inputChannelId = inputChannelId;
+		this.outputChannelId = outputChannelId;
+	}
+	
+	
+	public IntermediateResultPartition getSource() {
+		return source;
+	}
+	
+	public ExecutionVertex getTarget() {
+		return target;
+	}
+	
+	public int getInputNum() {
+		return inputNum;
+	}
+	
+	public ChannelID getInputChannelId() {
+		return inputChannelId;
+	}
+	
+	public ChannelID getOutputChannelId() {
+		return outputChannelId;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java
deleted file mode 100644
index a7cbeaf..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * 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 org.apache.flink.runtime.io.network.channels.ChannelID;
-
-public class ExecutionEdge2 {
-
-	private final IntermediateResultPartition source;
-	
-	private final ExecutionVertex2 target;
-	
-	private final int inputNum;
-
-	private final ChannelID inputChannelId;
-	
-	private final ChannelID outputChannelId;
-	
-	
-	public ExecutionEdge2(IntermediateResultPartition source, ExecutionVertex2 target, int inputNum) {
-		this.source = source;
-		this.target = target;
-		this.inputNum = inputNum;
-		
-		this.inputChannelId = new ChannelID();
-		this.outputChannelId = new ChannelID();
-	}
-	
-	public ExecutionEdge2(IntermediateResultPartition source, ExecutionVertex2 target, int inputNum, ChannelID inputChannelId, ChannelID outputChannelId) {
-		this.source = source;
-		this.target = target;
-		this.inputNum = inputNum;
-		
-		this.inputChannelId = inputChannelId;
-		this.outputChannelId = outputChannelId;
-	}
-	
-	
-	public IntermediateResultPartition getSource() {
-		return source;
-	}
-	
-	public ExecutionVertex2 getTarget() {
-		return target;
-	}
-	
-	public int getInputNum() {
-		return inputNum;
-	}
-	
-	public ChannelID getInputChannelId() {
-		return inputChannelId;
-	}
-	
-	public ChannelID getOutputChannelId() {
-		return outputChannelId;
-	}
-}


[53/63] [abbrv] git commit: Port streaming package to new JobGraph API and adjust all runtime-level tests

Posted by se...@apache.org.
Port streaming package to new JobGraph API and adjust all runtime-level tests


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/5d13ddb7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/5d13ddb7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/5d13ddb7

Branch: refs/heads/master
Commit: 5d13ddb7f61870f6ce70cfaeb394c65aa0f8b8fd
Parents: f229d5b
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 00:02:19 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:20:58 2014 +0200

----------------------------------------------------------------------
 .../flink/streaming/api/JobGraphBuilder.java    | 109 +++------
 .../plantranslate/NepheleJobGraphGenerator.java |  29 ++-
 .../apache/flink/core/fs/FileInputSplit.java    |   2 +-
 .../example/java/graph/ConnectedComponents.java |   4 +-
 .../runtime/jobgraph/AbstractJobVertex.java     |   2 +-
 .../runtime/jobmanager/JobManagerITCase.java    |  24 +-
 .../src/test/resources/logback-test.xml         |   1 +
 .../broadcastvars/BroadcastBranchingITCase.java |   1 -
 .../KMeansIterativeNepheleITCase.java           |  29 ++-
 .../test/cancelling/CancellingTestBase.java     |   5 +-
 .../test/cancelling/MapCancelingITCase.java     |   1 -
 .../ConnectedComponentsNepheleITCase.java       | 223 ++++++++++---------
 .../IterationWithChainingNepheleITCase.java     |  43 ++--
 .../test/iterative/nephele/JobGraphUtils.java   |   4 +-
 .../CustomCompensatableDanglingPageRank.java    |  46 ++--
 ...mpensatableDanglingPageRankWithCombiner.java |  40 ++--
 .../CompensatableDanglingPageRank.java          |  43 ++--
 .../test/recordJobs/kmeans/KMeansBroadcast.java |   2 -
 flink-tests/src/test/resources/logback-test.xml |   9 +-
 19 files changed, 304 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
index e6c5042..837265e 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/JobGraphBuilder.java
@@ -24,15 +24,11 @@ import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.streaming.api.collector.OutputSelector;
 import org.apache.flink.streaming.api.invokable.SinkInvokable;
 import org.apache.flink.streaming.api.invokable.SourceInvokable;
@@ -87,9 +83,6 @@ public class JobGraphBuilder {
 	private int degreeOfParallelism;
 	private int executionParallelism;
 
-	private String maxParallelismVertexName;
-	private int maxParallelism;
-
 	/**
 	 * Creates an new {@link JobGraph} with the given name. A JobGraph is a DAG
 	 * and consists of sources, tasks (intermediate vertices) and sinks. A
@@ -127,8 +120,6 @@ public class JobGraphBuilder {
 		iterationTailCount = new HashMap<String, Integer>();
 		iterationWaitTime = new HashMap<String, Long>();
 
-		maxParallelismVertexName = "";
-		maxParallelism = 0;
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("JobGraph created");
 		}
@@ -303,8 +294,6 @@ public class JobGraphBuilder {
 	 *            ID of iteration for mulitple iterations
 	 * @param parallelism
 	 *            Number of parallel instances created
-	 * @param directName
-	 *            Id of the output direction
 	 * @param waitTime
 	 *            Max waiting time for next record
 	 */
@@ -332,8 +321,6 @@ public class JobGraphBuilder {
 	 *            Name of the component
 	 * @param componentClass
 	 *            The class of the vertex
-	 * @param typeWrapper
-	 *            Wrapper of the types for serialization
 	 * @param invokableObject
 	 *            The user defined invokable object
 	 * @param operatorName
@@ -389,22 +376,12 @@ public class JobGraphBuilder {
 		byte[] outputSelector = outputSelectors.get(componentName);
 
 		// Create vertex object
-		AbstractJobVertex component = null;
-		if (componentClass.equals(StreamSource.class)
-				|| componentClass.equals(StreamIterationSource.class)) {
-			component = new JobInputVertex(componentName, this.jobGraph);
-		} else if (componentClass.equals(StreamTask.class)
-				|| componentClass.equals(CoStreamTask.class)) {
-			component = new JobTaskVertex(componentName, this.jobGraph);
-		} else if (componentClass.equals(StreamSink.class)
-				|| componentClass.equals(StreamIterationSink.class)) {
-			component = new JobOutputVertex(componentName, this.jobGraph);
-		} else {
-			throw new RuntimeException("Unsupported component class");
-		}
+		AbstractJobVertex component = new AbstractJobVertex(componentName);
+
+		this.jobGraph.addVertex(component);
 
 		component.setInvokableClass(componentClass);
-		component.setNumberOfSubtasks(parallelism);
+		component.setParallelism(parallelism);
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("Parallelism set: {} for {}", parallelism, componentName);
 		}
@@ -432,11 +409,6 @@ public class JobGraphBuilder {
 		}
 
 		components.put(componentName, component);
-
-		if (parallelism > maxParallelism) {
-			maxParallelism = parallelism;
-			maxParallelismVertexName = componentName;
-		}
 	}
 
 	/**
@@ -504,26 +476,18 @@ public class JobGraphBuilder {
 
 		StreamConfig config = new StreamConfig(upStreamComponent.getConfiguration());
 
-		try {
-			if (partitionerObject.getClass().equals(ForwardPartitioner.class)) {
-				upStreamComponent.connectTo(downStreamComponent, ChannelType.NETWORK,
-						DistributionPattern.POINTWISE);
-			} else {
-				upStreamComponent.connectTo(downStreamComponent, ChannelType.NETWORK,
-						DistributionPattern.BIPARTITE);
-			}
-
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("CONNECTED: {} - {} -> {}", partitionerObject.getClass().getSimpleName(),
-						upStreamComponentName, downStreamComponentName);
-			}
-
-		} catch (JobGraphDefinitionException e) {
-			throw new RuntimeException("Cannot connect components: " + upStreamComponentName
-					+ " to " + downStreamComponentName, e);
+		if (partitionerObject.getClass().equals(ForwardPartitioner.class)) {
+			downStreamComponent.connectNewDataSetAsInput(upStreamComponent, DistributionPattern.POINTWISE);
+		} else {
+			downStreamComponent.connectNewDataSetAsInput(upStreamComponent, DistributionPattern.BIPARTITE);
 		}
 
-		int outputIndex = upStreamComponent.getNumberOfForwardConnections() - 1;
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("CONNECTED: {} - {} -> {}", partitionerObject.getClass().getSimpleName(),
+					upStreamComponentName, downStreamComponentName);
+		}
+		
+		int outputIndex = upStreamComponent.getNumberOfProducedIntermediateDataSets() - 1;
 
 		config.setOutputName(outputIndex, outEdgeNames.get(upStreamComponentName).get(outputIndex));
 		config.setSelectAll(outputIndex,
@@ -595,34 +559,31 @@ public class JobGraphBuilder {
 		return typeWrapperOut1.get(id).getTypeInfo();
 	}
 
-	/**
-	 * Sets instance sharing between the given components
-	 * 
-	 * @param component1
-	 *            Share will be called on this component
-	 * @param component2
-	 *            Share will be called to this component
-	 */
-	public void setInstanceSharing(String component1, String component2) {
-		AbstractJobVertex c1 = components.get(component1);
-		AbstractJobVertex c2 = components.get(component2);
-
-		c1.setVertexToShareInstancesWith(c2);
-	}
+//  TODO: This should be adjusted to the sharing groups
+//	/**
+//	 * Sets instance sharing between the given components
+//	 * 
+//	 * @param component1
+//	 *            Share will be called on this component
+//	 * @param component2
+//	 *            Share will be called to this component
+//	 */
+//	public void setInstanceSharing(String component1, String component2) {
+//		AbstractJobVertex c1 = components.get(component1);
+//		AbstractJobVertex c2 = components.get(component2);
+//
+//		c1.setVertexToShareInstancesWith(c2);
+//	}
 
 	/**
 	 * Sets all components to share with the one with highest parallelism
 	 */
 	private void setAutomaticInstanceSharing() {
+		SlotSharingGroup shareGroup = new SlotSharingGroup();
 
-		AbstractJobVertex maxParallelismVertex = components.get(maxParallelismVertexName);
-
-		for (String componentName : components.keySet()) {
-			if (!componentName.equals(maxParallelismVertexName)) {
-				components.get(componentName).setVertexToShareInstancesWith(maxParallelismVertex);
-			}
+		for (AbstractJobVertex vertex : components.values()) {
+			vertex.setSlotSharingGroup(shareGroup);
 		}
-
 	}
 
 	/**
@@ -631,7 +592,7 @@ public class JobGraphBuilder {
 	private void setNumberOfJobInputs() {
 		for (AbstractJobVertex component : components.values()) {
 			(new StreamConfig(component.getConfiguration())).setNumberOfInputs(component
-					.getNumberOfBackwardConnections());
+					.getNumberOfInputs());
 		}
 	}
 
@@ -642,7 +603,7 @@ public class JobGraphBuilder {
 	private void setNumberOfJobOutputs() {
 		for (AbstractJobVertex component : components.values()) {
 			(new StreamConfig(component.getConfiguration())).setNumberOfOutputs(component
-					.getNumberOfForwardConnections());
+					.getNumberOfProducedIntermediateDataSets());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index 39647d2..a3fef17 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -123,6 +123,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	
 	private IterationPlanNode currentIteration;	// hack: as long as no nesting is possible, remember the enclosing iteration
 	
+	private SlotSharingGroup sharingGroup;
 	
 	// ------------------------------------------------------------------------
 
@@ -157,6 +158,8 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		this.auxVertices = new ArrayList<AbstractJobVertex>();
 		this.iterations = new HashMap<IterationPlanNode, IterationDescriptor>();
 		
+		this.sharingGroup = new SlotSharingGroup();
+		
 		// generate Nephele job graph
 		program.accept(this);
 		
@@ -183,13 +186,9 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		JobGraph graph = new JobGraph(program.getJobName());
 		graph.setAllowQueuedScheduling(false);
 		
-		// all vertices share the same slot sharing group, for now
-		SlotSharingGroup sharingGroup = new SlotSharingGroup();
-		
 		// add vertices to the graph
 		for (AbstractJobVertex vertex : this.vertices.values()) {
 			graph.addVertex(vertex);
-			vertex.setSlotSharingGroup(sharingGroup);
 		}
 		
 		for (AbstractJobVertex vertex : this.auxVertices) {
@@ -346,6 +345,8 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			int pd = node.getDegreeOfParallelism();
 			vertex.setParallelism(pd);
 			
+			vertex.setSlotSharingGroup(sharingGroup);
+			
 			// check whether this vertex is part of an iteration step function
 			if (this.currentIteration != null) {
 				// check that the task has the same DOP as the iteration as such
@@ -357,10 +358,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				// store the id of the iterations the step functions participate in
 				IterationDescriptor descr = this.iterations.get(this.currentIteration);
 				new TaskConfig(vertex.getConfiguration()).setIterationId(descr.getId());
-				
-				// make sure tasks inside iterations are co-located with the head
-				AbstractJobVertex headVertex = this.iterations.get(this.currentIteration).getHeadTask();
-				vertex.setStrictlyCoLocatedWith(headVertex);
 			}
 	
 			// store in the map
@@ -417,14 +414,15 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				return;
 			}
 			
+			final AbstractJobVertex targetVertex = this.vertices.get(node);
+			
+			
 			// --------- Main Path: Translation of channels ----------
 			// 
 			// There are two paths of translation: One for chained tasks (or merged tasks in general),
 			// which do not have their own task vertex. The other for tasks that have their own vertex,
 			// or are the primary task in a vertex (to which the others are chained).
 			
-			final AbstractJobVertex targetVertex = this.vertices.get(node);
-			
 			// check whether this node has its own task, or is merged with another one
 			if (targetVertex == null) {
 				// node's task is merged with another task. it is either chained, of a merged head vertex
@@ -492,6 +490,17 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			
 			// -------- Here, we translate non-chained tasks -------------
 			
+			
+			if (this.currentIteration != null) {
+				AbstractJobVertex head = this.iterations.get(this.currentIteration).getHeadTask();
+				if (head == null) {
+					throw new CompilerException("Found no iteration head task in the postVisit of translating a task inside an iteration");
+				}
+				
+				targetVertex.setStrictlyCoLocatedWith(head);
+			}
+			
+			
 			// create the config that will contain all the description of the inputs
 			final TaskConfig targetVertexConfig = new TaskConfig(targetVertex.getConfiguration());
 						

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
index ebee5d0..aae472e 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
@@ -165,6 +165,6 @@ public class FileInputSplit extends LocatableInputSplit {
 	
 	@Override
 	public String toString() {
-		return '[' + getSplitNumber() + "] " + file + ":" + start + "+" + length;
+		return "[" + getSplitNumber() + "] " + file + ":" + start + "+" + length;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
----------------------------------------------------------------------
diff --git a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
index 6a75a7b..f0ea7dc 100644
--- a/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
+++ b/flink-examples/flink-java-examples/src/main/java/org/apache/flink/example/java/graph/ConnectedComponents.java
@@ -116,9 +116,7 @@ public class ConnectedComponents implements ProgramDescription {
 		}
 		
 		// execute program
-//		env.execute("Connected Components Example");
-		
-		System.out.println(env.getExecutionPlan());
+		env.execute("Connected Components Example");
 	}
 	
 	// *************************************************************************

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index 899210f..dbe3f72 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -276,7 +276,7 @@ public class AbstractJobVertex implements java.io.Serializable {
 	 */
 	public void setStrictlyCoLocatedWith(AbstractJobVertex strictlyCoLocatedWith) {
 		if (this.slotSharingGroup == null || this.slotSharingGroup != strictlyCoLocatedWith.slotSharingGroup) {
-			throw new IllegalArgumentException();
+			throw new IllegalArgumentException("Strict co-location requires that both vertices are in the same slot sharing group.");
 		}
 		
 		CoLocationGroup thisGroup = this.coLocationGroup;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index 44d1c11..70b3ad9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -100,7 +100,7 @@ public class JobManagerITCase {
 					
 					assertTrue("The job did not finish successfully.", success);
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -177,7 +177,7 @@ public class JobManagerITCase {
 					
 					assertTrue("The job did not finish successfully.", success);
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -234,7 +234,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -296,7 +296,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -358,7 +358,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -424,7 +424,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -490,7 +490,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -554,7 +554,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -617,8 +617,6 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
-					
-					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -681,7 +679,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -748,7 +746,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -815,7 +813,7 @@ public class JobManagerITCase {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
 					
-					assertEquals(0, eg.getRegisteredExecutions().size());
+//					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-runtime/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/logback-test.xml b/flink-runtime/src/test/resources/logback-test.xml
index f817d4d..565c360 100644
--- a/flink-runtime/src/test/resources/logback-test.xml
+++ b/flink-runtime/src/test/resources/logback-test.xml
@@ -38,4 +38,5 @@
     <logger name="org.apache.flink.runtime.taskmanager.TaskManager" level="OFF"/>
     <logger name="org.apache.flink.runtime.executiongraph.ExecutionGraph" level="OFF"/>
     <logger name="org.apache.flink.runtime.jobmanager.EventCollector" level="OFF"/>
+    <logger name="org.apache.flink.runtime.instance.InstanceManager" level="OFF"/>
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
index e9873ec..3c94150 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastBranchingITCase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.broadcastvars;
 
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
index a31539f..edc6467 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.GroupReduceDriver;
@@ -221,7 +222,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	private static AbstractJobVertex createReducer(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> inputSerializer,
 			TypeComparatorFactory<?> inputComparator, TypeSerializerFactory<?> outputSerializer)
 	{
-		// ---------------- the tail (co group) --------------------
+		// ---------------- the tail (reduce) --------------------
 		
 		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "Reduce / Iteration Tail", jobGraph,
 			numSubTasks);
@@ -248,7 +249,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		tailConfig.setOutputSerializer(outputSerializer);
 		
 		// the udf
-		tailConfig.setStubWrapper(new UserCodeObjectWrapper<RecomputeClusterCenter>(new RecomputeClusterCenter()));
+		tailConfig.setStubWrapper(new UserCodeObjectWrapper<WrappingReduceFunction>(new WrappingReduceFunction(new RecomputeClusterCenter())));
 		
 		return tail;
 	}
@@ -283,7 +284,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		
 		AbstractJobVertex reducer = createReducer(jobGraph, numSubTasks, serializer, int0Comparator, serializer);
 		
-		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
+		AbstractJobVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		
 		AbstractJobVertex sync = createSync(jobGraph, numIterations, numSubTasks);
 		
@@ -310,13 +311,21 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
 
 		// -- instance sharing -------------------------------------------------------------------------------------
-		points.setVertexToShareInstancesWith(output);
-		centers.setVertexToShareInstancesWith(output);
-		head.setVertexToShareInstancesWith(output);
-		mapper.setVertexToShareInstancesWith(output);
-		reducer.setVertexToShareInstancesWith(output);
-		fakeTailOutput.setVertexToShareInstancesWith(output);
-		sync.setVertexToShareInstancesWith(output);
+		
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		
+		points.setSlotSharingGroup(sharingGroup);
+		centers.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		mapper.setSlotSharingGroup(sharingGroup);
+		reducer.setSlotSharingGroup(sharingGroup);
+		fakeTailOutput.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		
+		mapper.setStrictlyCoLocatedWith(head);
+		reducer.setStrictlyCoLocatedWith(head);
+		fakeTailOutput.setStrictlyCoLocatedWith(reducer);
 
 		return jobGraph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
index 8bf74c0..8129b3c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
@@ -197,9 +197,10 @@ public abstract class CancellingTestBase {
 							exitLoop = true;
 							break;
 						case RUNNING:
+						case CANCELLING:
+						case FAILING:
+						case CREATED:
 							break;
-						default:
-							throw new Exception("Bug: Unrecognized Job Status.");
 						}
 					}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
index 1946d25..e8c394e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.cancelling;
 
 //import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
index dad2370..8cf2c69 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
@@ -43,13 +43,12 @@ import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCri
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
@@ -175,8 +174,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 	// -----------------------------------------------------------------------------------------------------------------
 
 	private static InputFormatVertex createVerticesInput(JobGraph jobGraph, String verticesPath, int numSubTasks,
-			TypeSerializerFactory<?> serializer,
-			TypeComparatorFactory<?> comparator) {
+			TypeSerializerFactory<?> serializer, TypeComparatorFactory<?> comparator)
+	{
 		@SuppressWarnings("unchecked")
 		CsvInputFormat verticesInFormat = new CsvInputFormat(' ', LongValue.class);
 		InputFormatVertex verticesInput = JobGraphUtils.createInput(verticesInFormat, verticesPath, "VerticesInput",
@@ -205,13 +204,13 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return verticesInput;
 	}
 
-	private static InputFormatInputVertex createEdgesInput(JobGraph jobGraph, String edgesPath, int numSubTasks,
-			TypeSerializerFactory<?> serializer,
-			TypeComparatorFactory<?> comparator) {
+	private static InputFormatVertex createEdgesInput(JobGraph jobGraph, String edgesPath, int numSubTasks,
+			TypeSerializerFactory<?> serializer, TypeComparatorFactory<?> comparator)
+	{
 		// edges
 		@SuppressWarnings("unchecked")
 		CsvInputFormat edgesInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class);
-		InputFormatInputVertex edgesInput = JobGraphUtils.createInput(edgesInFormat, edgesPath, "EdgesInput", jobGraph,
+		InputFormatVertex edgesInput = JobGraphUtils.createInput(edgesInFormat, edgesPath, "EdgesInput", jobGraph,
 			numSubTasks);
 		TaskConfig edgesInputConfig = new TaskConfig(edgesInput.getConfiguration());
 		{
@@ -223,13 +222,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return edgesInput;
 	}
 
-	private static JobTaskVertex createIterationHead(JobGraph jobGraph, int numSubTasks,
+	private static AbstractJobVertex createIterationHead(JobGraph jobGraph, int numSubTasks,
 			TypeSerializerFactory<?> serializer,
 			TypeComparatorFactory<?> comparator,
 			TypePairComparatorFactory<?, ?> pairComparator) {
 
-		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Join With Edges (Iteration Head)",
-			jobGraph, numSubTasks);
+		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Join With Edges (Iteration Head)", jobGraph, numSubTasks);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		{
 			headConfig.setIterationId(ITERATION_ID);
@@ -295,12 +293,11 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return head;
 	}
 
-	private static JobTaskVertex createIterationIntermediate(JobGraph jobGraph, int numSubTasks,
-			TypeSerializerFactory<?> serializer,
-			TypeComparatorFactory<?> comparator) {
-
+	private static AbstractJobVertex createIterationIntermediate(JobGraph jobGraph, int numSubTasks,
+			TypeSerializerFactory<?> serializer, TypeComparatorFactory<?> comparator)
+	{
 		// --------------- the intermediate (reduce to min id) ---------------
-		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
+		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
 			"Find Min Component-ID", jobGraph, numSubTasks);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		{
@@ -352,14 +349,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return output;
 	}
 
-	private static OutputFormatVertex createFakeTail(JobGraph jobGraph, int numSubTasks) {
-		OutputFormatVertex fakeTailOutput =
-			JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
-		return fakeTailOutput;
+	private static AbstractJobVertex createFakeTail(JobGraph jobGraph, int numSubTasks) {
+		return JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 	}
 
-	private static OutputFormatVertex createSync(JobGraph jobGraph, int numSubTasks, int maxIterations) {
-		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
+	private static AbstractJobVertex createSync(JobGraph jobGraph, int numSubTasks, int maxIterations) {
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(maxIterations);
 		syncConfig.setIterationId(ITERATION_ID);
@@ -377,7 +372,6 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	public JobGraph createJobGraphUnifiedTails(
 			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-			throws JobGraphDefinitionException
 	{
 		// -- init -------------------------------------------------------------------------------------------------
 		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
@@ -391,18 +385,17 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		// -- invariant vertices -----------------------------------------------------------------------------------
 		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
 		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
-		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
+		AbstractJobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
 
-		JobTaskVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
+		AbstractJobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		OutputFormatVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		AbstractJobVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		AbstractJobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// --------------- the tail (solution set join) ---------------
-		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
-			numSubTasks);
+		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph, numSubTasks);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		{
 			tailConfig.setIterationId(ITERATION_ID);
@@ -446,22 +439,25 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
-		vertices.setVertexToShareInstancesWith(head);
-		edges.setVertexToShareInstancesWith(head);
-
-		intermediate.setVertexToShareInstancesWith(head);
-		tail.setVertexToShareInstancesWith(head);
-
-		output.setVertexToShareInstancesWith(head);
-		sync.setVertexToShareInstancesWith(head);
-		fakeTail.setVertexToShareInstancesWith(tail);
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		vertices.setSlotSharingGroup(sharingGroup);
+		edges.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		intermediate.setSlotSharingGroup(sharingGroup);
+		tail.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		fakeTail.setSlotSharingGroup(sharingGroup);
+		
+		intermediate.setStrictlyCoLocatedWith(head);
+		tail.setStrictlyCoLocatedWith(head);
+		fakeTail.setStrictlyCoLocatedWith(tail);
 
 		return jobGraph;
 	}
 
 	public JobGraph createJobGraphSeparateTails(
 			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-		throws JobGraphDefinitionException
 	{
 		// -- init -------------------------------------------------------------------------------------------------
 		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
@@ -477,22 +473,22 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
-		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
+		AbstractJobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setWaitForSolutionSetUpdate();
 
 		// intermediate
-		JobTaskVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
+		AbstractJobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
 		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		OutputFormatVertex ssFakeTail = createFakeTail(jobGraph, numSubTasks);
-		OutputFormatVertex wsFakeTail = createFakeTail(jobGraph, numSubTasks);
-		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		AbstractJobVertex ssFakeTail = createFakeTail(jobGraph, numSubTasks);
+		AbstractJobVertex wsFakeTail = createFakeTail(jobGraph, numSubTasks);
+		AbstractJobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ss join) ----------------------
-		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
+		AbstractJobVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
 			"Solution Set Join", jobGraph, numSubTasks);
 		TaskConfig ssJoinIntermediateConfig = new TaskConfig(ssJoinIntermediate.getConfiguration());
 		{
@@ -521,7 +517,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		}
 
 		// -------------------------- ss tail --------------------------------
-		JobTaskVertex ssTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail",
+		AbstractJobVertex ssTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail",
 			jobGraph, numSubTasks);
 		TaskConfig ssTailConfig = new TaskConfig(ssTail.getConfiguration());
 		{
@@ -546,7 +542,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		}
 
 		// -------------------------- ws tail --------------------------------
-		JobTaskVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail",
+		AbstractJobVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail",
 			jobGraph, numSubTasks);
 		TaskConfig wsTailConfig = new TaskConfig(wsTail.getConfiguration());
 		{
@@ -593,27 +589,32 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
-		vertices.setVertexToShareInstancesWith(head);
-		edges.setVertexToShareInstancesWith(head);
-
-		intermediate.setVertexToShareInstancesWith(head);
-
-		ssJoinIntermediate.setVertexToShareInstancesWith(head);
-		wsTail.setVertexToShareInstancesWith(head);
-
-		output.setVertexToShareInstancesWith(head);
-		sync.setVertexToShareInstancesWith(head);
-
-		ssTail.setVertexToShareInstancesWith(wsTail);
-		ssFakeTail.setVertexToShareInstancesWith(ssTail);
-		wsFakeTail.setVertexToShareInstancesWith(wsTail);
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		vertices.setSlotSharingGroup(sharingGroup);
+		edges.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		intermediate.setSlotSharingGroup(sharingGroup);
+		ssJoinIntermediate.setSlotSharingGroup(sharingGroup);
+		wsTail.setSlotSharingGroup(sharingGroup);
+		ssTail.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		wsFakeTail.setSlotSharingGroup(sharingGroup);
+		ssFakeTail.setSlotSharingGroup(sharingGroup);
+		
+		intermediate.setStrictlyCoLocatedWith(head);
+		ssJoinIntermediate.setStrictlyCoLocatedWith(head);
+		wsTail.setStrictlyCoLocatedWith(head);
+		ssTail.setStrictlyCoLocatedWith(head);
+		wsFakeTail.setStrictlyCoLocatedWith(wsTail);
+		ssFakeTail.setStrictlyCoLocatedWith(ssTail);
 
 		return jobGraph;
 	}
 
 	public JobGraph createJobGraphIntermediateWorksetUpdateAndSolutionSetTail(
 			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-			throws JobGraphDefinitionException {
+	{
 		// -- init -------------------------------------------------------------------------------------------------
 		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
 		@SuppressWarnings("unchecked")
@@ -628,23 +629,22 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
-		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
+		AbstractJobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setWaitForSolutionSetUpdate();
 
 		// intermediate
-		JobTaskVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
+		AbstractJobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		OutputFormatVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		AbstractJobVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		AbstractJobVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		AbstractJobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ws update) ----------------------
-		JobTaskVertex wsUpdateIntermediate =
-			JobGraphUtils.createTask(IterationIntermediatePactTask.class, "WorksetUpdate", jobGraph,
-				numSubTasks);
+		AbstractJobVertex wsUpdateIntermediate = 
+			JobGraphUtils.createTask(IterationIntermediatePactTask.class, "WorksetUpdate", jobGraph, numSubTasks);
 		TaskConfig wsUpdateConfig = new TaskConfig(wsUpdateIntermediate.getConfiguration());
 		{
 			wsUpdateConfig.setIterationId(ITERATION_ID);
@@ -672,9 +672,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		}
 
 		// -------------------------- ss tail --------------------------------
-		JobTaskVertex ssTail =
-			JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail", jobGraph,
-				numSubTasks);
+		AbstractJobVertex ssTail =
+			JobGraphUtils.createTask(IterationTailPactTask.class, "IterationSolutionSetTail", jobGraph, numSubTasks);
 		TaskConfig ssTailConfig = new TaskConfig(ssTail.getConfiguration());
 		{
 			ssTailConfig.setIterationId(ITERATION_ID);
@@ -717,18 +716,21 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
-		vertices.setVertexToShareInstancesWith(head);
-		edges.setVertexToShareInstancesWith(head);
-
-		intermediate.setVertexToShareInstancesWith(head);
-
-		wsUpdateIntermediate.setVertexToShareInstancesWith(head);
-		ssTail.setVertexToShareInstancesWith(head);
-
-		output.setVertexToShareInstancesWith(head);
-		sync.setVertexToShareInstancesWith(head);
-
-		fakeTail.setVertexToShareInstancesWith(ssTail);
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		vertices.setSlotSharingGroup(sharingGroup);
+		edges.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		intermediate.setSlotSharingGroup(sharingGroup);
+		wsUpdateIntermediate.setSlotSharingGroup(sharingGroup);
+		ssTail.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		fakeTail.setSlotSharingGroup(sharingGroup);
+
+		intermediate.setStrictlyCoLocatedWith(head);
+		wsUpdateIntermediate.setStrictlyCoLocatedWith(head);
+		ssTail.setStrictlyCoLocatedWith(head);
+		fakeTail.setStrictlyCoLocatedWith(ssTail);
 
 		return jobGraph;
 	}
@@ -739,7 +741,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 	public JobGraph createJobGraphSolutionSetUpdateAndWorksetTail(
 			String verticesPath, String edgesPath, String resultPath, int numSubTasks, int maxIterations)
-			throws JobGraphDefinitionException {
+	{
 		// -- init -------------------------------------------------------------------------------------------------
 		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
 		@SuppressWarnings("unchecked")
@@ -754,19 +756,19 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
-		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
+		AbstractJobVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
 
 		// intermediate
-		JobTaskVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
+		AbstractJobVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		OutputFormatVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		AbstractJobVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		AbstractJobVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		AbstractJobVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ss update) ----------------------
-		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
+		AbstractJobVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
 			"Solution Set Update", jobGraph, numSubTasks);
 		TaskConfig ssJoinIntermediateConfig = new TaskConfig(ssJoinIntermediate.getConfiguration());
 		{
@@ -794,8 +796,7 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		}
 
 		// -------------------------- ws tail --------------------------------
-		JobTaskVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail",
-			jobGraph, numSubTasks);
+		AbstractJobVertex wsTail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationWorksetTail", jobGraph, numSubTasks);
 		TaskConfig wsTailConfig = new TaskConfig(wsTail.getConfiguration());
 		{
 			wsTailConfig.setIterationId(ITERATION_ID);
@@ -837,18 +838,22 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
-		vertices.setVertexToShareInstancesWith(head);
-		edges.setVertexToShareInstancesWith(head);
-
-		intermediate.setVertexToShareInstancesWith(head);
-
-		ssJoinIntermediate.setVertexToShareInstancesWith(head);
-		wsTail.setVertexToShareInstancesWith(head);
-
-		output.setVertexToShareInstancesWith(head);
-		sync.setVertexToShareInstancesWith(head);
-
-		fakeTail.setVertexToShareInstancesWith(wsTail);
+		
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		vertices.setSlotSharingGroup(sharingGroup);
+		edges.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		intermediate.setSlotSharingGroup(sharingGroup);
+		ssJoinIntermediate.setSlotSharingGroup(sharingGroup);
+		wsTail.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		fakeTail.setSlotSharingGroup(sharingGroup);
+
+		intermediate.setStrictlyCoLocatedWith(head);
+		ssJoinIntermediate.setStrictlyCoLocatedWith(head);
+		wsTail.setStrictlyCoLocatedWith(head);
+		fakeTail.setStrictlyCoLocatedWith(wsTail);
 
 		return jobGraph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
index 65c9857..aa939ed 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
@@ -32,13 +32,12 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.GroupReduceDriver;
@@ -113,8 +112,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		return getTestJobGraph(dataPath, resultPath, numSubTasks, maxIterations);
 	}
 
-	private JobGraph getTestJobGraph(String inputPath, String outputPath, int numSubTasks, int maxIterations)
-			throws JobGraphDefinitionException {
+	private JobGraph getTestJobGraph(String inputPath, String outputPath, int numSubTasks, int maxIterations) {
 
 		final JobGraph jobGraph = new JobGraph("Iteration Tail with Chaining");
 
@@ -140,8 +138,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - head ------------------------------------------------------------------------------------------------------
-		JobTaskVertex head = JobGraphUtils.createTask(
-			IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks);
+		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		{
 			headConfig.setIterationId(ITERATION_ID);
@@ -176,8 +173,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - tail ------------------------------------------------------------------------------------------------------
-		JobTaskVertex tail = JobGraphUtils.createTask(
-			IterationTailPactTask.class, "Chained Iteration Tail", jobGraph, numSubTasks);
+		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "Chained Iteration Tail", jobGraph, numSubTasks);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		{
 			tailConfig.setIterationId(ITERATION_ID);
@@ -225,10 +221,10 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - fake tail -------------------------------------------------------------------------------------------------
-		OutputFormatVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks);
+		AbstractJobVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks);
 
 		// - sync ------------------------------------------------------------------------------------------------------
-		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(maxIterations);
 		syncConfig.setIterationId(ITERATION_ID);
@@ -250,15 +246,18 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		// --------------------------------------------------------------------------------------------------------------
 		// 3. INSTANCE SHARING
 		// --------------------------------------------------------------------------------------------------------------
-		input.setVertexToShareInstancesWith(head);
-
-		tail.setVertexToShareInstancesWith(head);
-
-		output.setVertexToShareInstancesWith(head);
-
-		sync.setVertexToShareInstancesWith(head);
-
-		fakeTail.setVertexToShareInstancesWith(tail);
+		
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		
+		input.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		tail.setSlotSharingGroup(sharingGroup);
+		fakeTail.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		
+		tail.setStrictlyCoLocatedWith(head);
+		fakeTail.setStrictlyCoLocatedWith(tail);
 
 		return jobGraph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
index 2b4b779..1734a15 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
@@ -110,8 +110,8 @@ public class JobGraphUtils {
 		return sync;
 	}
 
-	public static OutputFormatVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism) {
-		OutputFormatVertex outputVertex = new OutputFormatVertex(name);
+	public static AbstractJobVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism) {
+		AbstractJobVertex outputVertex = new AbstractJobVertex(name);
 		jobGraph.addVertex(outputVertex);
 		
 		outputVertex.setInvokableClass(FakeOutputTask.class);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
index a6771ba..662805e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
@@ -29,12 +29,12 @@ import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
@@ -138,7 +138,7 @@ public class CustomCompensatableDanglingPageRank {
 		// --------------- the inputs ---------------------
 
 		// page rank input
-		InputFormatInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
+		InputFormatVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
 			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -147,7 +147,7 @@ public class CustomCompensatableDanglingPageRank {
 		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
 
 		// edges as adjacency list
-		InputFormatInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
+		InputFormatVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
 			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -155,7 +155,7 @@ public class CustomCompensatableDanglingPageRank {
 		adjacencyListInputConfig.setOutputComparator(vertexWithAdjacencyListComparator, 0);
 
 		// --------------- the head ---------------------
-		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
+		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
 			degreeOfParallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
@@ -200,7 +200,7 @@ public class CustomCompensatableDanglingPageRank {
 
 		// --------------- the join ---------------------
 		
-		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
+		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
 			"IterationIntermediate", jobGraph, degreeOfParallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
@@ -228,12 +228,11 @@ public class CustomCompensatableDanglingPageRank {
 
 		// ---------------- the tail (co group) --------------------
 		
-		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
+		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
 			degreeOfParallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
-        tailConfig.setIsWorksetUpdate();
-		// TODO we need to combine!
+		tailConfig.setIsWorksetUpdate();
 		
 		// inputs and driver
 		tailConfig.setDriver(CoGroupDriver.class);
@@ -276,10 +275,9 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
-			degreeOfParallelism);
+		AbstractJobVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", degreeOfParallelism);
 
-		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
@@ -305,13 +303,19 @@ public class CustomCompensatableDanglingPageRank {
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		
-		fakeTailOutput.setVertexToShareInstancesWith(tail);
-		tail.setVertexToShareInstancesWith(head);
-		pageWithRankInput.setVertexToShareInstancesWith(head);
-		adjacencyListInput.setVertexToShareInstancesWith(head);
-		intermediate.setVertexToShareInstancesWith(head);
-		output.setVertexToShareInstancesWith(head);
-		sync.setVertexToShareInstancesWith(head);
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		pageWithRankInput.setSlotSharingGroup(sharingGroup);
+		adjacencyListInput.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		intermediate.setSlotSharingGroup(sharingGroup);
+		tail.setSlotSharingGroup(sharingGroup);
+		fakeTailOutput.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		
+		fakeTailOutput.setStrictlyCoLocatedWith(tail);
+		tail.setStrictlyCoLocatedWith(head);
+		intermediate.setStrictlyCoLocatedWith(head);
 
 		return jobGraph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
index 7eacf1b..072db21 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
@@ -29,12 +29,12 @@ import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
@@ -155,7 +155,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		adjacencyListInputConfig.setOutputComparator(vertexWithAdjacencyListComparator, 0);
 
 		// --------------- the head ---------------------
-		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
+		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
 			degreeOfParallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
@@ -200,7 +200,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		// --------------- the join ---------------------
 		
-		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
+		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
 			"IterationIntermediate", jobGraph, degreeOfParallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
@@ -240,11 +240,11 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		// ---------------- the tail (co group) --------------------
 		
-		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
+		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
 			degreeOfParallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
-        tailConfig.setIsWorksetUpdate();
+		tailConfig.setIsWorksetUpdate();
 		
 		// inputs and driver
 		tailConfig.setDriver(CoGroupDriver.class);
@@ -288,10 +288,10 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
+		AbstractJobVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
 			degreeOfParallelism);
 
-		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
@@ -317,13 +317,19 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		
-		fakeTailOutput.setVertexToShareInstancesWith(tail);
-		tail.setVertexToShareInstancesWith(head);
-		pageWithRankInput.setVertexToShareInstancesWith(head);
-		adjacencyListInput.setVertexToShareInstancesWith(head);
-		intermediate.setVertexToShareInstancesWith(head);
-		output.setVertexToShareInstancesWith(head);
-		sync.setVertexToShareInstancesWith(head);
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		pageWithRankInput.setSlotSharingGroup(sharingGroup);
+		adjacencyListInput.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		intermediate.setSlotSharingGroup(sharingGroup);
+		tail.setSlotSharingGroup(sharingGroup);
+		fakeTailOutput.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		
+		fakeTailOutput.setStrictlyCoLocatedWith(tail);
+		tail.setStrictlyCoLocatedWith(head);
+		intermediate.setStrictlyCoLocatedWith(head);
 
 		return jobGraph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
index 317963b..269378b 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
@@ -32,12 +32,12 @@ import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
@@ -136,8 +136,7 @@ public class CompensatableDanglingPageRank {
 		adjacencyListInputConfig.setOutputComparator(fieldZeroComparator, 0);
 
 		// --------------- the head ---------------------
-		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph,
-			degreeOfParallelism);
+		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "IterationHead", jobGraph, degreeOfParallelism);
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
 		
@@ -181,8 +180,7 @@ public class CompensatableDanglingPageRank {
 
 		// --------------- the join ---------------------
 		
-		JobTaskVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
-			"IterationIntermediate", jobGraph, degreeOfParallelism);
+		AbstractJobVertex intermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class, "IterationIntermediate", jobGraph, degreeOfParallelism);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 		intermediateConfig.setIterationId(ITERATION_ID);
 //		intermediateConfig.setDriver(RepeatableHashjoinMatchDriverWithCachedBuildside.class);
@@ -209,11 +207,11 @@ public class CompensatableDanglingPageRank {
 
 		// ---------------- the tail (co group) --------------------
 		
-		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
+		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
 			degreeOfParallelism);
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
 		tailConfig.setIterationId(ITERATION_ID);
-        tailConfig.setIsWorksetUpdate();
+		tailConfig.setIsWorksetUpdate();
 		// TODO we need to combine!
 		
 		// inputs and driver
@@ -257,10 +255,9 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
-			degreeOfParallelism);
+		AbstractJobVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", degreeOfParallelism);
 
-		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());
@@ -286,13 +283,19 @@ public class CompensatableDanglingPageRank {
 
 		JobGraphUtils.connect(head, sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 		
-		fakeTailOutput.setVertexToShareInstancesWith(tail);
-		tail.setVertexToShareInstancesWith(head);
-		pageWithRankInput.setVertexToShareInstancesWith(head);
-		adjacencyListInput.setVertexToShareInstancesWith(head);
-		intermediate.setVertexToShareInstancesWith(head);
-		output.setVertexToShareInstancesWith(head);
-		sync.setVertexToShareInstancesWith(head);
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		pageWithRankInput.setSlotSharingGroup(sharingGroup);
+		adjacencyListInput.setSlotSharingGroup(sharingGroup);
+		head.setSlotSharingGroup(sharingGroup);
+		intermediate.setSlotSharingGroup(sharingGroup);
+		tail.setSlotSharingGroup(sharingGroup);
+		fakeTailOutput.setSlotSharingGroup(sharingGroup);
+		output.setSlotSharingGroup(sharingGroup);
+		sync.setSlotSharingGroup(sharingGroup);
+		
+		fakeTailOutput.setStrictlyCoLocatedWith(tail);
+		tail.setStrictlyCoLocatedWith(head);
+		intermediate.setStrictlyCoLocatedWith(head);
 
 		return jobGraph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
index 66c8aae..99e5ee7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recordJobs/kmeans/KMeansBroadcast.java
@@ -16,10 +16,8 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.recordJobs.kmeans;
 
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/5d13ddb7/flink-tests/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/logback-test.xml b/flink-tests/src/test/resources/logback-test.xml
index ec37329..993441a 100644
--- a/flink-tests/src/test/resources/logback-test.xml
+++ b/flink-tests/src/test/resources/logback-test.xml
@@ -23,14 +23,15 @@
         </encoder>
     </appender>
 
-    <root level="INFO">
+    <root level="ERROR">
         <appender-ref ref="STDOUT"/>
     </root>
 
-<!-- 
-    <logger name="org.apache.flink.test.recordJobs.relational.query1Util.LineItemFilter" level="ERROR"/>
     <logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>
+    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>
     <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>
-    -->
+    <logger name="org.apache.flink.runtime.taskmanager.TaskManager" level="OFF"/>
+    <logger name="org.apache.flink.runtime.executiongraph.ExecutionGraph" level="OFF"/>
+    <logger name="org.apache.flink.runtime.jobmanager.EventCollector" level="OFF"/>
 </configuration>
\ No newline at end of file


[44/63] [abbrv] git commit: More graceful failing/errors/logging when canceling in early job stages

Posted by se...@apache.org.
More graceful failing/errors/logging when canceling in early job stages


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/ae57c7c0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/ae57c7c0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/ae57c7c0

Branch: refs/heads/master
Commit: ae57c7c03dafcbbf728947ee453d29bdf42ee6bc
Parents: 9187175
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 02:43:18 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../runtime/execution/RuntimeEnvironment.java   |  64 ++---
 .../runtime/executiongraph/ExecutionGraph.java  |   8 +-
 .../runtime/executiongraph/ExecutionVertex.java |  10 +-
 .../flink/runtime/jobmanager/JobManager.java    |  20 +-
 .../flink/runtime/taskmanager/TaskManager.java  |   5 +-
 .../jobmanager/ExceptionOutputFormat.java       |  55 ----
 .../flink/runtime/jobmanager/ExceptionTask.java |  70 -----
 .../runtime/jobmanager/JobManagerITCase.java    | 271 +++++++++++++++++--
 .../jobmanager/tasks/BlockingNoOpInvokable.java |  38 +++
 .../src/test/resources/logback-test.xml         |   1 +
 flink-runtime/src/test/resources/topology.txt   |  16 --
 11 files changed, 351 insertions(+), 207 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
index 79a4aaa..ade878f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
@@ -55,7 +55,6 @@ import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.protocols.AccumulatorProtocol;
 import org.apache.flink.runtime.taskmanager.Task;
-import org.apache.flink.util.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -67,9 +66,13 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	/** The log object used for debugging. */
 	private static final Logger LOG = LoggerFactory.getLogger(RuntimeEnvironment.class);
 
+	private static final ThreadGroup TASK_THREADS = new ThreadGroup("Task Threads");
+	
 	/** The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds). */
 	private static final int SLEEPINTERVAL = 100;
 	
+	
+	
 	// --------------------------------------------------------------------------------------------
 
 	/** The task that owns this environment */
@@ -235,33 +238,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			if (this.owner.isCanceled()) {
 				throw new CancelTaskException();
 			}
-		}
-		catch (Throwable t) {
 			
-			if (!this.owner.isCanceled()) {
-
-				// Perform clean up when the task failed and has been not canceled by the user
-				try {
-					this.invokable.cancel();
-				} catch (Throwable t2) {
-					LOG.error(StringUtils.stringifyException(t2));
-				}
-			}
-
-			// Release all resources that may currently be allocated by the individual channels
-			releaseAllChannelResources();
-
-			if (this.owner.isCanceled() || t instanceof CancelTaskException) {
-				this.owner.cancelingDone();
-			}
-			else {
-				this.owner.markFailed(t);
-			}
-
-			return;
-		}
-		
-		try {
 			// If there is any unclosed input gate, close it and propagate close operation to corresponding output gate
 			closeInputGates();
 
@@ -273,9 +250,28 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 			// Now we wait until all output channels have written out their data and are closed
 			waitForOutputChannelsToBeClosed();
+			
+			if (this.owner.isCanceled()) {
+				throw new CancelTaskException();
+			}
+			
+			// Finally, switch execution state to FINISHED and report to job manager
+			if (!owner.markAsFinished()) {
+				throw new Exception("Could notify job manager that the task is finished.");
+			}
 		}
 		catch (Throwable t) {
 			
+			if (!this.owner.isCanceled()) {
+
+				// Perform clean up when the task failed and has been not canceled by the user
+				try {
+					this.invokable.cancel();
+				} catch (Throwable t2) {
+					LOG.error("Error while canceling the task", t2);
+				}
+			}
+
 			// Release all resources that may currently be allocated by the individual channels
 			releaseAllChannelResources();
 
@@ -285,16 +281,10 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			else {
 				this.owner.markFailed(t);
 			}
-
-			return;
 		}
-
-		// Release all resources that may currently be allocated by the individual channels
-		releaseAllChannelResources();
-
-		// Finally, switch execution state to FINISHED and report to job manager
-		if (!owner.markAsFinished()) {
-			owner.markFailed(new Exception());
+		finally {
+			// Release all resources that may currently be allocated by the individual channels
+			releaseAllChannelResources();
 		}
 	}
 
@@ -373,7 +363,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 			if (this.executingThread == null) {
 				String name = owner.getTaskNameWithSubtasks();
-				this.executingThread = new Thread(this, name);
+				this.executingThread = new Thread(TASK_THREADS, this, name);
 			}
 
 			return this.executingThread;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 3dab13e..d916f74 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -334,14 +334,18 @@ public class ExecutionGraph {
 		}
 	}
 	
-	public void waitForJobEnd() throws InterruptedException {
+	public void waitForJobEnd(long timeout) throws InterruptedException {
 		synchronized (progressLock) {
 			while (nextVertexToFinish < verticesInCreationOrder.size()) {
-				progressLock.wait();
+				progressLock.wait(timeout);
 			}
 		}
 	}
 	
+	public void waitForJobEnd() throws InterruptedException {
+		waitForJobEnd(0);
+	}
+	
 	
 	private boolean transitionState(JobStatus current, JobStatus newState) {
 		return transitionState(current, newState, null);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index 3c65f2e..fcd21af 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -274,10 +274,12 @@ public class ExecutionVertex {
 			ExecutionEdge[] sources = inputEdges[i];
 			if (sources != null) {
 				for (int k = 0; k < sources.length; k++) {
-					Instance source = sources[k].getSource().getProducer().getCurrentAssignedResource().getInstance();
-					locations.add(source);
-					if (locations.size() > MAX_DISTINCT_LOCATIONS_TO_CONSIDER) {
-						return null;
+					AllocatedSlot sourceSlot = sources[k].getSource().getProducer().getCurrentAssignedResource();
+					if (sourceSlot != null) {
+						locations.add(sourceSlot.getInstance());
+						if (locations.size() > MAX_DISTINCT_LOCATIONS_TO_CONSIDER) {
+							return null;
+						}
 					}
 				}
 			}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index 3526e15..113f8fd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -272,6 +272,8 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 	@Override
 	public JobSubmissionResult submitJob(JobGraph job) throws IOException {
 		
+		
+		ExecutionGraph executionGraph = null;
 		boolean success = false;
 		
 		try {
@@ -285,7 +287,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 			}
 			
 			// get the existing execution graph (if we attach), or construct a new empty one to attach
-			ExecutionGraph executionGraph = this.currentJobs.get(job.getJobID());
+			executionGraph = this.currentJobs.get(job.getJobID());
 			if (executionGraph == null) {
 				if (LOG.isInfoEnabled()) {
 					LOG.info("Creating new execution graph for job " + job.getJobID() + " (" + job.getName() + ')');
@@ -331,7 +333,9 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 				}
 			}
 			catch (FileNotFoundException e) {
-				LOG.error("File-not-Found: " + e.getMessage());
+				String message = "File-not-Found: " + e.getMessage();
+				LOG.error(message);
+				executionGraph.fail(e);
 				return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, e.getMessage());
 			}
 			
@@ -373,10 +377,22 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 		}
 		catch (Throwable t) {
 			LOG.error("Job submission failed.", t);
+			executionGraph.fail(t);
 			return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(t));
 		}
 		finally {
 			if (!success) {
+				if (executionGraph != null) {
+					if (executionGraph.getState() != JobStatus.FAILING && executionGraph.getState() != JobStatus.FAILED) {
+						executionGraph.fail(new Exception("Could not set up and start execution graph on JobManager"));
+					}
+					try {
+						executionGraph.waitForJobEnd(10000);
+					} catch (InterruptedException e) {
+						LOG.error("Interrupted while waiting for job to finish canceling.");
+					}
+				}
+				
 				this.currentJobs.remove(job.getJobID());
 				
 				try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
index e8f8b72..1fd5a71 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
@@ -667,9 +667,10 @@ public class TaskManager implements TaskOperationProtocol {
 		// Unregister task from library cache manager
 		try {
 			LibraryCacheManager.unregister(task.getJobID());
-		} catch (IOException e) {
+		}
+		catch (Throwable t) {
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("Unregistering the execution " + executionId + " caused an IOException");
+				LOG.debug("Unregistering the cached libraries caused an exception: ",  t);
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionOutputFormat.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionOutputFormat.java
deleted file mode 100644
index 616eaf4..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionOutputFormat.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * 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.jobmanager;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.io.InitializeOnMaster;
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.StringRecord;
-
-
-public class ExceptionOutputFormat implements OutputFormat<StringRecord>, InitializeOnMaster {
-
-	private static final long serialVersionUID = 1L;
-	
-	/**
-	 * The message which is used for the test runtime exception.
-	 */
-	public static final String RUNTIME_EXCEPTION_MESSAGE = "This is a test runtime exception";
-
-	@Override
-	public void configure(Configuration parameters) {}
-
-	@Override
-	public void open(int taskNumber, int numTasks) {}
-
-	@Override
-	public void writeRecord(StringRecord record) {}
-
-	@Override
-	public void close() {}
-
-	@Override
-	public void initializeGlobal(int parallelism) throws IOException {
-		throw new RuntimeException(RUNTIME_EXCEPTION_MESSAGE);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionTask.java
deleted file mode 100644
index 7a0f9a5..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ExceptionTask.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/**
- * 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.jobmanager;
-
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * This task is used during the unit tests to generate a custom exception and check the proper response of the execution
- * engine.
- */
-public class ExceptionTask extends AbstractInvokable {
-
-	/**
-	 * The test error message included in the thrown exception
-	 */
-	public static final String ERROR_MESSAGE = "This is an expected test exception";
-
-	/**
-	 * The custom exception thrown by the this task.
-	 * 
-	 */
-	public static class TestException extends Exception {
-
-		/**
-		 * The generated serial version UID.
-		 */
-		private static final long serialVersionUID = -974961143742490972L;
-
-		/**
-		 * Constructs a new test exception.
-		 * 
-		 * @param msg
-		 *        the error message
-		 */
-		public TestException(String msg) {
-			super(msg);
-		}
-	}
-
-	@Override
-	public void registerInputOutput() {
-		new RecordReader<StringRecord>(this, StringRecord.class);
-		new RecordWriter<StringRecord>(this);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		throw new TestException(ERROR_MESSAGE);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index 8e87e7b..f661ea0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -32,19 +32,19 @@ import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.client.AbstractJobResult;
 import org.apache.flink.runtime.client.JobSubmissionResult;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.instance.LocalInstanceManager;
 import org.apache.flink.runtime.io.network.api.RecordReader;
 import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.io.network.bufferprovider.GlobalBufferPool;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.tasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.jobmanager.tasks.NoOpInvokable;
 import org.apache.flink.runtime.types.IntegerRecord;
-import org.apache.flink.util.StringUtils;
-
 import org.junit.Test;
 
 /**
@@ -53,6 +53,77 @@ import org.junit.Test;
 public class JobManagerITCase {
 	
 	@Test
+	public void testScheduleNotEnoughSlots() {
+		
+		try {
+			final AbstractJobVertex vertex = new AbstractJobVertex("Test Vertex");
+			vertex.setParallelism(2);
+			vertex.setInvokableClass(BlockingNoOpInvokable.class);
+			
+			final JobGraph jobGraph = new JobGraph("Test Job", vertex);
+			
+			final JobManager jm = startJobManager(1);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
+			try {
+				
+				assertEquals(1, jm.getAvailableSlots());
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+				assertEquals(AbstractJobResult.ReturnCode.ERROR, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					
+					long deadline = System.currentTimeMillis() + 60*1000;
+					boolean success = false;
+					
+					while (System.currentTimeMillis() < deadline) {
+						JobStatus state = eg.getState();
+						if (state == JobStatus.FINISHED) {
+							success = true;
+							break;
+						}
+						else if (state == JobStatus.FAILED || state == JobStatus.CANCELED) {
+							break;
+						}
+						else {
+							Thread.sleep(200);
+						}
+					}
+					
+					assertTrue("The job did not finish successfully.", success);
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
 	public void testSingleVertexJobImmediately() {
 		
 		final int NUM_TASKS = 133;
@@ -64,7 +135,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Test Job", vertex);
 			
-			JobManager jm = startJobManager(NUM_TASKS);
+			final JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				
 				assertEquals(NUM_TASKS, jm.getAvailableSlots());
@@ -106,6 +181,12 @@ public class JobManagerITCase {
 				else {
 					// already done, that was fast;
 				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -130,7 +211,11 @@ public class JobManagerITCase {
 			final JobGraph jobGraph = new JobGraph("Test Job", vertex);
 			jobGraph.setAllowQueuedScheduling(true);
 			
-			JobManager jm = startJobManager(10);
+			final JobManager jm = startJobManager(10);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				
 				// we need to register the job at the library cache manager (with no libraries)
@@ -150,6 +235,12 @@ public class JobManagerITCase {
 				else {
 					// already done, that was fast;
 				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -180,7 +271,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
-			JobManager jm = startJobManager(2 * NUM_TASKS);
+			final JobManager jm = startJobManager(2 * NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -199,6 +294,11 @@ public class JobManagerITCase {
 				else {
 					// already done, that was fast;
 				}
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(0, eg.getRegisteredExecutions().size());
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -229,7 +329,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Bipartite Job", sender, receiver);
 			
-			JobManager jm = startJobManager(2 * NUM_TASKS);
+			final JobManager jm = startJobManager(2 * NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -248,6 +352,12 @@ public class JobManagerITCase {
 				else {
 					// already done, that was fast;
 				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -260,9 +370,9 @@ public class JobManagerITCase {
 	}
 	
 	@Test
-	public void testTwoInputJob() {
+	public void testTwoInputJobFailingEdgeMismatch() {
 		
-		final int NUM_TASKS = 13;
+		final int NUM_TASKS = 2;
 		
 		try {
 			final AbstractJobVertex sender1 = new AbstractJobVertex("Sender1");
@@ -274,6 +384,68 @@ public class JobManagerITCase {
 			receiver.setInvokableClass(AgnosticReceiver.class);
 			
 			sender1.setParallelism(NUM_TASKS);
+			sender2.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE);
+			receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE);
+			
+			final JobGraph jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2);
+			
+			final JobManager jm = startJobManager(3 * NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
+			try {
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FAILED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(0, eg.getRegisteredExecutions().size());
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTwoInputJob() {
+		
+		final int NUM_TASKS = 11;
+		
+		try {
+			final AbstractJobVertex sender1 = new AbstractJobVertex("Sender1");
+			final AbstractJobVertex sender2 = new AbstractJobVertex("Sender2");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender1.setInvokableClass(Sender.class);
+			sender2.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(AgnosticBinaryReceiver.class);
+			
+			sender1.setParallelism(NUM_TASKS);
 			sender2.setParallelism(2*NUM_TASKS);
 			receiver.setParallelism(3*NUM_TASKS);
 			
@@ -283,6 +455,10 @@ public class JobManagerITCase {
 			final JobGraph jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2);
 			
 			JobManager jm = startJobManager(6 * NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+								.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -296,11 +472,17 @@ public class JobManagerITCase {
 				
 				if (eg != null) {
 					eg.waitForJobEnd();
-					assertEquals(JobStatus.FAILED, eg.getState());
+					assertEquals(JobStatus.FINISHED, eg.getState());
 				}
 				else {
 					// already done, that was fast;
 				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -331,7 +513,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
-			JobManager jm = startJobManager(NUM_TASKS);
+			final JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				assertEquals(NUM_TASKS, jm.getAvailableSlots());
 				
@@ -354,6 +540,10 @@ public class JobManagerITCase {
 				}
 				
 				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -384,7 +574,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
-			JobManager jm = startJobManager(NUM_TASKS);
+			final JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				assertEquals(NUM_TASKS, jm.getAvailableSlots());
 				
@@ -407,6 +601,10 @@ public class JobManagerITCase {
 				}
 				
 				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -437,7 +635,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
-			JobManager jm = startJobManager(2 * NUM_TASKS);
+			final JobManager jm = startJobManager(2 * NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				
 				// we need to register the job at the library cache manager (with no libraries)
@@ -459,6 +661,10 @@ public class JobManagerITCase {
 				}
 				
 				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -492,7 +698,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
-			JobManager jm = startJobManager(NUM_TASKS);
+			final JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				assertEquals(NUM_TASKS, jm.getAvailableSlots());
 				
@@ -515,6 +725,10 @@ public class JobManagerITCase {
 				}
 				
 				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -548,7 +762,11 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
-			JobManager jm = startJobManager(NUM_TASKS);
+			final JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
 			try {
 				assertEquals(NUM_TASKS, jm.getAvailableSlots());
 				
@@ -570,11 +788,11 @@ public class JobManagerITCase {
 					// already done, that was fast;
 				}
 				
-				for (Execution e : eg.getRegisteredExecutions().values()) {
-					System.out.println(e + StringUtils.arrayAwareToString(e.getStateTimestamps()));
-				}
-				
 				assertEquals(0, eg.getRegisteredExecutions().size());
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
 				jm.shutdown();
@@ -629,6 +847,21 @@ public class JobManagerITCase {
 		throw new IOException("could not find free port");
 	}
 	
+	private static void waitForTaskThreadsToBeTerminated() throws InterruptedException {
+		Thread[] threads = new Thread[Thread.activeCount()];
+		Thread.enumerate(threads);
+		
+		for (Thread t : threads) {
+			if (t == null) {
+				continue;
+			}
+			ThreadGroup tg = t.getThreadGroup();
+			if (tg != null && tg.getName() != null && tg.getName().equals("Task Threads")) {
+				t.join();
+			}
+		}
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	//  Simple test tasks
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/BlockingNoOpInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/BlockingNoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/BlockingNoOpInvokable.java
new file mode 100644
index 0000000..c8d1c98
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/BlockingNoOpInvokable.java
@@ -0,0 +1,38 @@
+/**
+ * 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.jobmanager.tasks;
+
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+/**
+ * An invokable that does nothing.
+ */
+public class BlockingNoOpInvokable extends AbstractInvokable {
+
+	@Override
+	public void registerInputOutput() {}
+
+	@Override
+	public void invoke() throws Exception {
+		Object o = new Object();
+		synchronized (o) {
+			o.wait();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/logback-test.xml b/flink-runtime/src/test/resources/logback-test.xml
index 7fb3387..f817d4d 100644
--- a/flink-runtime/src/test/resources/logback-test.xml
+++ b/flink-runtime/src/test/resources/logback-test.xml
@@ -37,4 +37,5 @@
     <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.TaskManager" level="OFF"/>
     <logger name="org.apache.flink.runtime.executiongraph.ExecutionGraph" level="OFF"/>
+    <logger name="org.apache.flink.runtime.jobmanager.EventCollector" level="OFF"/>
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae57c7c0/flink-runtime/src/test/resources/topology.txt
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/topology.txt b/flink-runtime/src/test/resources/topology.txt
deleted file mode 100644
index b199929..0000000
--- a/flink-runtime/src/test/resources/topology.txt
+++ /dev/null
@@ -1,16 +0,0 @@
-/mainswitch1/rackswitch1/node01
-/mainswitch1/rackswitch1/node02
-/mainswitch1/rackswitch1/node03
-/mainswitch1/rackswitch1/node04
-/mainswitch1/rackswitch2/node05
-/mainswitch1/rackswitch2/node06
-/mainswitch1/rackswitch2/node07
-/mainswitch1/rackswitch2/node08
-/mainswitch2/rackswitch3/node09
-/mainswitch2/rackswitch3/node10
-/mainswitch2/rackswitch3/node11
-/mainswitch2/rackswitch3/node12
-/mainswitch2/rackswitch4/node13
-/mainswitch2/rackswitch4/node14
-/mainswitch2/rackswitch4/node15
-/mainswitch2/rackswitch4/node16


[09/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java
deleted file mode 100644
index 08a03bc..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * 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.jobgraph;
-
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.api.common.operators.util.UserCodeWrapper;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-
-/**
- * A JobOutputVertex is a specific sub-type of a {@link AbstractJobOutputVertex} and is designed
- * for Nephele tasks which sink data in a not further specified way. As every job output vertex,
- * a JobOutputVertex must not have any further output.
- */
-public class OutputFormatOutputVertex extends AbstractJobOutputVertex {
-	/**
-	 * Contains the output format associated to this output vertex. It can be <pre>null</pre>.
-	 */
-	private OutputFormat<?> outputFormat;
-
-
-	/**
-	 * Creates a new job file output vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file output vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public OutputFormatOutputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-	
-	public OutputFormatOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public OutputFormatOutputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-	
-	public void setOutputFormat(OutputFormat<?> format) {
-		this.outputFormat = format;
-	}
-	
-	public void initializeOutputFormatFromTaskConfig(ClassLoader cl) {
-		TaskConfig cfg = new TaskConfig(getConfiguration());
-		UserCodeWrapper<OutputFormat<?>> wrapper = cfg.<OutputFormat<?>>getStubWrapper(cl);
-		
-		if (wrapper != null) {
-			this.outputFormat = wrapper.getUserCodeObject(OutputFormat.class, cl);
-			this.outputFormat.configure(cfg.getStubParameters());
-		}
-	}
-
-	/**
-	 * Returns the output format. It can also be <pre>null</pre>.
-	 *
-	 * @return output format or <pre>null</pre>
-	 */
-	public OutputFormat<?> getOutputFormat() { return outputFormat; }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatVertex.java
new file mode 100644
index 0000000..029d109
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatVertex.java
@@ -0,0 +1,66 @@
+/**
+ * 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.jobgraph;
+
+import org.apache.flink.api.common.io.InitializeOnMaster;
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.common.operators.util.UserCodeWrapper;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+
+/**
+ * A task vertex that run an initialization on the master, trying to deserialize an output format
+ * and initializing it on master, if necessary.
+ */
+public class OutputFormatVertex extends AbstractJobVertex {
+	
+	private static final long serialVersionUID = 1L;
+	
+	
+	/** Caches the output format associated to this output vertex. */
+	private transient OutputFormat<?> outputFormat;
+
+	/**
+	 * Creates a new task vertex with the specified name.
+	 * 
+	 * @param name The name of the task vertex.
+	 */
+	public OutputFormatVertex(String name) {
+		super(name);
+	}
+	
+	
+	@Override
+	public void initializeOnMaster(ClassLoader loader) throws Exception {
+		if (this.outputFormat == null) {
+			TaskConfig cfg = new TaskConfig(getConfiguration());
+			UserCodeWrapper<OutputFormat<?>> wrapper = cfg.<OutputFormat<?>>getStubWrapper(loader);
+		
+			if (wrapper == null) {
+				throw new Exception("No output format present in OutputFormatVertex's task configuration.");
+			}
+
+			this.outputFormat = wrapper.getUserCodeObject(OutputFormat.class, loader);
+			this.outputFormat.configure(cfg.getStubParameters());
+		}
+		
+		if (this.outputFormat instanceof InitializeOnMaster) {
+			((InitializeOnMaster) this.outputFormat).initializeGlobal(getParallelism());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java
deleted file mode 100644
index 3699f0e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * 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.jobgraph;
-
-import org.apache.flink.core.io.InputSplit;
-
-
-public class SimpleInputVertex extends AbstractJobInputVertex {
-
-	/**
-	 * Creates a new job file output vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file output vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public SimpleInputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-	
-	public SimpleInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public SimpleInputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-
-	@Override
-	public Class<? extends InputSplit> getInputSplitType() {
-		return null;
-	}
-
-	@Override
-	public InputSplit[] getInputSplits(int minNumSplits) throws Exception {
-		return null;
-	}	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java
deleted file mode 100644
index 8709a07..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * 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.jobgraph;
-
-/**
- * A JobOutputVertex is a specific sub-type of a {@link AbstractJobOutputVertex} and is designed
- * for Nephele tasks which sink data in a not further specified way. As every job output vertex,
- * a JobOutputVertex must not have any further output.
- */
-public class SimpleOutputVertex extends AbstractJobOutputVertex {
-
-	/**
-	 * Creates a new job file output vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file output vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public SimpleOutputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-	
-	public SimpleOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public SimpleOutputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}	
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
index d3ad516..aab9c89 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
@@ -29,7 +29,7 @@ public abstract class AbstractInvokable {
 	/**
 	 * The environment assigned to this invokable.
 	 */
-	private volatile Environment environment = null;
+	private volatile Environment environment;
 
 	/**
 	 * Must be overwritten by the concrete task to instantiate the required record reader and record writer.
@@ -60,7 +60,6 @@ public abstract class AbstractInvokable {
 	 * 
 	 * @return the environment of this task or <code>null</code> if the environment has not yet been set
 	 */
-	// TODO: This method should be final
 	public Environment getEnvironment() {
 		return this.environment;
 	}
@@ -72,7 +71,6 @@ public abstract class AbstractInvokable {
 	 * @return the current number of subtasks the respective task is split into
 	 */
 	public final int getCurrentNumberOfSubtasks() {
-
 		return this.environment.getCurrentNumberOfSubtasks();
 	}
 
@@ -82,7 +80,6 @@ public abstract class AbstractInvokable {
 	 * @return the index of this subtask in the subtask group
 	 */
 	public final int getIndexInSubtaskGroup() {
-
 		return this.environment.getIndexInSubtaskGroup();
 	}
 
@@ -92,7 +89,6 @@ public abstract class AbstractInvokable {
 	 * @return the task configuration object which was attached to the original {@link org.apache.flink.runtime.jobgraph.AbstractJobVertex}
 	 */
 	public final Configuration getTaskConfiguration() {
-
 		return this.environment.getTaskConfiguration();
 	}
 
@@ -102,40 +98,10 @@ public abstract class AbstractInvokable {
 	 * @return the job configuration object which was attached to the original {@link org.apache.flink.runtime.jobgraph.JobGraph}
 	 */
 	public final Configuration getJobConfiguration() {
-
 		return this.environment.getJobConfiguration();
 	}
 
 	/**
-	 * This method should be called by the user code if a custom
-	 * user thread has been started.
-	 * 
-	 * @param userThread
-	 *        the user thread which has been started
-	 */
-	public final void userThreadStarted(Thread userThread) {
-
-		if (this.environment != null) {
-			this.environment.userThreadStarted(userThread);
-		}
-
-	}
-
-	/**
-	 * This method should be called by the user code if a custom
-	 * user thread has finished.
-	 * 
-	 * @param userThread
-	 *        the user thread which has finished
-	 */
-	public final void userThreadFinished(Thread userThread) {
-
-		if (this.environment != null) {
-			this.environment.userThreadFinished(userThread);
-		}
-	}
-
-	/**
 	 * This method is called when a task is canceled either as a result of a user abort or an execution failure. It can
 	 * be overwritten to respond to shut down the user code properly.
 	 * 
@@ -143,7 +109,6 @@ public abstract class AbstractInvokable {
 	 *         thrown if any exception occurs during the execution of the user code
 	 */
 	public void cancel() throws Exception {
-
 		// The default implementation does nothing.
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java
index 7aa3374..94e6cab 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitIterator.java
@@ -24,7 +24,7 @@ import java.util.Iterator;
 import org.apache.flink.core.io.InputSplit;
 
 /**
- * The input split iterator allows an {@link AbstractInputTask} to iterator over all input splits it is supposed to
+ * The input split iterator allows a task to iterate over all input splits it is supposed to
  * consume. Internally, the input split iterator calls an {@link InputSplitProvider} on each <code>next</code> call in
  * order to facilitate lazy split assignment.
  * 
@@ -72,7 +72,6 @@ public class InputSplitIterator<T extends InputSplit> implements Iterator<T> {
 	@SuppressWarnings("unchecked")
 	@Override
 	public T next() {
-
 		T retVal = null;
 
 		if (this.nextInputSplit == null) {
@@ -88,8 +87,6 @@ public class InputSplitIterator<T extends InputSplit> implements Iterator<T> {
 
 	@Override
 	public void remove() {
-
 		throw new RuntimeException("The InputSplitIterator does not implement the remove method");
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
index 22722e7..20a4ab1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/InputSplitProvider.java
@@ -16,23 +16,21 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph.tasks;
 
 import org.apache.flink.core.io.InputSplit;
 
 /**
- * An input split provider can be successively queried to provide a series of {@link InputSplit} objects an
- * {@link AbstractInputTask} is supposed to consume in the course of its execution.
- * 
+ * An input split provider can be successively queried to provide a series of {@link InputSplit} objects a
+ * task is supposed to consume in the course of its execution.
  */
 public interface InputSplitProvider {
 
 	/**
-	 * Requests the next input split to be consumed by the calling {@link AbstractInputTask}.
+	 * Requests the next input split to be consumed by the calling task.
 	 * 
-	 * @return the next input split to be consumed by the calling {@link AbstractInputTask} or <code>null</code> if the
-	 *         {@link AbstractInputTask} shall not consume any further input splits.
+	 * @return the next input split to be consumed by the calling task or <code>null</code> if the
+	 *         task shall not consume any further input splits.
 	 */
 	InputSplit getNextInputSplit();
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DeploymentManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DeploymentManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DeploymentManager.java
deleted file mode 100644
index b8d9557..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DeploymentManager.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * 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.jobmanager;
-
-import java.util.List;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobgraph.JobID;
-
-/**
- * A deployment manager is responsible for deploying a list of {@link ExecutionVertex} objects the given
- * {@link org.apache.flink.runtime.instance.Instance}. It is called by a {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler} implementation whenever at least one
- * {@link ExecutionVertex} has become ready to be executed.
- * 
- */
-public interface DeploymentManager {
-
-	/**
-	 * Deploys the list of vertices on the given {@link org.apache.flink.runtime.instance.Instance}.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the vertices to be deployed belong to
-	 * @param instance
-	 *        the instance on which the vertices shall be deployed
-	 * @param verticesToBeDeployed
-	 *        the list of vertices to be deployed
-	 */
-	void deploy(JobID jobID, Instance instance, List<ExecutionVertex> verticesToBeDeployed);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
index 6800a68..d36659f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobmanager;
 
 import java.util.ArrayList;
@@ -32,27 +31,20 @@ import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent;
 import org.apache.flink.runtime.event.job.JobEvent;
 import org.apache.flink.runtime.event.job.ManagementEvent;
 import org.apache.flink.runtime.event.job.RecentJobEvent;
-import org.apache.flink.runtime.event.job.VertexAssignmentEvent;
 import org.apache.flink.runtime.event.job.VertexEvent;
 import org.apache.flink.runtime.execution.ExecutionListener;
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionGraphIterator;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.executiongraph.InternalJobStatus;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.JobStatusListener;
 import org.apache.flink.runtime.executiongraph.ManagementGraphFactory;
-import org.apache.flink.runtime.executiongraph.VertexAssignmentListener;
-import org.apache.flink.runtime.instance.AllocatedResource;
-import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.archive.ArchiveListener;
 import org.apache.flink.runtime.managementgraph.ManagementGraph;
 import org.apache.flink.runtime.managementgraph.ManagementVertex;
-import org.apache.flink.runtime.managementgraph.ManagementVertexID;
 import org.apache.flink.runtime.profiling.ProfilingListener;
 import org.apache.flink.runtime.profiling.types.ProfilingEvent;
 
@@ -62,7 +54,6 @@ import org.apache.flink.runtime.profiling.types.ProfilingEvent;
  * the event collector removes all intervals which are older than the interval.
  * <p>
  * This class is thread-safe.
- * 
  */
 public final class EventCollector extends TimerTask implements ProfilingListener {
 
@@ -72,85 +63,43 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 * the data provided by the <code>executionStateChanged</code> callback method.
 	 * However, these IDs are needed to create the construct the {@link VertexEvent} and the
 	 * {@link ExecutionStateChangeEvent}.
-	 * 
 	 */
 	private static final class ExecutionListenerWrapper implements ExecutionListener {
 
-		/**
-		 * The event collector to forward the created event to.
-		 */
+		/** The event collector to forward the created event to. */
 		private final EventCollector eventCollector;
 
-		/**
-		 * The vertex this listener belongs to.
-		 */
-		private final ExecutionVertex vertex;
+		private final ExecutionGraph graph;
+		
 
-		/**
-		 * Constructs a new execution listener object.
-		 * 
-		 * @param eventCollector
-		 *        the event collector to forward the created event to
-		 * @param vertex
-		 *        the vertex this listener belongs to.
-		 */
-		public ExecutionListenerWrapper(final EventCollector eventCollector, final ExecutionVertex vertex) {
+		public ExecutionListenerWrapper(EventCollector eventCollector, ExecutionGraph graph) {
 			this.eventCollector = eventCollector;
-			this.vertex = vertex;
+			this.graph = graph;
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
-		public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID,
-				final ExecutionState newExecutionState, final String optionalMessage) {
-
+		public void executionStateChanged(JobID jobID, JobVertexID vertexId, int subtask, ExecutionAttemptID executionId,
+				ExecutionState2 newExecutionState, String optionalMessage)
+		{
 			final long timestamp = System.currentTimeMillis();
 
-			final JobVertexID jobVertexID = this.vertex.getGroupVertex().getJobVertexID();
-			final String taskName = this.vertex.getGroupVertex().getName();
-			final int totalNumberOfSubtasks = this.vertex.getGroupVertex().getCurrentNumberOfGroupMembers();
-			final int indexInSubtaskGroup = this.vertex.getIndexInVertexGroup();
+			final ExecutionJobVertex vertex = graph.getJobVertex(vertexId);
+			
+			final String taskName = vertex == null ? "(null)" : vertex.getJobVertex().getName();
+			final int totalNumberOfSubtasks = vertex == null ? -1 : vertex.getParallelism();
 
 			// Create a new vertex event
-			final VertexEvent vertexEvent = new VertexEvent(timestamp, jobVertexID, taskName, totalNumberOfSubtasks,
-				indexInSubtaskGroup, newExecutionState, optionalMessage);
+			final VertexEvent vertexEvent = new VertexEvent(timestamp, vertexId, taskName, totalNumberOfSubtasks,
+					subtask, newExecutionState, optionalMessage);
 
 			this.eventCollector.addEvent(jobID, vertexEvent);
 
 			final ExecutionStateChangeEvent executionStateChangeEvent = new ExecutionStateChangeEvent(timestamp,
-				vertexID.toManagementVertexID(), newExecutionState);
+					vertexId.toManagementVertexId(subtask), newExecutionState);
 
 			this.eventCollector.updateManagementGraph(jobID, executionStateChangeEvent, optionalMessage);
 			this.eventCollector.addEvent(jobID, executionStateChangeEvent);
 		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-			// Nothing to do here
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-			// Nothing to do here
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public int getPriority() {
-
-			return 20;
-		}
-
 	}
 
 	/**
@@ -162,24 +111,16 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 */
 	private static final class JobStatusListenerWrapper implements JobStatusListener {
 
-		/**
-		 * The event collector to forward the created event to.
-		 */
+		/** The event collector to forward the created event to. */
 		private final EventCollector eventCollector;
 
-		/**
-		 * The name of the job this wrapper has been created for.
-		 */
+		/** The name of the job this wrapper has been created for. */
 		private final String jobName;
 
-		/**
-		 * <code>true</code> if profiling events are collected for the job, <code>false</code> otherwise.
-		 */
+		/** <code>true</code> if profiling events are collected for the job, <code>false</code> otherwise. */
 		private final boolean isProfilingAvailable;
 
-		/**
-		 * The time stamp of the job submission
-		 */
+		/** The time stamp of the job submission */
 		private final long submissionTimestamp;
 
 		/**
@@ -194,101 +135,32 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 		 * @param submissionTimestamp
 		 *        the submission time stamp of the job
 		 */
-		public JobStatusListenerWrapper(final EventCollector eventCollector, final String jobName,
-				final boolean isProfilingAvailable, final long submissionTimestamp) {
-
+		public JobStatusListenerWrapper(EventCollector eventCollector, String jobName,
+				boolean isProfilingAvailable, long submissionTimestamp)
+		{
 			this.eventCollector = eventCollector;
 			this.jobName = jobName;
 			this.isProfilingAvailable = isProfilingAvailable;
 			this.submissionTimestamp = submissionTimestamp;
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
-		public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
-				final String optionalMessage) {
+		public void jobStatusHasChanged(ExecutionGraph executionGraph, JobStatus newJobStatus, String optionalMessage) {
 
 			final JobID jobID = executionGraph.getJobID();
 
-			if (newJobStatus == InternalJobStatus.SCHEDULED) {
+			if (newJobStatus == JobStatus.RUNNING) {
 
 				final ManagementGraph managementGraph = ManagementGraphFactory.fromExecutionGraph(executionGraph);
 				this.eventCollector.addManagementGraph(jobID, managementGraph);
 			}
 
 			// Update recent job event
-			final JobStatus jobStatus = InternalJobStatus.toJobStatus(newJobStatus);
-			if (jobStatus != null) {
-				this.eventCollector.updateRecentJobEvent(jobID, this.jobName, this.isProfilingAvailable,
-					this.submissionTimestamp, jobStatus);
-
-				this.eventCollector.addEvent(jobID,
-					new JobEvent(System.currentTimeMillis(), jobStatus, optionalMessage));
-			}
-		}
-	}
-
-	/**
-	 * The vertex assignment listener wrapper is an auxiliary class. It is required
-	 * because the job ID cannot be accessed from the data provided by the <code>vertexAssignmentChanged</code> callback
-	 * method. However, this job ID is needed to prepare the {@link VertexAssignmentEvent} for transmission.
-	 * 
-	 */
-	private static final class VertexAssignmentListenerWrapper implements VertexAssignmentListener {
-
-		/**
-		 * The event collector to forward the created event to.
-		 */
-		private final EventCollector eventCollector;
-
-		/**
-		 * The ID the job this wrapper has been created for.
-		 */
-		private final JobID jobID;
-
-		/**
-		 * Constructs a new vertex assignment listener wrapper.
-		 * 
-		 * @param eventCollector
-		 *        the event collector to forward the events to
-		 * @param jobID
-		 *        the ID of the job
-		 */
-		public VertexAssignmentListenerWrapper(final EventCollector eventCollector, final JobID jobID) {
-			this.eventCollector = eventCollector;
-			this.jobID = jobID;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public void vertexAssignmentChanged(final ExecutionVertexID id, final AllocatedResource newAllocatedResource) {
-
-			// Create a new vertex assignment event
-			final ManagementVertexID managementVertexID = id.toManagementVertexID();
-			final long timestamp = System.currentTimeMillis();
-
-			final Instance instance = newAllocatedResource.getInstance();
-			VertexAssignmentEvent event;
-			if (instance == null) {
-				event = new VertexAssignmentEvent(timestamp, managementVertexID, "null");
-			} else {
-
-				String instanceName = null;
-				if (instance.getInstanceConnectionInfo() != null) {
-					instanceName = instance.getInstanceConnectionInfo().toString();
-				} else {
-					instanceName = instance.toString();
-				}
+			this.eventCollector.updateRecentJobEvent(jobID, this.jobName, this.isProfilingAvailable,
+					this.submissionTimestamp, newJobStatus);
 
-				event = new VertexAssignmentEvent(timestamp, managementVertexID, instanceName);
-			}
-
-			this.eventCollector.updateManagementGraph(jobID, event);
-			this.eventCollector.addEvent(this.jobID, event);
+			this.eventCollector.addEvent(jobID,
+					new JobEvent(System.currentTimeMillis(), newJobStatus, optionalMessage));
 		}
 	}
 
@@ -344,8 +216,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 *        <code>true</code> if {@link ManagementEvent} objects shall be added to the list as well,
 	 *        <code>false</code> otherwise
 	 */
-	public void getEventsForJob(final JobID jobID, final List<AbstractEvent> eventList,
-			final boolean includeManagementEvents) {
+	public void getEventsForJob(JobID jobID, List<AbstractEvent> eventList, boolean includeManagementEvents) {
 
 		synchronized (this.collectedEvents) {
 
@@ -431,15 +302,15 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 * @param jobStatus
 	 *        the status of the job
 	 */
-	private void updateRecentJobEvent(final JobID jobID, final String jobName, final boolean isProfilingEnabled,
-			final long submissionTimestamp, final JobStatus jobStatus) {
-
+	private void updateRecentJobEvent(JobID jobID, String jobName, boolean isProfilingEnabled,
+			long submissionTimestamp, JobStatus jobStatus)
+	{
 		final long currentTime = System.currentTimeMillis();
+		
 		final RecentJobEvent recentJobEvent = new RecentJobEvent(jobID, jobName, jobStatus, isProfilingEnabled,
 			submissionTimestamp, currentTime);
 
 		synchronized (this.recentJobs) {
-
 			this.recentJobs.put(jobID, recentJobEvent);
 		}
 	}
@@ -448,7 +319,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 * Registers a job in form of its execution graph representation
 	 * with the job progress collector. The collector will subscribe
 	 * to state changes of the individual subtasks. A separate
-	 * deregistration is not necessary since the job progress collector
+	 * de-registration is not necessary since the job progress collector
 	 * periodically discards outdated progress information.
 	 * 
 	 * @param executionGraph
@@ -458,26 +329,12 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 * @param submissionTimestamp
 	 *        the submission time stamp of the job
 	 */
-	public void registerJob(final ExecutionGraph executionGraph, final boolean profilingAvailable,
-			final long submissionTimestamp) {
-
-		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(executionGraph, true);
-
-		while (it.hasNext()) {
+	public void registerJob(ExecutionGraph executionGraph, boolean profilingAvailable, long submissionTimestamp) {
 
-			final ExecutionVertex vertex = it.next();
+		executionGraph.registerExecutionListener(new ExecutionListenerWrapper(this, executionGraph));
 
-			// Register the listener object which will pass state changes on to the collector
-			vertex.registerExecutionListener(new ExecutionListenerWrapper(this, vertex));
-
-			// Register the listener object which will pass assignment changes on to the collector
-			vertex.registerVertexAssignmentListener(new VertexAssignmentListenerWrapper(this, executionGraph.getJobID()));
-		}
-
-		// Register one job status listener wrapper for the entire job
 		executionGraph.registerJobStatusListener(new JobStatusListenerWrapper(this, executionGraph.getJobName(),
 			profilingAvailable, submissionTimestamp));
-
 	}
 
 	/**
@@ -547,7 +404,6 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 
 	@Override
 	public void processProfilingEvents(final ProfilingEvent profilingEvent) {
-
 		// Simply add profiling events to the job's event queue
 		addEvent(profilingEvent.getJobID(), profilingEvent);
 	}
@@ -561,7 +417,6 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 *        the management graph to be added
 	 */
 	void addManagementGraph(final JobID jobID, final ManagementGraph managementGraph) {
-
 		synchronized (this.recentManagementGraphs) {
 			this.recentManagementGraphs.put(jobID, managementGraph);
 		}
@@ -576,38 +431,12 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 * @return the management graph for the job with the given ID or <code>null</code> if no such graph exists
 	 */
 	public ManagementGraph getManagementGraph(final JobID jobID) {
-
 		synchronized (this.recentManagementGraphs) {
 			return this.recentManagementGraphs.get(jobID);
 		}
 	}
 
 	/**
-	 * Applies changes in the vertex assignment to the stored management graph.
-	 * 
-	 * @param jobID
-	 *        the ID of the job whose management graph shall be updated
-	 * @param vertexAssignmentEvent
-	 *        the event describing the changes in the vertex assignment
-	 */
-	private void updateManagementGraph(final JobID jobID, final VertexAssignmentEvent vertexAssignmentEvent) {
-
-		synchronized (this.recentManagementGraphs) {
-
-			final ManagementGraph managementGraph = this.recentManagementGraphs.get(jobID);
-			if (managementGraph == null) {
-				return;
-			}
-			final ManagementVertex vertex = managementGraph.getVertexByID(vertexAssignmentEvent.getVertexID());
-			if (vertex == null) {
-				return;
-			}
-
-			vertex.setInstanceName(vertexAssignmentEvent.getInstanceName());
-		}
-	}
-
-	/**
 	 * Applies changes in the state of an execution vertex to the stored management graph.
 	 * 
 	 * @param jobID
@@ -615,7 +444,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 	 * @param executionStateChangeEvent
 	 *        the event describing the changes in the execution state of the vertex
 	 */
-	private void updateManagementGraph(final JobID jobID, final ExecutionStateChangeEvent executionStateChangeEvent, String optionalMessage) {
+	private void updateManagementGraph(JobID jobID, ExecutionStateChangeEvent executionStateChangeEvent, String optionalMessage) {
 
 		synchronized (this.recentManagementGraphs) {
 
@@ -629,7 +458,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 			}
 
 			vertex.setExecutionState(executionStateChangeEvent.getNewExecutionState());
-			if (executionStateChangeEvent.getNewExecutionState() == ExecutionState.FAILED) {
+			if (optionalMessage != null) {
 				vertex.setOptMessage(optionalMessage);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index 3b76b78..fc76d73 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobmanager;
 
 import java.io.File;
@@ -25,11 +24,10 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
-import java.util.HashSet;
-import java.util.Iterator;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
@@ -48,61 +46,49 @@ import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.io.StringRecord;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.runtime.ExecutionMode;
+import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.accumulators.AccumulatorEvent;
 import org.apache.flink.runtime.client.AbstractJobResult;
+import org.apache.flink.runtime.client.AbstractJobResult.ReturnCode;
 import org.apache.flink.runtime.client.JobCancelResult;
 import org.apache.flink.runtime.client.JobProgressResult;
 import org.apache.flink.runtime.client.JobSubmissionResult;
-import org.apache.flink.runtime.client.AbstractJobResult.ReturnCode;
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.event.job.AbstractEvent;
 import org.apache.flink.runtime.event.job.RecentJobEvent;
-import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.executiongraph.ExecutionEdge;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionGraphIterator;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.executiongraph.GraphConversionException;
-import org.apache.flink.runtime.executiongraph.InternalJobStatus;
+import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.JobStatusListener;
-import org.apache.flink.runtime.instance.DefaultInstanceManager;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.instance.InstanceManager;
 import org.apache.flink.runtime.instance.LocalInstanceManager;
 import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
-import org.apache.flink.runtime.io.network.RemoteReceiver;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.ipc.RPC;
 import org.apache.flink.runtime.ipc.Server;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager;
 import org.apache.flink.runtime.jobmanager.archive.ArchiveListener;
 import org.apache.flink.runtime.jobmanager.archive.MemoryArchivist;
 import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
-import org.apache.flink.runtime.jobmanager.splitassigner.InputSplitWrapper;
 import org.apache.flink.runtime.jobmanager.web.WebInfoServer;
 import org.apache.flink.runtime.managementgraph.ManagementGraph;
-import org.apache.flink.runtime.managementgraph.ManagementVertexID;
-import org.apache.flink.runtime.profiling.JobManagerProfiler;
-import org.apache.flink.runtime.profiling.ProfilingUtils;
 import org.apache.flink.runtime.protocols.AccumulatorProtocol;
 import org.apache.flink.runtime.protocols.ChannelLookupProtocol;
 import org.apache.flink.runtime.protocols.ExtendedManagementProtocol;
 import org.apache.flink.runtime.protocols.InputSplitProviderProtocol;
 import org.apache.flink.runtime.protocols.JobManagerProtocol;
-import org.apache.flink.runtime.taskmanager.AbstractTaskResult;
-import org.apache.flink.runtime.taskmanager.TaskCancelResult;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import org.apache.flink.runtime.taskmanager.TaskKillResult;
-import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
 import org.apache.flink.runtime.types.IntegerRecord;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
@@ -113,15 +99,13 @@ import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 import org.apache.log4j.PatternLayout;
 
+import com.google.common.base.Preconditions;
+
 /**
- * In Nephele the job manager is the central component for communication with clients, creating
- * schedules for incoming jobs and supervise their execution. A job manager may only exist once in
- * the system and its address must be known the clients.
- * Task managers can discover the job manager by means of an UDP broadcast and afterwards advertise
- * themselves as new workers for tasks.
- * 
+ * The JobManager is the master that coordinates the distributed execution.
+ * It receives jobs from clients, tracks the distributed execution.
  */
-public class JobManager implements DeploymentManager, ExtendedManagementProtocol, InputSplitProviderProtocol,
+public class JobManager implements ExtendedManagementProtocol, InputSplitProviderProtocol,
 		JobManagerProtocol, ChannelLookupProtocol, JobStatusListener, AccumulatorProtocol
 {
 
@@ -130,32 +114,46 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	private final static int FAILURE_RETURN_CODE = 1;
 	
 	
+	/** Executor service for asynchronous commands (to relieve the RPC threads of work) */
 	private final ExecutorService executorService = Executors.newCachedThreadPool(ExecutorThreadFactory.INSTANCE);
 	
-	private final Server jobManagerServer;
 
-	private final EventCollector eventCollector;
-	
-	private final ArchiveListener archive;
+	/** The RPC end point through which the JobManager gets its calls */
+	private final Server jobManagerServer;
 
+	/** Keeps track of the currently available task managers */
 	private final InstanceManager instanceManager;
 	
+	/** Assigns tasks to slots and keeps track on available and allocated task slots*/
 	private final DefaultScheduler scheduler;
 	
+	/** The currently running jobs */
+	private final ConcurrentHashMap<JobID, ExecutionGraph> currentJobs;
+	
+	
+	// begin: these will be consolidated / removed 
+	private final EventCollector eventCollector;
+	
+	private final ArchiveListener archive;
+	
 	private final AccumulatorManager accumulatorManager;
-
 	
 	private final int recommendedClientPollingInterval;
-
+	// end: these will be consolidated / removed
+	
 	
 	private final AtomicBoolean isShutdownInProgress = new AtomicBoolean(false);
-
+	
 	private volatile boolean isShutDown;
 	
 	
 	private WebInfoServer server;
 	
 	
+	// --------------------------------------------------------------------------------------------
+	//  Initialization & Shutdown
+	// --------------------------------------------------------------------------------------------
+	
 	public JobManager(ExecutionMode executionMode) throws Exception {
 
 		final String ipcAddressString = GlobalConfiguration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
@@ -190,6 +188,8 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			this.archive = null;
 		}
 		
+		this.currentJobs = new ConcurrentHashMap<JobID, ExecutionGraph>();
+		
 		// Create the accumulator manager, with same archiving limit as web
 		// interface. We need to store the accumulators for at least one job.
 		// Otherwise they might be deleted before the client requested the
@@ -218,21 +218,15 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			this.instanceManager = new LocalInstanceManager(numTaskManagers);
 		}
 		else if (executionMode == ExecutionMode.CLUSTER) {
-			this.instanceManager = new DefaultInstanceManager();
+			this.instanceManager = new InstanceManager();
 		}
 		else {
 			throw new IllegalArgumentException("ExecutionMode");
 		}
 
-		// Try to load the scheduler for the given execution mode
-		final String schedulerClassName = JobManagerUtils.getSchedulerClassName(executionMode);
-		LOG.info("Trying to load " + schedulerClassName + " as scheduler");
-
-		// Try to get the instance manager class name
-		this.scheduler = JobManagerUtils.loadScheduler(schedulerClassName, this, this.instanceManager);
-		if (this.scheduler == null) {
-			throw new Exception("Unable to load scheduler " + schedulerClassName);
-		}
+		// create the scheduler and make it listen at the availability of new instances
+		this.scheduler = new DefaultScheduler();
+		this.instanceManager.addInstanceListener(this.scheduler);
 	}
 
 	public void shutdown() {
@@ -275,393 +269,223 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		LOG.debug("Shutdown of job manager completed");
 	}
 	
-	/**
-	 * Entry point for the program
-	 * 
-	 * @param args
-	 *        arguments from the command line
-	 */
-	
-	public static void main(String[] args) {
-		// determine if a valid log4j config exists and initialize a default logger if not
-		if (System.getProperty("log4j.configuration") == null) {
-			Logger root = Logger.getRootLogger();
-			root.removeAllAppenders();
-			PatternLayout layout = new PatternLayout("%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n");
-			ConsoleAppender appender = new ConsoleAppender(layout, "System.err");
-			root.addAppender(appender);
-			root.setLevel(Level.INFO);
-		}
-		
-		JobManager jobManager;
-		try {
-			jobManager = initialize(args);
-			// Start info server for jobmanager
-			jobManager.startInfoServer();
-		}
-		catch (Exception e) {
-			LOG.fatal(e.getMessage(), e);
-			System.exit(FAILURE_RETURN_CODE);
-		}
-		
-		// Clean up is triggered through a shutdown hook
-		// freeze this thread to keep the JVM alive (the job manager threads are daemon threads)
-		Object w = new Object();
-		synchronized (w) {
-			try {
-				w.wait();
-			} catch (InterruptedException e) {}
-		}
-	}
-	
-	@SuppressWarnings("static-access")
-	public static JobManager initialize(String[] args) throws Exception {
-		final Option configDirOpt = OptionBuilder.withArgName("config directory").hasArg()
-			.withDescription("Specify configuration directory.").create("configDir");
-
-		final Option executionModeOpt = OptionBuilder.withArgName("execution mode").hasArg()
-			.withDescription("Specify execution mode.").create("executionMode");
-
-		final Options options = new Options();
-		options.addOption(configDirOpt);
-		options.addOption(executionModeOpt);
-
-		CommandLineParser parser = new GnuParser();
-		CommandLine line = null;
-		try {
-			line = parser.parse(options, args);
-		} catch (ParseException e) {
-			LOG.error("CLI Parsing failed. Reason: " + e.getMessage());
-			System.exit(FAILURE_RETURN_CODE);
-		}
-
-		final String configDir = line.getOptionValue(configDirOpt.getOpt(), null);
-		final String executionModeName = line.getOptionValue(executionModeOpt.getOpt(), "local");
-		
-		ExecutionMode executionMode = null;
-		if ("local".equals(executionModeName)) {
-			executionMode = ExecutionMode.LOCAL;
-		} else if ("cluster".equals(executionModeName)) {
-			executionMode = ExecutionMode.CLUSTER;
-		} else {
-			System.err.println("Unrecognized execution mode: " + executionModeName);
-			System.exit(FAILURE_RETURN_CODE);
-		}
-		
-		// print some startup environment info, like user, code revision, etc
-		EnvironmentInformation.logEnvironmentInfo(LOG, "JobManager");
-		
-		// First, try to load global configuration
-		GlobalConfiguration.loadConfiguration(configDir);
-
-		// Create a new job manager object
-		JobManager jobManager = new JobManager(executionMode);
-		
-		// Set base dir for info server
-		Configuration infoserverConfig = GlobalConfiguration.getConfiguration();
-		if (configDir != null && new File(configDir).isDirectory()) {
-			infoserverConfig.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, configDir+"/..");
-		}
-		GlobalConfiguration.includeConfiguration(infoserverConfig);
-		return jobManager;
-	}
-
+	// --------------------------------------------------------------------------------------------
+	//  Job Execution
+	// --------------------------------------------------------------------------------------------
 
 	@Override
 	public JobSubmissionResult submitJob(JobGraph job) throws IOException {
+		
+		boolean success = false;
+		
 		try {
 			// First check if job is null
 			if (job == null) {
 				return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, "Submitted job is null!");
 			}
 	
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Submitted job " + job.getName() + " is not null");
-			}
-	
-			// Check if any vertex of the graph has null edges
-			AbstractJobVertex jv = job.findVertexWithNullEdges();
-			if (jv != null) {
-				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, "Vertex "
-					+ jv.getName() + " has at least one null edge");
-				return result;
+			if (LOG.isInfoEnabled()) {
+				LOG.info(String.format("Received job %s (%s)", job.getJobID(), job.getName()));
 			}
-	
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Submitted job " + job.getName() + " has no null edges");
+			
+			// get the existing execution graph (if we attach), or construct a new empty one to attach
+			ExecutionGraph executionGraph = this.currentJobs.get(job.getJobID());
+			if (executionGraph == null) {
+				if (LOG.isInfoEnabled()) {
+					LOG.info("Creating new execution graph for job " + job.getJobID() + " (" + job.getName() + ')');
+				}
+				
+				executionGraph = new ExecutionGraph(job.getJobID(), job.getName(), job.getJobConfiguration(), this.executorService);
+				ExecutionGraph previous = this.currentJobs.putIfAbsent(job.getJobID(), executionGraph);
+				if (previous != null) {
+					throw new JobException("Concurrent submission of a job with the same jobId: " + job.getJobID());
+				}
 			}
-	
-			// Next, check if the graph is weakly connected
-			if (!job.isWeaklyConnected()) {
-				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR,
-					"Job graph is not weakly connected");
-				return result;
+			else {
+				if (LOG.isInfoEnabled()) {
+					LOG.info(String.format("Found existing execution graph for id %s, attaching this job.", job.getJobID()));
+				}
 			}
-	
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("The graph of job " + job.getName() + " is weakly connected");
+			
+			// grab the class loader for user-defined code
+			final ClassLoader userCodeLoader = LibraryCacheManager.getClassLoader(job.getJobID());
+			if (userCodeLoader == null) {
+				throw new JobException("The user code class loader could not be initialized.");
 			}
-	
-			// Check if job graph has cycles
-			if (!job.isAcyclic()) {
-				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR,
-					"Job graph is not a DAG");
-				return result;
+			
+			String[] jarFilesForJob = LibraryCacheManager.getRequiredJarFiles(job.getJobID());
+			for (String fileId : jarFilesForJob) {
+				executionGraph.addUserCodeJarFile(fileId);
 			}
-	
+			
+			// first, perform the master initialization of the nodes
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("The graph of job " + job.getName() + " is acyclic");
-			}
-	
-			// Check constrains on degree
-			jv = job.areVertexDegreesCorrect();
-			if (jv != null) {
-				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR,
-					"Degree of vertex " + jv.getName() + " is incorrect");
-				return result;
+				LOG.debug(String.format("Running master initialization of job %s (%s)", job.getJobID(), job.getName()));
 			}
-	
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("All vertices of job " + job.getName() + " have the correct degree");
+			try {
+				for (AbstractJobVertex vertex : job.getVertices()) {
+					// check that the vertex has an executable class
+					String executableClass = vertex.getInvokableClassName();
+					if (executableClass == null || executableClass.length() == 0) {
+						throw new JobException(String.format("The vertex %s (%s) has no invokable class.", vertex.getID(), vertex.getName()));
+					}
+					
+					// master side initialization
+					vertex.initializeOnMaster(userCodeLoader);
+				}
 			}
-	
-			if (!job.isInstanceDependencyChainAcyclic()) {
-				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR,
-					"The dependency chain for instance sharing contains a cycle");
-	
-				return result;
+			catch (FileNotFoundException e) {
+				LOG.error("File-not-Found: " + e.getMessage());
+				return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, e.getMessage());
 			}
-	
+			
+			// first topologically sort the job vertices to form the basis of creating the execution graph
+			List<AbstractJobVertex> topoSorted = job.getVerticesSortedTopologicallyFromSources();
+			
+			// first convert this job graph to an execution graph
 			if (LOG.isDebugEnabled()) {
-				LOG.debug("The dependency chain for instance sharing is acyclic");
+				LOG.debug(String.format("Adding %d vertices from job graph %s (%s)", topoSorted.size(), job.getJobID(), job.getName()));
 			}
-	
-			// Try to create initial execution graph from job graph
-			LOG.info("Creating initial execution graph from job graph " + job.getName());
-			ExecutionGraph eg;
-	
-			try {
-				eg = new ExecutionGraph(job, 1);
-			} catch (GraphConversionException e) {
-				if (e.getCause() == null) {
-					return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(e));
-				} else {
-					Throwable t = e.getCause();
-					if (t instanceof FileNotFoundException) {
-						return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, t.getMessage());
-					} else {
-						return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(t));
-					}
-				}
+			
+			executionGraph.attachJobGraph(topoSorted);
+			
+			if (LOG.isDebugEnabled()) {
+				LOG.debug(String.format("Successfully created execution graph from job graph %s (%s)", job.getJobID(), job.getName()));
 			}
 	
 			// Register job with the progress collector
 			if (this.eventCollector != null) {
-				this.eventCollector.registerJob(eg, false, System.currentTimeMillis());
+				this.eventCollector.registerJob(executionGraph, false, System.currentTimeMillis());
 			}
 	
 			// Register for updates on the job status
-			eg.registerJobStatusListener(this);
+			executionGraph.registerJobStatusListener(this);
 	
 			// Schedule job
 			if (LOG.isInfoEnabled()) {
 				LOG.info("Scheduling job " + job.getName());
 			}
 	
-			try {
-				this.scheduler.scheduleJob(eg);
-			} catch (SchedulingException e) {
-				unregisterJob(eg);
-				JobSubmissionResult result = new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(e));
-				return result;
-			}
+			executionGraph.scheduleForExecution(this.scheduler);
 	
 			// Return on success
+			success = true;
 			return new JobSubmissionResult(AbstractJobResult.ReturnCode.SUCCESS, null);
 		}
 		catch (Throwable t) {
 			LOG.error("Job submission failed.", t);
 			return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, StringUtils.stringifyException(t));
 		}
-	}
-	
-
-	public InstanceManager getInstanceManager() {
-		return this.instanceManager;
-	}
-
-	/**
-	 * This method is a convenience method to unregister a job from all of
-	 * Nephele's monitoring, profiling and optimization components at once.
-	 * Currently, it is only being used to unregister from profiling (if activated).
-	 * 
-	 * @param executionGraph
-	 *        the execution graph to remove from the job manager
-	 */
-	private void unregisterJob(final ExecutionGraph executionGraph) {
-
-		// Remove job from profiler (if activated)
-		if (this.profiler != null
-			&& executionGraph.getJobConfiguration().getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) {
-			this.profiler.unregisterProfilingJob(executionGraph);
-
-			if (this.eventCollector != null) {
-				this.profiler.unregisterFromProfilingData(executionGraph.getJobID(), this.eventCollector);
-			}
-		}
-
-		// Remove job from input split manager
-		if (this.inputSplitManager != null) {
-			this.inputSplitManager.unregisterJob(executionGraph);
-		}
-
-		// Unregister job with library cache manager
-		try {
-			LibraryCacheManager.unregister(executionGraph.getJobID());
-		} catch (IOException ioe) {
-			if (LOG.isWarnEnabled()) {
-				LOG.warn(ioe);
+		finally {
+			if (!success) {
+				this.currentJobs.remove(job.getJobID());
+				
+				try {
+					LibraryCacheManager.unregister(job.getJobID());
+				}
+				catch (IllegalStateException e) {
+					// may happen if the job failed before being registered at the
+					// library cache manager
+				}
+				catch (Throwable t) {
+					LOG.error("Error while de-registering job at library cache manager.", t);
+				}
 			}
 		}
 	}
 
-
 	@Override
-	public void sendHeartbeat(final InstanceConnectionInfo instanceConnectionInfo) {
-
-		// Delegate call to instance manager
-		if (this.instanceManager != null) {
-
-			final Runnable heartBeatRunnable = new Runnable() {
+	public JobCancelResult cancelJob(JobID jobID) throws IOException {
 
-				@Override
-				public void run() {
-					instanceManager.reportHeartBeat(instanceConnectionInfo);
-				}
-			};
+		LOG.info("Trying to cancel job with ID " + jobID);
 
-			this.executorService.execute(heartBeatRunnable);
+		final ExecutionGraph eg = this.currentJobs.get(jobID);
+		if (eg == null) {
+			LOG.info("No job found with ID " + jobID);
+			return new JobCancelResult(ReturnCode.ERROR, "Cannot find job with ID " + jobID);
 		}
-	}
 
-	@Override
-	public RegisterTaskManagerResult registerTaskManager(final InstanceConnectionInfo instanceConnectionInfo,
-									final HardwareDescription hardwareDescription, final IntegerRecord numberOfSlots){
-		if(this.instanceManager != null) {
-			final Runnable registerTaskManagerRunnable = new Runnable() {
-				@Override
-				public void run(){
-					instanceManager.registerTaskManager(instanceConnectionInfo, hardwareDescription,
-							numberOfSlots.getValue());
-				}
-			};
+		final Runnable cancelJobRunnable = new Runnable() {
+			@Override
+			public void run() {
+				eg.cancel();
+			}
+		};
 
-			this.executorService.execute(registerTaskManagerRunnable);
-			return new RegisterTaskManagerResult(RegisterTaskManagerResult.ReturnCode.SUCCESS);
-		}
+		eg.execute(cancelJobRunnable);
 
-		return new RegisterTaskManagerResult(RegisterTaskManagerResult.ReturnCode.FAILURE);
+		return new JobCancelResult(AbstractJobResult.ReturnCode.SUCCESS, null);
 	}
-
-
+	
 	@Override
-	public void updateTaskExecutionState(final TaskExecutionState executionState) throws IOException {
-
-		// Ignore calls with executionResult == null
-		if (executionState == null) {
-			LOG.error("Received call to updateTaskExecutionState with executionState == null");
-			return;
-		}
+	public void updateTaskExecutionState(TaskExecutionState executionState) throws IOException {
+		Preconditions.checkNotNull(executionState);
 
-		if (executionState.getExecutionState() == ExecutionState.FAILED) {
-			LOG.error(executionState.getDescription());
-		}
 
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(executionState.getJobID());
+		final ExecutionGraph eg = this.currentJobs.get(executionState.getJobID());
 		if (eg == null) {
 			LOG.error("Cannot find execution graph for ID " + executionState.getJobID() + " to change state to "
 				+ executionState.getExecutionState());
 			return;
 		}
 
-		final ExecutionVertex vertex = eg.getVertexByID(executionState.getID());
-		if (vertex == null) {
-			LOG.error("Cannot find vertex with ID " + executionState.getID() + " of job " + eg.getJobID()
-				+ " to change state to " + executionState.getExecutionState());
-			return;
-		}
-
-		// Asynchronously update execute state of vertex
-		vertex.updateExecutionStateAsynchronously(executionState.getExecutionState(), executionState.getDescription());
+		eg.updateState(executionState);
 	}
-
-
+	
 	@Override
-	public JobCancelResult cancelJob(final JobID jobID) throws IOException {
+	public InputSplit requestNextInputSplit(JobID jobID, JobVertexID vertexId) throws IOException {
 
-		LOG.info("Trying to cancel job with ID " + jobID);
-
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-		if (eg == null) {
-			return new JobCancelResult(ReturnCode.ERROR, "Cannot find job with ID " + jobID);
+		final ExecutionGraph graph = this.currentJobs.get(jobID);
+		if (graph == null) {
+			LOG.error("Cannot find execution graph to job ID " + jobID);
+			return null;
 		}
 
-		final Runnable cancelJobRunnable = new Runnable() {
-
-			@Override
-			public void run() {
-				eg.updateJobStatus(InternalJobStatus.CANCELING, "Job canceled by user");
-				final TaskCancelResult cancelResult = cancelJob(eg);
-				if (cancelResult != null) {
-					LOG.error(cancelResult.getDescription());
-				}
-			}
-		};
-
-		eg.executeCommand(cancelJobRunnable);
-
-		LOG.info("Cancel of job " + jobID + " successfully triggered");
+		final ExecutionJobVertex vertex = graph.getJobVertex(vertexId);
+		if (vertex == null) {
+			LOG.error("Cannot find execution vertex for vertex ID " + vertexId);
+			return null;
+		}
 
-		return new JobCancelResult(AbstractJobResult.ReturnCode.SUCCESS, null);
+		InputSplitAssigner splitAssigner = vertex.getSplitAssigner();
+		if (splitAssigner == null) {
+			LOG.error("No InputSplitAssigner for vertex ID " + vertexId);
+			return null;
+		}
+		
+		
+		return splitAssigner.getNextInputSplit(null);
 	}
+	
+	@Override
+	public void jobStatusHasChanged(ExecutionGraph executionGraph, JobStatus newJobStatus, String optionalMessage) {
 
-	/**
-	 * Cancels all the tasks in the current and upper stages of the
-	 * given execution graph.
-	 * 
-	 * @param eg
-	 *        the execution graph representing the job to cancel.
-	 * @return <code>null</code> if no error occurred during the cancel attempt,
-	 *         otherwise the returned object will describe the error
-	 */
-	private TaskCancelResult cancelJob(final ExecutionGraph eg) {
-
-		TaskCancelResult errorResult = null;
-
-		/**
-		 * Cancel all nodes in the current and upper execution stages.
-		 */
-		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(eg, eg.getIndexOfCurrentExecutionStage(),
-			false, true);
-		while (it.hasNext()) {
+		final JobID jid = executionGraph.getJobID();
+		
+		if (LOG.isInfoEnabled()) {
+			String message = optionalMessage == null ? "." : ": " + optionalMessage;
+			LOG.info(String.format("Status of job %s (%s) changed to %s%s", 
+					jid, executionGraph.getJobName(), newJobStatus, message));
+		}
 
-			final ExecutionVertex vertex = it.next();
-			final TaskCancelResult result = vertex.cancelTask();
-			if (result.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) {
-				errorResult = result;
+		// remove the job graph if the state is any terminal state
+		if (newJobStatus == JobStatus.FINISHED || newJobStatus == JobStatus.CANCELED || newJobStatus == JobStatus.FAILED) {
+			this.currentJobs.remove(jid);
+			
+			try {
+				LibraryCacheManager.unregister(jid);
+			}
+			catch (Throwable t) {
+				LOG.warn("Could not properly unregister job " + jid + " from the library cache.");
 			}
 		}
-
-		return errorResult;
 	}
 
-
 	@Override
 	public JobProgressResult getJobProgress(final JobID jobID) throws IOException {
 
 		if (this.eventCollector == null) {
-			return new JobProgressResult(ReturnCode.ERROR, "JobManager does not support progress reports for jobs",
-				null);
+			return new JobProgressResult(ReturnCode.ERROR, "JobManager does not support progress reports for jobs", null);
 		}
 
 		final SerializableArrayList<AbstractEvent> eventList = new SerializableArrayList<AbstractEvent>();
@@ -674,104 +498,32 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	@Override
 	public ConnectionInfoLookupResponse lookupConnectionInfo(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) {
 
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
+		final ExecutionGraph eg = this.currentJobs.get(jobID);
 		if (eg == null) {
 			LOG.error("Cannot find execution graph to job ID " + jobID);
 			return ConnectionInfoLookupResponse.createReceiverNotFound();
 		}
 
-		final InternalJobStatus jobStatus = eg.getJobStatus();
-		if (jobStatus == InternalJobStatus.FAILING || jobStatus == InternalJobStatus.CANCELING) {
-			return ConnectionInfoLookupResponse.createJobIsAborting();
-		}
-
-		final ExecutionEdge edge = eg.getEdgeByID(sourceChannelID);
-		if (edge == null) {
-			LOG.error("Cannot find execution edge associated with ID " + sourceChannelID);
-			return ConnectionInfoLookupResponse.createReceiverNotFound();
-		}
-
-		if (sourceChannelID.equals(edge.getInputChannelID())) {
-			// Request was sent from an input channel
+		return eg.lookupConnectionInfoAndDeployReceivers(caller, sourceChannelID);
+	}
 
-			final ExecutionVertex connectedVertex = edge.getOutputGate().getVertex();
-
-			final Instance assignedInstance = connectedVertex.getAllocatedResource().getInstance();
-			if (assignedInstance == null) {
-				LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getOutputGateIndex()
-					+ " but no instance assigned");
-				// LOG.info("Created receiverNotReady for " + connectedVertex + " 1");
-				return ConnectionInfoLookupResponse.createReceiverNotReady();
-			}
-
-			// Check execution state
-			final ExecutionState executionState = connectedVertex.getExecutionState();
-			if (executionState == ExecutionState.FINISHED) {
-				// that should not happen. if there is data pending, the receiver cannot be ready
-				return ConnectionInfoLookupResponse.createReceiverNotFound();
-			}
-
-			// running is common, finishing is happens when the lookup is for the close event
-			if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING) {
-				// LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2");
-				return ConnectionInfoLookupResponse.createReceiverNotReady();
-			}
-
-			if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
-				// Receiver runs on the same task manager
-				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getOutputChannelID());
-			} else {
-				// Receiver runs on a different task manager
-
-				final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
-				final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
-
-				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
-			}
-		}
-		// else, the request is for an output channel
-		// Find vertex of connected input channel
-		final ExecutionVertex targetVertex = edge.getInputGate().getVertex();
-
-		// Check execution state
-		final ExecutionState executionState = targetVertex.getExecutionState();
-
-		// check whether the task needs to be deployed
-		if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) {
-
-			if (executionState == ExecutionState.ASSIGNED) {
-				final Runnable command = new Runnable() {
-					@Override
-					public void run() {
-						scheduler.deployAssignedVertices(targetVertex);
-					}
-				};
-				eg.executeCommand(command);
-			}
-
-			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3");
-			return ConnectionInfoLookupResponse.createReceiverNotReady();
-		}
-
-		final Instance assignedInstance = targetVertex.getAllocatedResource().getInstance();
-		if (assignedInstance == null) {
-			LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getInputChannelID() + " but no instance assigned");
-			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4");
-			return ConnectionInfoLookupResponse.createReceiverNotReady();
-		}
-
-		if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
-			// Receiver runs on the same task manager
-			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelID());
-		} else {
-			// Receiver runs on a different task manager
-			final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
-			final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
-
-			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
-		}
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Tests whether the job manager has been shut down completely.
+	 * 
+	 * @return <code>true</code> if the job manager has been shut down completely, <code>false</code> otherwise
+	 */
+	public boolean isShutDown() {
+		return this.isShutDown;
 	}
-
+	
+	public InstanceManager getInstanceManager() {
+		return this.instanceManager;
+	}
+	
 	/**
 	 * Returns current ManagementGraph from eventCollector and, if not current, from archive
 	 * 
@@ -828,239 +580,10 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		return eventList;
 	}
 
-
-	@Override
-	public void killTask(final JobID jobID, final ManagementVertexID id) throws IOException {
-
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-		if (eg == null) {
-			LOG.error("Cannot find execution graph for job " + jobID);
-			return;
-		}
-
-		final ExecutionVertex vertex = eg.getVertexByID(ExecutionVertexID.fromManagementVertexID(id));
-		if (vertex == null) {
-			LOG.error("Cannot find execution vertex with ID " + id);
-			return;
-		}
-
-		LOG.info("Killing task " + vertex + " of job " + jobID);
-
-		final Runnable runnable = new Runnable() {
-
-			@Override
-			public void run() {
-
-				final TaskKillResult result = vertex.killTask();
-				if (result.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) {
-					LOG.error(result.getDescription());
-				}
-			}
-		};
-
-		eg.executeCommand(runnable);
-	}
-
-	/**
-	 * Tests whether the job manager has been shut down completely.
-	 * 
-	 * @return <code>true</code> if the job manager has been shut down completely, <code>false</code> otherwise
-	 */
-	public boolean isShutDown() {
-		return this.isShutDown;
-	}
-
-
-
-	@Override
-	public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
-			final String optionalMessage) {
-
-		LOG.info("Status of job " + executionGraph.getJobName() + "(" + executionGraph.getJobID() + ")"
-			+ " changed to " + newJobStatus);
-
-		if (newJobStatus == InternalJobStatus.FAILING) {
-
-			// Cancel all remaining tasks
-			cancelJob(executionGraph);
-		}
-
-		if (newJobStatus == InternalJobStatus.CANCELED || newJobStatus == InternalJobStatus.FAILED
-			|| newJobStatus == InternalJobStatus.FINISHED) {
-			// Unregister job for Nephele's monitoring, optimization components, and dynamic input split assignment
-			unregisterJob(executionGraph);
-		}
-	}
-
-
-	@Override
-	public void logBufferUtilization(final JobID jobID) throws IOException {
-
-		final ExecutionGraph eg = this.scheduler.getExecutionGraphByID(jobID);
-		if (eg == null) {
-			return;
-		}
-
-		final Set<Instance> allocatedInstance = new HashSet<Instance>();
-
-		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(eg, true);
-		while (it.hasNext()) {
-
-			final ExecutionVertex vertex = it.next();
-			final ExecutionState state = vertex.getExecutionState();
-			if (state == ExecutionState.RUNNING || state == ExecutionState.FINISHING) {
-				final Instance instance = vertex.getAllocatedResource().getInstance();
-
-				if (instance instanceof DummyInstance) {
-					LOG.error("Found instance of type DummyInstance for vertex " + vertex.getName() + " (state "
-						+ state + ")");
-					continue;
-				}
-
-				allocatedInstance.add(instance);
-			}
-		}
-
-		// Send requests to task managers from separate thread
-		final Runnable requestRunnable = new Runnable() {
-
-			@Override
-			public void run() {
-
-				final Iterator<Instance> it2 = allocatedInstance.iterator();
-
-				try {
-					while (it2.hasNext()) {
-						it2.next().logBufferUtilization();
-					}
-				} catch (IOException ioe) {
-					LOG.error(ioe);
-				}
-
-			}
-		};
-
-		// Hand over to the executor service
-		this.executorService.execute(requestRunnable);
-	}
-
 	@Override
 	public int getAvailableSlots() {
 		return getInstanceManager().getTotalNumberOfSlots();
 	}
-
-
-	@Override
-	public void deploy(final JobID jobID, final Instance instance,
-			final List<ExecutionVertex> verticesToBeDeployed) {
-
-		if (verticesToBeDeployed.isEmpty()) {
-			LOG.error("Method 'deploy' called but list of vertices to be deployed is empty");
-			return;
-		}
-
-		for (final ExecutionVertex vertex : verticesToBeDeployed) {
-
-			// Check vertex state
-			if (vertex.getExecutionState() != ExecutionState.READY) {
-				LOG.error("Expected vertex " + vertex + " to be in state READY but it is in state "
-					+ vertex.getExecutionState());
-			}
-
-			vertex.updateExecutionState(ExecutionState.STARTING, null);
-		}
-
-		// Create a new runnable and pass it the executor service
-		final Runnable deploymentRunnable = new Runnable() {
-
-			/**
-			 * {@inheritDoc}
-			 */
-			@Override
-			public void run() {
-
-				// Check if all required libraries are available on the instance
-				try {
-					instance.checkLibraryAvailability(jobID);
-				} catch (IOException ioe) {
-					LOG.error("Cannot check library availability: " + StringUtils.stringifyException(ioe));
-				}
-
-				final List<TaskDeploymentDescriptor> submissionList = new SerializableArrayList<TaskDeploymentDescriptor>();
-
-				// Check the consistency of the call
-				for (final ExecutionVertex vertex : verticesToBeDeployed) {
-
-					submissionList.add(vertex.constructDeploymentDescriptor());
-
-					LOG.info("Starting task " + vertex + " on " + vertex.getAllocatedResource().getInstance());
-				}
-
-				List<TaskSubmissionResult> submissionResultList = null;
-
-				try {
-					submissionResultList = instance.submitTasks(submissionList);
-				} catch (final IOException ioe) {
-					final String errorMsg = StringUtils.stringifyException(ioe);
-					for (final ExecutionVertex vertex : verticesToBeDeployed) {
-						vertex.updateExecutionStateAsynchronously(ExecutionState.FAILED, errorMsg);
-					}
-				}
-
-				if (verticesToBeDeployed.size() != submissionResultList.size()) {
-					LOG.error("size of submission result list does not match size of list with vertices to be deployed");
-				}
-
-				int count = 0;
-				for (final TaskSubmissionResult tsr : submissionResultList) {
-
-					ExecutionVertex vertex = verticesToBeDeployed.get(count++);
-					if (!vertex.getID().equals(tsr.getVertexID())) {
-						LOG.error("Expected different order of objects in task result list");
-						vertex = null;
-						for (final ExecutionVertex candVertex : verticesToBeDeployed) {
-							if (tsr.getVertexID().equals(candVertex.getID())) {
-								vertex = candVertex;
-								break;
-							}
-						}
-
-						if (vertex == null) {
-							LOG.error("Cannot find execution vertex for vertex ID " + tsr.getVertexID());
-							continue;
-						}
-					}
-
-					if (tsr.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS) {
-						// Change the execution state to failed and let the scheduler deal with the rest
-						vertex.updateExecutionStateAsynchronously(ExecutionState.FAILED, tsr.getDescription());
-					}
-				}
-			}
-		};
-
-		this.executorService.execute(deploymentRunnable);
-	}
-
-
-	@Override
-	public InputSplitWrapper requestNextInputSplit(final JobID jobID, final ExecutionVertexID vertexID,
-			final IntegerRecord sequenceNumber) throws IOException {
-
-		final ExecutionGraph graph = this.scheduler.getExecutionGraphByID(jobID);
-		if (graph == null) {
-			LOG.error("Cannot find execution graph to job ID " + jobID);
-			return null;
-		}
-
-		final ExecutionVertex vertex = graph.getVertexByID(vertexID);
-		if (vertex == null) {
-			LOG.error("Cannot find execution vertex for vertex ID " + vertexID);
-			return null;
-		}
-
-		return new InputSplitWrapper(jobID, this.inputSplitManager.getNextInputSplit(vertex, sequenceNumber.getValue()));
-	}
 	
 	/**
 	 * Starts the Jetty Infoserver for the Jobmanager
@@ -1081,17 +604,11 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	}
 	
 	
-	// TODO Add to RPC?
 	public List<RecentJobEvent> getOldJobs() throws IOException {
-
-		//final List<RecentJobEvent> eventList = new SerializableArrayList<RecentJobEvent>();
-
 		if (this.archive == null) {
 			throw new IOException("No instance of the event collector found");
 		}
 
-		//this.eventCollector.getRecentJobs(eventList);
-
 		return this.archive.getJobs();
 	}
 	
@@ -1103,8 +620,8 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		return this.instanceManager.getNumberOfRegisteredTaskManagers();
 	}
 	
-	public Map<InstanceConnectionInfo, Instance> getInstances() {
-		return this.instanceManager.getInstances();
+	public Map<InstanceID, Instance> getInstances() {
+		return this.instanceManager.getAllRegisteredInstances();
 	}
 
 	@Override
@@ -1120,4 +637,118 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	public Map<String, Accumulator<?, ?>> getAccumulators(JobID jobID) {
 		return this.accumulatorManager.getJobAccumulators(jobID);
 	}
+	
+	public Map<JobID, ExecutionGraph> getCurrentJobs() {
+		return Collections.unmodifiableMap(currentJobs);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  TaskManager to JobManager communication
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public boolean sendHeartbeat(InstanceID taskManagerId) {
+		return this.instanceManager.reportHeartBeat(taskManagerId);
+	}
+
+	@Override
+	public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) {
+		return this.instanceManager.registerTaskManager(instanceConnectionInfo, hardwareDescription, numberOfSlots);
+	}
+	
+	
+	// --------------------------------------------------------------------------------------------
+	//  Executable
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Entry point for the program
+	 * 
+	 * @param args
+	 *        arguments from the command line
+	 */
+	
+	public static void main(String[] args) {
+		// determine if a valid log4j config exists and initialize a default logger if not
+		if (System.getProperty("log4j.configuration") == null) {
+			Logger root = Logger.getRootLogger();
+			root.removeAllAppenders();
+			PatternLayout layout = new PatternLayout("%d{HH:mm:ss,SSS} %-5p %-60c %x - %m%n");
+			ConsoleAppender appender = new ConsoleAppender(layout, "System.err");
+			root.addAppender(appender);
+			root.setLevel(Level.INFO);
+		}
+		
+		JobManager jobManager;
+		try {
+			jobManager = initialize(args);
+			// Start info server for jobmanager
+			jobManager.startInfoServer();
+		}
+		catch (Exception e) {
+			LOG.fatal(e.getMessage(), e);
+			System.exit(FAILURE_RETURN_CODE);
+		}
+		
+		// Clean up is triggered through a shutdown hook
+		// freeze this thread to keep the JVM alive (the job manager threads are daemon threads)
+		Object w = new Object();
+		synchronized (w) {
+			try {
+				w.wait();
+			} catch (InterruptedException e) {}
+		}
+	}
+	
+	@SuppressWarnings("static-access")
+	public static JobManager initialize(String[] args) throws Exception {
+		final Option configDirOpt = OptionBuilder.withArgName("config directory").hasArg()
+			.withDescription("Specify configuration directory.").create("configDir");
+
+		final Option executionModeOpt = OptionBuilder.withArgName("execution mode").hasArg()
+			.withDescription("Specify execution mode.").create("executionMode");
+
+		final Options options = new Options();
+		options.addOption(configDirOpt);
+		options.addOption(executionModeOpt);
+
+		CommandLineParser parser = new GnuParser();
+		CommandLine line = null;
+		try {
+			line = parser.parse(options, args);
+		} catch (ParseException e) {
+			LOG.error("CLI Parsing failed. Reason: " + e.getMessage());
+			System.exit(FAILURE_RETURN_CODE);
+		}
+
+		final String configDir = line.getOptionValue(configDirOpt.getOpt(), null);
+		final String executionModeName = line.getOptionValue(executionModeOpt.getOpt(), "local");
+		
+		ExecutionMode executionMode = null;
+		if ("local".equals(executionModeName)) {
+			executionMode = ExecutionMode.LOCAL;
+		} else if ("cluster".equals(executionModeName)) {
+			executionMode = ExecutionMode.CLUSTER;
+		} else {
+			System.err.println("Unrecognized execution mode: " + executionModeName);
+			System.exit(FAILURE_RETURN_CODE);
+		}
+		
+		// print some startup environment info, like user, code revision, etc
+		EnvironmentInformation.logEnvironmentInfo(LOG, "JobManager");
+		
+		// First, try to load global configuration
+		GlobalConfiguration.loadConfiguration(configDir);
+
+		// Create a new job manager object
+		JobManager jobManager = new JobManager(executionMode);
+		
+		// Set base dir for info server
+		Configuration infoserverConfig = GlobalConfiguration.getConfiguration();
+		if (configDir != null && new File(configDir).isDirectory()) {
+			infoserverConfig.setString(ConfigConstants.FLINK_BASE_DIR_PATH_KEY, configDir+"/..");
+		}
+		GlobalConfiguration.includeConfiguration(infoserverConfig);
+		return jobManager;
+	}
 }


[38/63] [abbrv] Finalize ExecutionGraph state machine and calls

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index adc0f09..bbc0c97 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -16,992 +16,388 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.executiongraph;
 
-import java.io.IOException;
+import static org.apache.flink.runtime.execution.ExecutionState.CANCELED;
+import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
+import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
+
 import java.util.ArrayList;
-import java.util.Iterator;
+import java.util.HashSet;
 import java.util.List;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
 
 import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.runtime.deployment.ChannelDeploymentDescriptor;
+
+import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionListener;
 import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.execution.ExecutionStateTransition;
-import org.apache.flink.runtime.instance.AllocatedResource;
-import org.apache.flink.runtime.instance.AllocationID;
-import org.apache.flink.runtime.io.network.gates.GateID;
-import org.apache.flink.runtime.taskmanager.TaskOperationResult;
-import org.apache.flink.runtime.taskmanager.TaskCancelResult;
-import org.apache.flink.runtime.taskmanager.TaskKillResult;
-import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
-import org.apache.flink.runtime.taskmanager.TaskOperationResult.ReturnCode;
-import org.apache.flink.runtime.util.AtomicEnum;
-import org.apache.flink.runtime.util.SerializableArrayList;
-import org.apache.flink.util.StringUtils;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 
 /**
- * An execution vertex represents an instance of a task in a Nephele job. An execution vertex
- * is initially created from a job vertex and always belongs to exactly one group vertex.
- * It is possible to duplicate execution vertices in order to distribute a task to several different
- * task managers and process the task in parallel.
- * <p>
- * This class is thread-safe.
- * 
+ * The ExecutionVertex is a parallel subtask of the execution. It may be executed once, or several times, each of
+ * which time it spawns an {@link Execution}.
  */
-public final class ExecutionVertex {
-
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(ExecutionVertex.class);
-
-	/**
-	 * The ID of the vertex.
-	 */
-	private final ExecutionVertexID vertexID;
-
-	/**
-	 * The group vertex this vertex belongs to.
-	 */
-	private final ExecutionGroupVertex groupVertex;
-
-	/**
-	 * The execution graph is vertex belongs to.
-	 */
-	private final ExecutionGraph executionGraph;
-
-	/**
-	 * The allocated resources assigned to this vertex.
-	 */
-	private final AtomicReference<AllocatedResource> allocatedResource = new AtomicReference<AllocatedResource>(null);
-
-	/**
-	 * The allocation ID identifying the allocated resources used by this vertex
-	 * within the instance.
-	 */
-	private volatile AllocationID allocationID = null;
-
-	/**
-	 * A list of {@link VertexAssignmentListener} objects to be notified about changes in the instance assignment.
-	 */
-	private final CopyOnWriteArrayList<VertexAssignmentListener> vertexAssignmentListeners = new CopyOnWriteArrayList<VertexAssignmentListener>();
-
-	/**
-	 * A map of {@link ExecutionListener} objects to be notified about the state changes of a vertex.
-	 */
-	private final ConcurrentMap<Integer, ExecutionListener> executionListeners = new ConcurrentSkipListMap<Integer, ExecutionListener>();
-
-	/**
-	 * The current execution state of the task represented by this vertex
-	 */
-	private final AtomicEnum<ExecutionState> executionState = new AtomicEnum<ExecutionState>(ExecutionState.CREATED);
-
-	/**
-	 * The output gates attached to this vertex.
-	 */
-	private final ExecutionGate[] outputGates;
-
-	/**
-	 * The input gates attached to his vertex.
-	 */
-	private final ExecutionGate[] inputGates;
-
-	/**
-	 * The index of this vertex in the vertex group.
-	 */
-	private volatile int indexInVertexGroup = 0;
-
-	/**
-	 * Stores the number of times the vertex may be still be started before the corresponding task is considered to be
-	 * failed.
-	 */
-	private final AtomicInteger retriesLeft;
-
-
-	/**
-	 * The execution pipeline this vertex is part of.
-	 */
-	private final AtomicReference<ExecutionPipeline> executionPipeline = new AtomicReference<ExecutionPipeline>(null);
-
-	/**
-	 * Flag to indicate whether the vertex has been requested to cancel while in state STARTING
-	 */
-	private final AtomicBoolean cancelRequested = new AtomicBoolean(false);
-
-	/**
-	 * Create a new execution vertex and instantiates its environment.
-	 * 
-	 * @param executionGraph
-	 *        the execution graph the new vertex belongs to
-	 * @param groupVertex
-	 *        the group vertex the new vertex belongs to
-	 * @param numberOfOutputGates
-	 *        the number of output gates attached to this vertex
-	 * @param numberOfInputGates
-	 *        the number of input gates attached to this vertex
-	 */
-	public ExecutionVertex(final ExecutionGraph executionGraph, final ExecutionGroupVertex groupVertex,
-			final int numberOfOutputGates, final int numberOfInputGates) {
-		this(new ExecutionVertexID(), executionGraph, groupVertex, numberOfOutputGates, numberOfInputGates);
-
-		this.groupVertex.addInitialSubtask(this);
-	}
-
-	/**
-	 * Private constructor used to duplicate execution vertices.
-	 * 
-	 * @param vertexID
-	 *        the ID of the new execution vertex.
-	 * @param executionGraph
-	 *        the execution graph the new vertex belongs to
-	 * @param groupVertex
-	 *        the group vertex the new vertex belongs to
-	 * @param numberOfOutputGates
-	 *        the number of output gates attached to this vertex
-	 * @param numberOfInputGates
-	 *        the number of input gates attached to this vertex
-	 */
-	private ExecutionVertex(final ExecutionVertexID vertexID, final ExecutionGraph executionGraph,
-			final ExecutionGroupVertex groupVertex, final int numberOfOutputGates, final int numberOfInputGates) {
-
-		this.vertexID = vertexID;
-		this.executionGraph = executionGraph;
-		this.groupVertex = groupVertex;
-
-		this.retriesLeft = new AtomicInteger(groupVertex.getNumberOfExecutionRetries());
-
-		this.outputGates = new ExecutionGate[numberOfOutputGates];
-		this.inputGates = new ExecutionGate[numberOfInputGates];
+public class ExecutionVertex {
 
-		// Register vertex with execution graph
-		this.executionGraph.registerExecutionVertex(this);
-
-		// Register the vertex itself as a listener for state changes
-		registerExecutionListener(this.executionGraph);
-	}
-
-	/**
-	 * Returns the group vertex this execution vertex belongs to.
-	 * 
-	 * @return the group vertex this execution vertex belongs to
-	 */
-	public ExecutionGroupVertex getGroupVertex() {
-		return this.groupVertex;
-	}
+	@SuppressWarnings("unused")
+	private static final Logger LOG = ExecutionGraph.LOG;
+	
+	private static final int MAX_DISTINCT_LOCATIONS_TO_CONSIDER = 8;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private final ExecutionJobVertex jobVertex;
+	
+	private final IntermediateResultPartition[] resultPartitions;
+	
+	private final ExecutionEdge[][] inputEdges;
+	
+	private final int subTaskIndex;
+	
+	private final List<Execution> priorExecutions;
+	
+	private volatile Execution currentExecution;	// this field must never be null
+	
+	// --------------------------------------------------------------------------------------------
 
-	/**
-	 * Returns the name of the execution vertex.
-	 * 
-	 * @return the name of the execution vertex
-	 */
-	public String getName() {
-		return this.groupVertex.getName();
+	public ExecutionVertex(ExecutionJobVertex jobVertex, int subTaskIndex, IntermediateResult[] producedDataSets) {
+		this(jobVertex, subTaskIndex, producedDataSets, System.currentTimeMillis());
 	}
-
-	/**
-	 * Returns a duplicate of this execution vertex.
-	 * 
-	 * @param preserveVertexID
-	 *        <code>true</code> to copy the vertex's ID to the duplicated vertex, <code>false</code> to create a new ID
-	 * @return a duplicate of this execution vertex
-	 */
-	public ExecutionVertex duplicateVertex(final boolean preserveVertexID) {
-
-		ExecutionVertexID newVertexID;
-		if (preserveVertexID) {
-			newVertexID = this.vertexID;
-		} else {
-			newVertexID = new ExecutionVertexID();
-		}
-
-		final ExecutionVertex duplicatedVertex = new ExecutionVertex(newVertexID, this.executionGraph,
-			this.groupVertex, this.outputGates.length, this.inputGates.length);
-
-		// Duplicate gates
-		for (int i = 0; i < this.outputGates.length; ++i) {
-			duplicatedVertex.outputGates[i] = new ExecutionGate(new GateID(), duplicatedVertex,
-				this.outputGates[i].getGroupEdge(), false);
-		}
-
-		for (int i = 0; i < this.inputGates.length; ++i) {
-			duplicatedVertex.inputGates[i] = new ExecutionGate(new GateID(), duplicatedVertex,
-				this.inputGates[i].getGroupEdge(), true);
+	
+	public ExecutionVertex(ExecutionJobVertex jobVertex, int subTaskIndex, IntermediateResult[] producedDataSets, long createTimestamp) {
+		this.jobVertex = jobVertex;
+		this.subTaskIndex = subTaskIndex;
+		
+		this.resultPartitions = new IntermediateResultPartition[producedDataSets.length];
+		for (int i = 0; i < producedDataSets.length; i++) {
+			IntermediateResultPartition irp = new IntermediateResultPartition(producedDataSets[i], this, subTaskIndex);
+			this.resultPartitions[i] = irp;
+			producedDataSets[i].setPartition(subTaskIndex, irp);
 		}
-
-		// TODO set new profiling record with new vertex id
-		duplicatedVertex.setAllocatedResource(this.allocatedResource.get());
-
-		return duplicatedVertex;
+		
+		this.inputEdges = new ExecutionEdge[jobVertex.getJobVertex().getInputs().size()][];
+		this.priorExecutions = new CopyOnWriteArrayList<Execution>();
+		
+		this.currentExecution = new Execution(this, 0, createTimestamp);
 	}
-
-	/**
-	 * Inserts the output gate at the given position.
-	 * 
-	 * @param pos
-	 *        the position to insert the output gate
-	 * @param outputGate
-	 *        the output gate to be inserted
-	 */
-	void insertOutputGate(final int pos, final ExecutionGate outputGate) {
-
-		if (this.outputGates[pos] != null) {
-			throw new IllegalStateException("Output gate at position " + pos + " is not null");
-		}
-
-		this.outputGates[pos] = outputGate;
+	
+	
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
+	
+	public JobID getJobId() {
+		return this.jobVertex.getJobId();
 	}
-
-	/**
-	 * Inserts the input gate at the given position.
-	 * 
-	 * @param pos
-	 *        the position to insert the input gate
-	 * @param outputGate
-	 *        the input gate to be inserted
-	 */
-	void insertInputGate(final int pos, final ExecutionGate inputGate) {
-
-		if (this.inputGates[pos] != null) {
-			throw new IllegalStateException("Input gate at position " + pos + " is not null");
-		}
-
-		this.inputGates[pos] = inputGate;
+	
+	public ExecutionJobVertex getJobVertex() {
+		return jobVertex;
 	}
-
-	/**
-	 * Returns a duplicate of this execution vertex. The duplicated vertex receives
-	 * a new vertex ID.
-	 * 
-	 * @return a duplicate of this execution vertex.
-	 */
-	public ExecutionVertex splitVertex() {
-
-		return duplicateVertex(false);
+	
+	public JobVertexID getJobvertexId() {
+		return this.jobVertex.getJobVertexId();
 	}
-
-	/**
-	 * Returns this execution vertex's current execution status.
-	 * 
-	 * @return this execution vertex's current execution status
-	 */
-	public ExecutionState getExecutionState() {
-		return this.executionState.get();
+	
+	public String getTaskName() {
+		return this.jobVertex.getJobVertex().getName();
 	}
-
-	/**
-	 * Updates the vertex's current execution state through the job's executor service.
-	 * 
-	 * @param newExecutionState
-	 *        the new execution state
-	 * @param optionalMessage
-	 *        an optional message related to the state change
-	 */
-	public void updateExecutionStateAsynchronously(final ExecutionState newExecutionState,
-			final String optionalMessage) {
-
-		final Runnable command = new Runnable() {
-
-			/**
-			 * {@inheritDoc}
-			 */
-			@Override
-			public void run() {
-
-				updateExecutionState(newExecutionState, optionalMessage);
-			}
-		};
-
-		this.executionGraph.executeCommand(command);
+	
+	public int getTotalNumberOfParallelSubtasks() {
+		return this.jobVertex.getParallelism();
 	}
-
-	/**
-	 * Updates the vertex's current execution state through the job's executor service.
-	 * 
-	 * @param newExecutionState
-	 *        the new execution state
-	 */
-	public void updateExecutionStateAsynchronously(final ExecutionState newExecutionState) {
-
-		updateExecutionStateAsynchronously(newExecutionState, null);
+	
+	public int getParallelSubtaskIndex() {
+		return this.subTaskIndex;
 	}
-
-	/**
-	 * Updates the vertex's current execution state.
-	 * 
-	 * @param newExecutionState
-	 *        the new execution state
-	 */
-	public ExecutionState updateExecutionState(final ExecutionState newExecutionState) {
-		return updateExecutionState(newExecutionState, null);
+	
+	public int getNumberOfInputs() {
+		return this.inputEdges.length;
 	}
-
-	/**
-	 * Updates the vertex's current execution state.
-	 * 
-	 * @param newExecutionState
-	 *        the new execution state
-	 * @param optionalMessage
-	 *        an optional message related to the state change
-	 */
-	public ExecutionState updateExecutionState(ExecutionState newExecutionState, final String optionalMessage) {
-
-		if (newExecutionState == null) {
-			throw new IllegalArgumentException("Argument newExecutionState must not be null");
-		}
-
-		final ExecutionState currentExecutionState = this.executionState.get();
-		if (currentExecutionState == ExecutionState.CANCELING) {
-
-			// If we are in CANCELING, ignore state changes to FINISHING
-			if (newExecutionState == ExecutionState.FINISHING) {
-				return currentExecutionState;
-			}
-
-			// Rewrite FINISHED to CANCELED if the task has been marked to be canceled
-			if (newExecutionState == ExecutionState.FINISHED) {
-				LOG.info("Received transition from CANCELING to FINISHED for vertex " + toString()
-					+ ", converting it to CANCELED");
-				newExecutionState = ExecutionState.CANCELED;
-			}
-		}
-
-		// Check and save the new execution state
-		final ExecutionState previousState = this.executionState.getAndSet(newExecutionState);
-		if (previousState == newExecutionState) {
-			return previousState;
-		}
-
-		// Check the transition
-		ExecutionStateTransition.checkTransition(true, toString(), previousState, newExecutionState);
-
-		// Notify the listener objects
-		final Iterator<ExecutionListener> it = this.executionListeners.values().iterator();
-		while (it.hasNext()) {
-			it.next().executionStateChanged(this.executionGraph.getJobID(), this.vertexID, newExecutionState,
-				optionalMessage);
+	
+	public ExecutionEdge[] getInputEdges(int input) {
+		if (input < 0 || input >= this.inputEdges.length) {
+			throw new IllegalArgumentException(String.format("Input %d is out of range [0..%d)", input, this.inputEdges.length));
 		}
-
-		// The vertex was requested to be canceled by another thread
-		checkCancelRequestedFlag();
-
-		return previousState;
+		return inputEdges[input];
 	}
-
-	public boolean compareAndUpdateExecutionState(final ExecutionState expected, final ExecutionState update) {
-
-		if (update == null) {
-			throw new IllegalArgumentException("Argument update must not be null");
-		}
-
-		if (!this.executionState.compareAndSet(expected, update)) {
-			return false;
-		}
-
-		// Check the transition
-		ExecutionStateTransition.checkTransition(true, toString(), expected, update);
-
-		// Notify the listener objects
-		final Iterator<ExecutionListener> it = this.executionListeners.values().iterator();
-		while (it.hasNext()) {
-			it.next().executionStateChanged(this.executionGraph.getJobID(), this.vertexID, update,
-				null);
-		}
-
-		// Check if the vertex was requested to be canceled by another thread
-		checkCancelRequestedFlag();
-
-		return true;
+	
+	public Execution getCurrentExecutionAttempt() {
+		return currentExecution;
 	}
-
-	/**
-	 * Checks if another thread requested the vertex to cancel while it was in state STARTING. If so, the method clears
-	 * the respective flag and repeats the cancel request.
-	 */
-	private void checkCancelRequestedFlag() {
-
-		if (this.cancelRequested.compareAndSet(true, false)) {
-			final TaskCancelResult tsr = cancelTask();
-			if (tsr.getReturnCode() != AbstractTaskResult.TaskOperationResult.SUCCESS
-				&& tsr.getReturnCode() != AbstractTaskResult.TaskOperationResult.TASK_NOT_FOUND) {
-				LOG.error("Unable to cancel vertex " + this + ": " + tsr.getReturnCode().toString()
-					+ ((tsr.getDescription() != null) ? (" (" + tsr.getDescription() + ")") : ""));
-			}
-		}
+	
+	public ExecutionState getExecutionState() {
+		return currentExecution.getState();
 	}
-
-	/**
-	 * Assigns the execution vertex with an {@link AllocatedResource}.
-	 * 
-	 * @param allocatedResource
-	 *        the resources which are supposed to be allocated to this vertex
-	 */
-	public void setAllocatedResource(final AllocatedResource allocatedResource) {
-
-		if (allocatedResource == null) {
-			throw new IllegalArgumentException("Argument allocatedResource must not be null");
-		}
-
-		final AllocatedResource previousResource = this.allocatedResource.getAndSet(allocatedResource);
-		if (previousResource != null) {
-			previousResource.removeVertexFromResource(this);
-		}
-
-		allocatedResource.assignVertexToResource(this);
-
-		// Notify all listener objects
-		final Iterator<VertexAssignmentListener> it = this.vertexAssignmentListeners.iterator();
-		while (it.hasNext()) {
-			it.next().vertexAssignmentChanged(this.vertexID, allocatedResource);
-		}
+	
+	public long getStateTimestamp(ExecutionState state) {
+		return currentExecution.getStateTimestamp(state);
 	}
-
-	/**
-	 * Returns the allocated resources assigned to this execution vertex.
-	 * 
-	 * @return the allocated resources assigned to this execution vertex
-	 */
-	public AllocatedResource getAllocatedResource() {
-
-		return this.allocatedResource.get();
+	
+	public Throwable getFailureCause() {
+		return currentExecution.getFailureCause();
 	}
-
-	/**
-	 * Returns the allocation ID which identifies the resources used
-	 * by this vertex within the assigned instance.
-	 * 
-	 * @return the allocation ID which identifies the resources used
-	 *         by this vertex within the assigned instance or <code>null</code> if the instance is still assigned to a
-	 *         {@link org.apache.flink.runtime.instance.DummyInstance}.
-	 */
-	public AllocationID getAllocationID() {
-		return this.allocationID;
+	
+	public AllocatedSlot getCurrentAssignedResource() {
+		return currentExecution.getAssignedResource();
 	}
-
-	/**
-	 * Returns the ID of this execution vertex.
-	 * 
-	 * @return the ID of this execution vertex
-	 */
-	public ExecutionVertexID getID() {
-		return this.vertexID;
+	
+	public ExecutionGraph getExecutionGraph() {
+		return this.jobVertex.getGraph();
 	}
-
-	/**
-	 * Returns the number of predecessors, i.e. the number of vertices
-	 * which connect to this vertex.
-	 * 
-	 * @return the number of predecessors
-	 */
-	public int getNumberOfPredecessors() {
-
-		int numberOfPredecessors = 0;
-
-		for (int i = 0; i < this.inputGates.length; ++i) {
-			numberOfPredecessors += this.inputGates[i].getNumberOfEdges();
+	
+	// --------------------------------------------------------------------------------------------
+	//  Graph building
+	// --------------------------------------------------------------------------------------------
+	
+	public void connectSource(int inputNumber, IntermediateResult source, JobEdge edge, int consumerNumber) {
+		
+		final DistributionPattern pattern = edge.getDistributionPattern();
+		final IntermediateResultPartition[] sourcePartitions = source.getPartitions();
+		
+		ExecutionEdge[] edges = null;
+		
+		switch (pattern) {
+			case POINTWISE:
+				edges = connectPointwise(sourcePartitions, inputNumber);
+				break;
+				
+			case BIPARTITE: 
+				edges = connectAllToAll(sourcePartitions, inputNumber);
+				break;
+				
+			default:
+				throw new RuntimeException("Unrecognized distribution pattern.");
+		
 		}
-
-		return numberOfPredecessors;
-	}
-
-	/**
-	 * Returns the number of successors, i.e. the number of vertices
-	 * this vertex is connected to.
-	 * 
-	 * @return the number of successors
-	 */
-	public int getNumberOfSuccessors() {
-
-		int numberOfSuccessors = 0;
-
-		for (int i = 0; i < this.outputGates.length; ++i) {
-			numberOfSuccessors += this.outputGates[i].getNumberOfEdges();
+		
+		this.inputEdges[inputNumber] = edges;
+		
+		ExecutionGraph graph = getExecutionGraph();
+		
+		// add the consumers to the source
+		// for now (until the receiver initiated handshake is in place), we need to register the 
+		// edges as the execution graph
+		for (ExecutionEdge ee : edges) {
+			ee.getSource().addConsumer(ee, consumerNumber);
+			graph.registerExecutionEdge(ee);
 		}
-
-		return numberOfSuccessors;
 	}
-
-	public ExecutionVertex getPredecessor(int index) {
-
-		if (index < 0) {
-			throw new IllegalArgumentException("Argument index must be greather or equal to 0");
-		}
-
-		for (int i = 0; i < this.inputGates.length; ++i) {
-
-			final ExecutionGate inputGate = this.inputGates[i];
-			final int numberOfEdges = inputGate.getNumberOfEdges();
-
-			if (index >= 0 && index < numberOfEdges) {
-
-				final ExecutionEdge edge = inputGate.getEdge(index);
-				return edge.getOutputGate().getVertex();
-			}
-			index -= numberOfEdges;
+	
+	private ExecutionEdge[] connectAllToAll(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
+		ExecutionEdge[] edges = new ExecutionEdge[sourcePartitions.length];
+		
+		for (int i = 0; i < sourcePartitions.length; i++) {
+			IntermediateResultPartition irp = sourcePartitions[i];
+			edges[i] = new ExecutionEdge(irp, this, inputNumber);
 		}
-
-		return null;
+		
+		return edges;
 	}
-
-	public ExecutionVertex getSuccessor(int index) {
-
-		if (index < 0) {
-			throw new IllegalArgumentException("Argument index must be greather or equal to 0");
-		}
-
-		for (int i = 0; i < this.outputGates.length; ++i) {
-
-			final ExecutionGate outputGate = this.outputGates[i];
-			final int numberOfEdges = outputGate.getNumberOfEdges();
-
-			if (index >= 0 && index < numberOfEdges) {
-
-				final ExecutionEdge edge = outputGate.getEdge(index);
-				return edge.getInputGate().getVertex();
+	
+	private ExecutionEdge[] connectPointwise(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
+		final int numSources = sourcePartitions.length;
+		final int parallelism = getTotalNumberOfParallelSubtasks();
+		
+		// simple case same number of sources as targets
+		if (numSources == parallelism) {
+			return new ExecutionEdge[] { new ExecutionEdge(sourcePartitions[subTaskIndex], this, inputNumber) };
+		}
+		else if (numSources < parallelism) {
+			
+			int sourcePartition;
+			
+			// check if the pattern is regular or irregular
+			// we use int arithmetics for regular, and floating point with rounding for irregular
+			if (parallelism % numSources == 0) {
+				// same number of targets per source
+				int factor = parallelism / numSources;
+				sourcePartition = subTaskIndex / factor;
+			}
+			else {
+				// different number of targets per source
+				float factor = ((float) parallelism) / numSources;
+				sourcePartition = (int) (subTaskIndex / factor);
+			}
+			
+			return new ExecutionEdge[] { new ExecutionEdge(sourcePartitions[sourcePartition], this, inputNumber) };
+		}
+		else {
+			if (numSources % parallelism == 0) {
+				// same number of targets per source
+				int factor = numSources / parallelism;
+				int startIndex = subTaskIndex * factor;
+				
+				ExecutionEdge[] edges = new ExecutionEdge[factor];
+				for (int i = 0; i < factor; i++) {
+					edges[i] = new ExecutionEdge(sourcePartitions[startIndex + i], this, inputNumber);
+				}
+				return edges;
+			}
+			else {
+				float factor = ((float) numSources) / parallelism;
+				
+				int start = (int) (subTaskIndex * factor);
+				int end = (subTaskIndex == getTotalNumberOfParallelSubtasks() - 1) ?
+						sourcePartitions.length : 
+						(int) ((subTaskIndex + 1) * factor);
+				
+				ExecutionEdge[] edges = new ExecutionEdge[end - start];
+				for (int i = 0; i < edges.length; i++) {
+					edges[i] = new ExecutionEdge(sourcePartitions[start + i], this, inputNumber);
+				}
+				
+				return edges;
 			}
-			index -= numberOfEdges;
 		}
-
-		return null;
-
-	}
-
-	/**
-	 * Checks if this vertex is an input vertex in its stage, i.e. has either no
-	 * incoming connections or only incoming connections to group vertices in a lower stage.
-	 * 
-	 * @return <code>true</code> if this vertex is an input vertex, <code>false</code> otherwise
-	 */
-	public boolean isInputVertex() {
-
-		return this.groupVertex.isInputVertex();
-	}
-
-	/**
-	 * Checks if this vertex is an output vertex in its stage, i.e. has either no
-	 * outgoing connections or only outgoing connections to group vertices in a higher stage.
-	 * 
-	 * @return <code>true</code> if this vertex is an output vertex, <code>false</code> otherwise
-	 */
-	public boolean isOutputVertex() {
-
-		return this.groupVertex.isOutputVertex();
-	}
-
-	/**
-	 * Returns the index of this vertex in the vertex group.
-	 * 
-	 * @return the index of this vertex in the vertex group
-	 */
-	public int getIndexInVertexGroup() {
-
-		return this.indexInVertexGroup;
-	}
-
-	/**
-	 * Sets the vertex' index in the vertex group.
-	 * 
-	 * @param indexInVertexGroup
-	 *        the vertex' index in the vertex group
-	 */
-	void setIndexInVertexGroup(final int indexInVertexGroup) {
-
-		this.indexInVertexGroup = indexInVertexGroup;
-	}
-
-	/**
-	 * Returns the number of output gates attached to this vertex.
-	 * 
-	 * @return the number of output gates attached to this vertex
-	 */
-	public int getNumberOfOutputGates() {
-
-		return this.outputGates.length;
 	}
 
 	/**
-	 * Returns the output gate with the given index.
+	 * Gets the location preferences of this task, determined by the locations of the predecessors from which
+	 * it receives input data.
+	 * If there are more than MAX_DISTINCT_LOCATIONS_TO_CONSIDER different locations of source data, this
+	 * method returns {@code null} to indicate no location preference.
 	 * 
-	 * @param index
-	 *        the index of the output gate to return
-	 * @return the output gate with the given index
-	 */
-	public ExecutionGate getOutputGate(final int index) {
-
-		return this.outputGates[index];
-	}
-
-	/**
-	 * Returns the number of input gates attached to this vertex.
-	 * 
-	 * @return the number of input gates attached to this vertex
-	 */
-	public int getNumberOfInputGates() {
-
-		return this.inputGates.length;
-	}
-
-	/**
-	 * Returns the input gate with the given index.
-	 * 
-	 * @param index
-	 *        the index of the input gate to return
-	 * @return the input gate with the given index
-	 */
-	public ExecutionGate getInputGate(final int index) {
-
-		return this.inputGates[index];
-	}
-
-	/**
-	 * Deploys and starts the task represented by this vertex
-	 * on the assigned instance.
-	 * 
-	 * @return the result of the task submission attempt
-	 */
-	public TaskSubmissionResult startTask() {
-
-		final AllocatedResource ar = this.allocatedResource.get();
-
-		if (ar == null) {
-			final TaskSubmissionResult result = new TaskSubmissionResult(getID(),
-				AbstractTaskResult.TaskOperationResult.NO_INSTANCE);
-			result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
-			return result;
-		}
-
-		final List<TaskDeploymentDescriptor> tasks = new SerializableArrayList<TaskDeploymentDescriptor>();
-		tasks.add(constructDeploymentDescriptor());
-
-		try {
-			final List<TaskSubmissionResult> results = ar.getInstance().submitTasks(tasks);
-
-			return results.get(0);
-
-		} catch (IOException e) {
-			final TaskSubmissionResult result = new TaskSubmissionResult(getID(),
-				AbstractTaskResult.TaskOperationResult.IPC_ERROR);
-			result.setDescription(StringUtils.stringifyException(e));
-			return result;
-		}
-	}
-
-	/**
-	 * Kills and removes the task represented by this vertex from the instance it is currently running on. If the
-	 * corresponding task is not in the state <code>RUNNING</code>, this call will be ignored. If the call has been
-	 * executed
-	 * successfully, the task will change the state <code>FAILED</code>.
-	 *
-	 * @return the result of the task kill attempt
+	 * @return The preferred locations for this vertex execution, or null, if there is no preference.
 	 */
-	public TaskKillResult killTask() {
-
-		final ExecutionState state = this.executionState.get();
-
-		if (state != ExecutionState.RUNNING) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.TaskOperationResult.ILLEGAL_STATE);
-			result.setDescription("Vertex " + this.toString() + " is in state " + state);
-			return result;
-		}
-
-		final AllocatedResource ar = this.allocatedResource.get();
-
-		if (ar == null) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.TaskOperationResult.NO_INSTANCE);
-			result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
-			return result;
-		}
-
-		try {
-			return ar.getInstance().killTask(this.vertexID);
-		} catch (IOException e) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.TaskOperationResult.IPC_ERROR);
-			result.setDescription(StringUtils.stringifyException(e));
-			return result;
+	public Iterable<Instance> getPreferredLocations() {
+		HashSet<Instance> locations = new HashSet<Instance>();
+		
+		for (int i = 0; i < inputEdges.length; i++) {
+			ExecutionEdge[] sources = inputEdges[i];
+			if (sources != null) {
+				for (int k = 0; k < sources.length; k++) {
+					Instance source = sources[k].getSource().getProducer().getCurrentAssignedResource().getInstance();
+					locations.add(source);
+					if (locations.size() > MAX_DISTINCT_LOCATIONS_TO_CONSIDER) {
+						return null;
+					}
+				}
+			}
 		}
+		return locations;
 	}
-
-	/**
-	 * Cancels and removes the task represented by this vertex
-	 * from the instance it is currently running on. If the task
-	 * is not currently running, its execution state is simply
-	 * updated to <code>CANCELLED</code>.
-	 * 
-	 * @return the result of the task cancel attempt
-	 */
-	public TaskCancelResult cancelTask() {
-
-		while (true) {
-
-			final ExecutionState previousState = this.executionState.get();
-
-			if (previousState == ExecutionState.CANCELED) {
-				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
-			}
-
-			if (previousState == ExecutionState.FAILED) {
-				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
-			}
-
-			if (previousState == ExecutionState.FINISHED) {
-				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
+	
+	// --------------------------------------------------------------------------------------------
+	//   Actions
+	// --------------------------------------------------------------------------------------------
+	
+	public void resetForNewExecution() {
+		synchronized (priorExecutions) {
+			Execution execution = currentExecution;
+			ExecutionState state = execution.getState();
+			
+			if (state == FINISHED || state == CANCELED || state == FAILED) {
+				priorExecutions.add(execution);
+				currentExecution = new Execution(this, execution.getAttemptNumber()+1, System.currentTimeMillis());
 			}
-
-			// The vertex has already received a cancel request
-			if (previousState == ExecutionState.CANCELING) {
-				return new TaskCancelResult(getID(), ReturnCode.SUCCESS);
-			}
-
-			// Do not trigger the cancel request when vertex is in state STARTING, this might cause a race between RPC
-			// calls.
-			if (previousState == ExecutionState.STARTING) {
-
-				this.cancelRequested.set(true);
-
-				// We had a race, so we unset the flag and take care of cancellation ourselves
-				if (this.executionState.get() != ExecutionState.STARTING) {
-					this.cancelRequested.set(false);
-					continue;
-				}
-
-				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
-			}
-
-			// Check if we had a race. If state change is accepted, send cancel request
-			if (compareAndUpdateExecutionState(previousState, ExecutionState.CANCELING)) {
-
-				if (this.groupVertex.getStageNumber() != this.executionGraph.getIndexOfCurrentExecutionStage()) {
-					// Set to canceled directly
-					updateExecutionState(ExecutionState.CANCELED, null);
-					return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
-				}
-
-				if (previousState != ExecutionState.RUNNING && previousState != ExecutionState.FINISHING) {
-					// Set to canceled directly
-					updateExecutionState(ExecutionState.CANCELED, null);
-					return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
-				}
-
-				final AllocatedResource ar = this.allocatedResource.get();
-
-				if (ar == null) {
-					final TaskCancelResult result = new TaskCancelResult(getID(),
-						AbstractTaskResult.TaskOperationResult.NO_INSTANCE);
-					result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
-					return result;
-				}
-
-				try {
-					return ar.getInstance().cancelTask(this.vertexID);
-
-				} catch (IOException e) {
-					final TaskCancelResult result = new TaskCancelResult(getID(),
-						AbstractTaskResult.TaskOperationResult.IPC_ERROR);
-					result.setDescription(StringUtils.stringifyException(e));
-					return result;
-				}
+			else {
+				throw new IllegalStateException("Cannot reset a vertex that is in state " + state);
 			}
 		}
 	}
-
-	/**
-	 * Returns the {@link ExecutionGraph} this vertex belongs to.
-	 * 
-	 * @return the {@link ExecutionGraph} this vertex belongs to
-	 */
-	public ExecutionGraph getExecutionGraph() {
-
-		return this.executionGraph;
+	
+	public void scheduleForExecution(DefaultScheduler scheduler, boolean queued) throws NoResourceAvailableException {
+		this.currentExecution.scheduleForExecution(scheduler, queued);
 	}
-
-
-	@Override
-	public String toString() {
-
-		final StringBuilder sb = new StringBuilder(this.groupVertex.getName());
-		sb.append(" (");
-		sb.append(this.indexInVertexGroup + 1);
-		sb.append('/');
-		sb.append(this.groupVertex.getCurrentNumberOfGroupMembers());
-		sb.append(')');
-
-		return sb.toString();
+	
+	public void deployToSlot(AllocatedSlot slot) throws JobException {
+		this.currentExecution.deployToSlot(slot);
 	}
-
-	/**
-	 * Returns the task represented by this vertex has
-	 * a retry attempt left in case of an execution
-	 * failure.
-	 * 
-	 * @return <code>true</code> if the task has a retry attempt left, <code>false</code> otherwise
-	 */
-	@Deprecated
-	public boolean hasRetriesLeft() {
-		if (this.retriesLeft.get() <= 0) {
-			return false;
-		}
-		return true;
+	
+	public void cancel() {
+		this.currentExecution.cancel();
 	}
-
-	/**
-	 * Decrements the number of retries left and checks whether another attempt to run the task is possible.
-	 * 
-	 * @return <code>true</code>if the task represented by this vertex can be started at least once more,
-	 *         <code>false/<code> otherwise
-	 */
-	public boolean decrementRetriesLeftAndCheck() {
-		return (this.retriesLeft.decrementAndGet() > 0);
+	
+	public void fail(Throwable t) {
+		this.currentExecution.fail(t);
 	}
-
-	/**
-	 * Registers the {@link VertexAssignmentListener} object for this vertex. This object
-	 * will be notified about reassignments of this vertex to another instance.
-	 * 
-	 * @param vertexAssignmentListener
-	 *        the object to be notified about reassignments of this vertex to another instance
-	 */
-	public void registerVertexAssignmentListener(final VertexAssignmentListener vertexAssignmentListener) {
-
-		this.vertexAssignmentListeners.addIfAbsent(vertexAssignmentListener);
+	
+	// --------------------------------------------------------------------------------------------
+	//   Notifications from the Execution Attempt
+	// --------------------------------------------------------------------------------------------
+	
+	void executionFinished() {
+		jobVertex.vertexFinished(subTaskIndex);
 	}
-
-	/**
-	 * Unregisters the {@link VertexAssignmentListener} object for this vertex. This object
-	 * will no longer be notified about reassignments of this vertex to another instance.
-	 * 
-	 * @param vertexAssignmentListener
-	 *        the listener to be unregistered
-	 */
-	public void unregisterVertexAssignmentListener(final VertexAssignmentListener vertexAssignmentListener) {
-
-		this.vertexAssignmentListeners.remove(vertexAssignmentListener);
+	
+	void executionCanceled() {
+		jobVertex.vertexCancelled(subTaskIndex);
 	}
-
-
-	/**
-	 * Registers the {@link ExecutionListener} object for this vertex. This object
-	 * will be notified about particular events during the vertex's lifetime.
-	 * 
-	 * @param executionListener
-	 *        the object to be notified about particular events during the vertex's lifetime
-	 */
-	public void registerExecutionListener(final ExecutionListener executionListener) {
-
-		final Integer priority = Integer.valueOf(executionListener.getPriority());
-
-		if (priority.intValue() < 0) {
-			LOG.error("Priority for execution listener " + executionListener.getClass() + " must be non-negative.");
-			return;
-		}
-
-		final ExecutionListener previousValue = this.executionListeners.putIfAbsent(priority, executionListener);
-
-		if (previousValue != null) {
-			LOG.error("Cannot register " + executionListener.getClass() + " as an execution listener. Priority "
-				+ priority.intValue() + " is already taken.");
-		}
+	
+	void executionFailed(Throwable t) {
+		jobVertex.vertexFailed(subTaskIndex, t);
 	}
-
+	
+	// --------------------------------------------------------------------------------------------
+	//   Miscellaneous
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * Unregisters the {@link ExecutionListener} object for this vertex. This object
-	 * will no longer be notified about particular events during the vertex's lifetime.
+	 * Simply forward this notification. This is for logs and event archivers.
 	 * 
-	 * @param executionListener
-	 *        the object to be unregistered
+	 * @param executionId
+	 * @param newState
+	 * @param error
 	 */
-	public void unregisterExecutionListener(final ExecutionListener executionListener) {
-
-		this.executionListeners.remove(Integer.valueOf(executionListener.getPriority()));
+	void notifyStateTransition(ExecutionAttemptID executionId, ExecutionState newState, Throwable error) {
+		getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, executionId, newState, error);
 	}
-
-
-	/**
-	 * Sets the {@link ExecutionPipeline} this vertex shall be part of.
-	 * 
-	 * @param executionPipeline
-	 *        the execution pipeline this vertex shall be part of
-	 */
-	void setExecutionPipeline(final ExecutionPipeline executionPipeline) {
-
-		final ExecutionPipeline oldPipeline = this.executionPipeline.getAndSet(executionPipeline);
-		if (oldPipeline != null) {
-			oldPipeline.removeFromPipeline(this);
+	
+	TaskDeploymentDescriptor createDeploymentDescriptor(ExecutionAttemptID executionId, AllocatedSlot slot) {
+		//  create the input gate deployment descriptors
+		List<GateDeploymentDescriptor> inputGates = new ArrayList<GateDeploymentDescriptor>(inputEdges.length);
+		for (ExecutionEdge[] channels : inputEdges) {
+			inputGates.add(GateDeploymentDescriptor.fromEdges(channels));
 		}
-
-		executionPipeline.addToPipeline(this);
+		
+		// create the output gate deployment descriptors
+		List<GateDeploymentDescriptor> outputGates = new ArrayList<GateDeploymentDescriptor>(resultPartitions.length);
+		for (IntermediateResultPartition partition : resultPartitions) {
+			for (List<ExecutionEdge> channels : partition.getConsumers()) {
+				outputGates.add(GateDeploymentDescriptor.fromEdges(channels));
+			}
+		}
+		
+		String[] jarFiles = getExecutionGraph().getUserCodeJarFiles();
+		
+		return new TaskDeploymentDescriptor(getJobId(), getJobvertexId(), executionId, getTaskName(), 
+				subTaskIndex, getTotalNumberOfParallelSubtasks(), 
+				getExecutionGraph().getJobConfiguration(), jobVertex.getJobVertex().getConfiguration(),
+				jobVertex.getJobVertex().getInvokableClassName(), outputGates, inputGates, jarFiles, slot.getSlotNumber());
 	}
-
-	/**
-	 * Returns the {@link ExecutionPipeline} this vertex is part of.
-	 * 
-	 * @return the execution pipeline this vertex is part of
-	 */
-	public ExecutionPipeline getExecutionPipeline() {
-
-		return this.executionPipeline.get();
+	
+	
+	// --------------------------------------------------------------------------------------------
+	//  Utilities
+	// --------------------------------------------------------------------------------------------
+	
+	public void execute(Runnable action) {
+		this.jobVertex.execute(action);
 	}
-
+	
 	/**
-	 * Constructs a new task deployment descriptor for this vertex.
+	 * Creates a simple name representation in the style 'taskname (x/y)', where
+	 * 'taskname' is the name as returned by {@link #getTaskName()}, 'x' is the parallel
+	 * subtask index as returned by {@link #getParallelSubtaskIndex()}{@code + 1}, and 'y' is the total
+	 * number of tasks, as returned by {@link #getTotalNumberOfParallelSubtasks()}.
 	 * 
-	 * @return a new task deployment descriptor for this vertex
+	 * @return A simple name representation.
 	 */
-	public TaskDeploymentDescriptor constructDeploymentDescriptor() {
-
-		final SerializableArrayList<GateDeploymentDescriptor> ogd = new SerializableArrayList<GateDeploymentDescriptor>(
-			this.outputGates.length);
-		for (int i = 0; i < this.outputGates.length; ++i) {
-
-			final ExecutionGate eg = this.outputGates[i];
-			final List<ChannelDeploymentDescriptor> cdd = new ArrayList<ChannelDeploymentDescriptor>(
-				eg.getNumberOfEdges());
-			final int numberOfOutputChannels = eg.getNumberOfEdges();
-			for (int j = 0; j < numberOfOutputChannels; ++j) {
-
-				final ExecutionEdge ee = eg.getEdge(j);
-				cdd.add(new ChannelDeploymentDescriptor(ee.getOutputChannelID(), ee.getInputChannelID()));
-			}
-
-			ogd.add(new GateDeploymentDescriptor(eg.getGateID(), eg.getChannelType(), cdd));
-		}
-
-		final SerializableArrayList<GateDeploymentDescriptor> igd = new SerializableArrayList<GateDeploymentDescriptor>(
-			this.inputGates.length);
-		for (int i = 0; i < this.inputGates.length; ++i) {
-
-			final ExecutionGate eg = this.inputGates[i];
-			final List<ChannelDeploymentDescriptor> cdd = new ArrayList<ChannelDeploymentDescriptor>(
-				eg.getNumberOfEdges());
-			final int numberOfInputChannels = eg.getNumberOfEdges();
-			for (int j = 0; j < numberOfInputChannels; ++j) {
-
-				final ExecutionEdge ee = eg.getEdge(j);
-				cdd.add(new ChannelDeploymentDescriptor(ee.getOutputChannelID(), ee.getInputChannelID()));
-			}
-
-			igd.add(new GateDeploymentDescriptor(eg.getGateID(), eg.getChannelType(), cdd));
-		}
-
-		final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(this.executionGraph.getJobID(),
-			this.vertexID, this.groupVertex.getName(), this.indexInVertexGroup,
-			this.groupVertex.getCurrentNumberOfGroupMembers(), this.executionGraph.getJobConfiguration(),
-			this.groupVertex.getConfiguration(), this.groupVertex.getInvokableClass(), ogd,
-			igd);
-
-		return tdd;
+	public String getSimpleName() {
+		return getTaskName() + " (" + (getParallelSubtaskIndex()+1) + '/' + getTotalNumberOfParallelSubtasks() + ')';
 	}
 	
-	public void handleException(Throwable t) {
-		
+	@Override
+	public String toString() {
+		return getSimpleName();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
deleted file mode 100644
index 787605d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
+++ /dev/null
@@ -1,710 +0,0 @@
-/**
- * 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 java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-
-import org.apache.commons.logging.Log;
-import org.apache.flink.runtime.JobException;
-import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState2;
-import org.apache.flink.runtime.instance.AllocatedSlot;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobEdge;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
-import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
-import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
-import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
-import org.apache.flink.runtime.taskmanager.TaskOperationResult;
-import org.apache.flink.util.StringUtils;
-import org.slf4j.Logger;
-
-import static org.apache.flink.runtime.execution.ExecutionState2.*;
-
-/**
- * 
- * NOTE ABOUT THE DESIGN RATIONAL:
- * 
- * In several points of the code, we need to deal with possible concurrent state changes and actions.
- * For example, while the call to deploy a task (send it to the TaskManager) happens, the task gets cancelled.
- * 
- * We could lock the entire portion of the code (decision to deploy, deploy, set state to running) such that
- * it is guaranteed that any "cancel command" will only pick up after deployment is done and that the "cancel
- * command" call will never overtake the deploying call.
- * 
- * This blocks the threads big time, because the remote calls may take long. Depending of their locking behavior, it
- * may even result in distributed deadlocks (unless carefully avoided). We therefore use atomic state updates and
- * occasional double-checking to ensure that the state after a completed call is as expected, and trigger correcting
- * actions if it is not. Many actions are also idempotent (like canceling).
- */
-public class ExecutionVertex2 {
-	
-	private static final AtomicReferenceFieldUpdater<ExecutionVertex2, ExecutionState2> STATE_UPDATER =
-			AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, ExecutionState2.class, "state");
-	
-	private static final AtomicReferenceFieldUpdater<ExecutionVertex2, ExecutionAttempt> ATTEMPT_UPDATER =
-			AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, ExecutionAttempt.class, "currentOrLastAttempt");
-
-	private static final Logger LOG = ExecutionGraph.LOG;
-	
-	private static final int NUM_CANCEL_CALL_TRIES = 3;
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private final ExecutionJobVertex jobVertex;
-	
-	private final IntermediateResultPartition[] resultPartitions;
-	
-	private final ExecutionEdge2[][] inputEdges;
-	
-	private final int subTaskIndex;
-	
-	private final long[] stateTimestamps;
-	
-	private final List<ExecutionAttempt> priorAttempts;
-	
-	private volatile ExecutionAttempt currentOrLastAttempt;
-	
-	private volatile ExecutionState2 state = CREATED;
-	
-	
-	public ExecutionVertex2(ExecutionJobVertex jobVertex, int subTaskIndex, IntermediateResult[] producedDataSets) {
-		this.jobVertex = jobVertex;
-		this.subTaskIndex = subTaskIndex;
-		
-		this.resultPartitions = new IntermediateResultPartition[producedDataSets.length];
-		for (int i = 0; i < producedDataSets.length; i++) {
-			IntermediateResultPartition irp = new IntermediateResultPartition(producedDataSets[i], this, subTaskIndex);
-			this.resultPartitions[i] = irp;
-			producedDataSets[i].setPartition(subTaskIndex, irp);
-		}
-		
-		this.inputEdges = new ExecutionEdge2[jobVertex.getJobVertex().getInputs().size()][];
-		
-		this.stateTimestamps = new long[ExecutionState2.values().length];
-		this.priorAttempts = new CopyOnWriteArrayList<ExecutionAttempt>();
-	}
-	
-	
-	// --------------------------------------------------------------------------------------------
-	//  Properties
-	// --------------------------------------------------------------------------------------------
-	
-	public JobID getJobId() {
-		return this.jobVertex.getJobId();
-	}
-	
-	public JobVertexID getJobvertexId() {
-		return this.jobVertex.getJobVertexId();
-	}
-	
-	public String getTaskName() {
-		return this.jobVertex.getJobVertex().getName();
-	}
-	
-	public int getTotalNumberOfParallelSubtasks() {
-		return this.jobVertex.getParallelism();
-	}
-	
-	public int getParallelSubtaskIndex() {
-		return this.subTaskIndex;
-	}
-	
-	public int getNumberOfInputs() {
-		return this.inputEdges.length;
-	}
-	
-	public ExecutionEdge2[] getInputEdges(int input) {
-		if (input < 0 || input >= this.inputEdges.length) {
-			throw new IllegalArgumentException(String.format("Input %d is out of range [0..%d)", input, this.inputEdges.length));
-		}
-		return inputEdges[input];
-	}
-	
-	public ExecutionState2 getExecutionState() {
-		return state;
-	}
-	
-	public long getStateTimestamp(ExecutionState2 state) {
-		return this.stateTimestamps[state.ordinal()];
-	}
-	
-	private ExecutionGraph getExecutionGraph() {
-		return this.jobVertex.getGraph();
-	}
-	
-	public Throwable getLastFailureCause() {
-		// copy reference to the stack
-		ExecutionAttempt attempt = this.currentOrLastAttempt;
-		if (attempt != null) {
-			return attempt.getFailureCause();
-		}
-		else if (priorAttempts.size() > 0) {
-			// since the list is append-only, this always works in the presence of concurrent modifications
-			return priorAttempts.get(priorAttempts.size() - 1).getFailureCause();
-		}
-		else {
-			return null;
-		}
-	}
-	
-	public AllocatedSlot getCurrentAssignedResource() {
-		// copy reference to the stack
-		ExecutionAttempt attempt = this.currentOrLastAttempt;
-		return attempt == null ? null : attempt.getAssignedResource();
-	}
-	
-	public AllocatedSlot getLastAssignedResource() {
-		// copy reference to the stack
-		ExecutionAttempt attempt = this.currentOrLastAttempt;
-		if (attempt != null) {
-			return attempt.getAssignedResource();
-		}
-		else if (priorAttempts.size() > 0) {
-			// since the list is append-only, this always works in the presence of concurrent modifications
-			return priorAttempts.get(priorAttempts.size() - 1).getAssignedResource();
-		}
-		else {
-			return null;
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  Graph building
-	// --------------------------------------------------------------------------------------------
-	
-	public void connectSource(int inputNumber, IntermediateResult source, JobEdge edge, int consumerNumber) {
-		
-		final DistributionPattern pattern = edge.getDistributionPattern();
-		final IntermediateResultPartition[] sourcePartitions = source.getPartitions();
-		
-		ExecutionEdge2[] edges = null;
-		
-		switch (pattern) {
-			case POINTWISE:
-				edges = connectPointwise(sourcePartitions, inputNumber);
-				break;
-				
-			case BIPARTITE: 
-				edges = connectAllToAll(sourcePartitions, inputNumber);
-				break;
-				
-			default:
-				throw new RuntimeException("Unrecognized distribution pattern.");
-		
-		}
-		
-		this.inputEdges[inputNumber] = edges;
-		
-		// add the cousumers to the source
-		for (ExecutionEdge2 ee : edges) {
-			ee.getSource().addConsumer(ee, consumerNumber);
-		}
-	}
-	
-	private ExecutionEdge2[] connectAllToAll(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
-		ExecutionEdge2[] edges = new ExecutionEdge2[sourcePartitions.length];
-		
-		for (int i = 0; i < sourcePartitions.length; i++) {
-			IntermediateResultPartition irp = sourcePartitions[i];
-			edges[i] = new ExecutionEdge2(irp, this, inputNumber);
-		}
-		
-		return edges;
-	}
-	
-	private ExecutionEdge2[] connectPointwise(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
-		final int numSources = sourcePartitions.length;
-		final int parallelism = getTotalNumberOfParallelSubtasks();
-		
-		// simple case same number of sources as targets
-		if (numSources == parallelism) {
-			return new ExecutionEdge2[] { new ExecutionEdge2(sourcePartitions[subTaskIndex], this, inputNumber) };
-		}
-		else if (numSources < parallelism) {
-			
-			int sourcePartition;
-			
-			// check if the pattern is regular or irregular
-			// we use int arithmetics for regular, and floating point with rounding for irregular
-			if (parallelism % numSources == 0) {
-				// same number of targets per source
-				int factor = parallelism / numSources;
-				sourcePartition = subTaskIndex / factor;
-			}
-			else {
-				// different number of targets per source
-				float factor = ((float) parallelism) / numSources;
-				sourcePartition = (int) (subTaskIndex / factor);
-			}
-			
-			return new ExecutionEdge2[] { new ExecutionEdge2(sourcePartitions[sourcePartition], this, inputNumber) };
-		}
-		else {
-			if (numSources % parallelism == 0) {
-				// same number of targets per source
-				int factor = numSources / parallelism;
-				int startIndex = subTaskIndex * factor;
-				
-				ExecutionEdge2[] edges = new ExecutionEdge2[factor];
-				for (int i = 0; i < factor; i++) {
-					edges[i] = new ExecutionEdge2(sourcePartitions[startIndex + i], this, inputNumber);
-				}
-				return edges;
-			}
-			else {
-				float factor = ((float) numSources) / parallelism;
-				
-				int start = (int) (subTaskIndex * factor);
-				int end = (subTaskIndex == getTotalNumberOfParallelSubtasks() - 1) ?
-						sourcePartitions.length : 
-						(int) ((subTaskIndex + 1) * factor);
-				
-				ExecutionEdge2[] edges = new ExecutionEdge2[end - start];
-				for (int i = 0; i < edges.length; i++) {
-					edges[i] = new ExecutionEdge2(sourcePartitions[start + i], this, inputNumber);
-				}
-				
-				return edges;
-			}
-		}
-	}
-
-	
-	// --------------------------------------------------------------------------------------------
-	//  Scheduling
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * NOTE: This method only throws exceptions if it is in an illegal state to be scheduled, or if the tasks needs
-	 *       to be scheduled immediately and no resource is available. If the task is accepted by the schedule, any
-	 *       error sets the vertex state to failed and triggers the recovery logic.
-	 * 
-	 * @param scheduler
-	 * 
-	 * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only state that permits scheduling.
-	 * @throws NoResourceAvailableException Thrown is no queued scheduling is allowed and no resources are currently available.
-	 */
-	public void scheduleForExecution(DefaultScheduler scheduler) throws NoResourceAvailableException {
-		if (scheduler == null) {
-			throw new NullPointerException();
-		}
-		
-		if (STATE_UPDATER.compareAndSet(this, CREATED, SCHEDULED)) {
-			
-			getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, SCHEDULED, null);
-			
-			ScheduledUnit toSchedule = new ScheduledUnit(this, jobVertex.getSlotSharingGroup());
-		
-			// IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned
-			//     in all cases where the deployment failed. we use many try {} finally {} clauses to assure that
-			
-			boolean queued = jobVertex.getGraph().isQueuedSchedulingAllowed();
-			if (queued) {
-				SlotAllocationFuture future = scheduler.scheduleQueued(toSchedule);
-				
-				future.setFutureAction(new SlotAllocationFutureAction() {
-					@Override
-					public void slotAllocated(AllocatedSlot slot) {
-						try {
-							deployToSlot(slot);
-						}
-						catch (Throwable t) {
-							try {
-								slot.releaseSlot();
-							} finally {
-								fail(t);
-							}
-						}
-					}
-				});
-			}
-			else {
-				AllocatedSlot slot = scheduler.scheduleImmediately(toSchedule);
-				try {
-					deployToSlot(slot);
-				}
-				catch (Throwable t) {
-					try {
-						slot.releaseSlot();
-					} finally {
-						fail(t);
-					}
-				}
-			}
-		}
-		else if (this.state == CANCELED) {
-			// this can occur very rarely through heavy races. if the task was canceled, we do not
-			// schedule it
-			return;
-		}
-		else {
-			throw new IllegalStateException("The vertex must be in CREATED state to be scheduled.");
-		}
-	}
-	
-
-	public void deployToSlot(final AllocatedSlot slot) throws JobException {
-		// sanity checks
-		if (slot == null) {
-			throw new NullPointerException();
-		}
-		if (!slot.isAlive()) {
-			throw new IllegalArgumentException("Cannot deploy to a slot that is not alive.");
-		}
-		
-		// make sure exactly one deployment call happens from the correct state
-		// note: the transition from CREATED to DEPLOYING is for testing purposes only
-		ExecutionState2 previous = this.state;
-		if (previous == SCHEDULED || previous == CREATED) {
-			if (!STATE_UPDATER.compareAndSet(this, previous, DEPLOYING)) {
-				// race condition, someone else beat us to the deploying call.
-				// this should actually not happen and indicates a race somewhere else
-				throw new IllegalStateException("Cannot deploy task: Concurrent deployment call race.");
-			}
-			
-			getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, DEPLOYING, null);
-		}
-		else {
-			// vertex may have been cancelled, or it was already scheduled
-			throw new IllegalStateException("The vertex must be in CREATED or SCHEDULED state to be deployed. Found state " + previous);
-		}
-		
-		// good, we are allowed to deploy
-		if (!slot.setExecutedVertex(this)) {
-			throw new JobException("Could not assign the ExecutionVertex to the slot " + slot);
-		}
-		setAssignedSlot(slot);
-		
-		
-		final TaskDeploymentDescriptor deployment = createDeploymentDescriptor();
-		
-		// we execute the actual deploy call in a concurrent action to prevent this call from blocking for long
-		Runnable deployaction = new Runnable() {
-
-			@Override
-			public void run() {
-				try {
-					Instance instance = slot.getInstance();
-					instance.checkLibraryAvailability(getJobId());
-					
-					TaskOperationResult result = instance.getTaskManagerProxy().submitTask(deployment);
-					if (result.isSuccess()) {
-						switchToRunning();
-					}
-					else {
-						// deployment failed :(
-						fail(new Exception("Failed to deploy the tast to slot " + slot + ": " + result.getDescription()));
-					}
-				}
-				catch (Throwable t) {
-					// some error occurred. fail the task
-					fail(t);
-				}
-			}
-		};
-		
-		execute(deployaction);
-	}
-	
-	private void switchToRunning() {
-		
-		// transition state
-		if (STATE_UPDATER.compareAndSet(ExecutionVertex2.this, DEPLOYING, RUNNING)) {
-			
-			getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, RUNNING, null);
-			
-			this.jobVertex.vertexSwitchedToRunning(subTaskIndex);
-		}
-		else {
-			// something happened while the call was in progress.
-			// typically, that means canceling while deployment was in progress
-			
-			ExecutionState2 currentState = ExecutionVertex2.this.state;
-			
-			if (currentState == CANCELING) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(String.format("Concurrent canceling of task %s while deployment was in progress.", ExecutionVertex2.this.toString()));
-				}
-				
-				sendCancelRpcCall();
-			} else {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(String.format("Concurrent unexpected state transition of task %s while deployment was in progress.", ExecutionVertex2.this.toString()));
-				}
-				
-				// undo the deployment
-				sendCancelRpcCall();
-				
-				// record the failure
-				fail(new Exception("Asynchronous state error. Execution Vertex switched to " + currentState + " while deployment was in progress."));
-			}
-		}
-	}
-	
-	public void cancel() {
-		// depending on the previous state, we go directly to cancelled (no cancel call necessary)
-		// -- or to canceling (cancel call needs to be sent to the task manager)
-		
-		// because of several possibly previous states, we need to again loop until we make a
-		// successful atomic state transition
-		while (true) {
-			
-			ExecutionState2 current = this.state;
-			
-			if (current == CANCELING || current == CANCELED) {
-				// already taken care of, no need to cancel again
-				return;
-			}
-				
-			// these two are the common cases where we need to send a cancel call
-			else if (current == RUNNING || current == DEPLOYING) {
-				// try to transition to canceling, if successful, send the cancel call
-				if (STATE_UPDATER.compareAndSet(this, current, CANCELING)) {
-					
-					getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, CANCELING, null);
-					
-					sendCancelRpcCall();
-					return;
-				}
-				// else: fall through the loop
-			}
-			
-			else if (current == FINISHED || current == FAILED) {
-				// nothing to do any more. finished failed before it could be cancelled.
-				// in any case, the task is removed from the TaskManager already
-				return;
-			}
-			else if (current == CREATED || current == SCHEDULED) {
-				// from here, we can directly switch to cancelled, because the no task has been deployed
-				if (STATE_UPDATER.compareAndSet(this, current, CANCELED)) {
-					
-					getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, CANCELED, null);
-					
-					return;
-				}
-				// else: fall through the loop
-			}
-			else {
-				throw new IllegalStateException(current.name());
-			}
-		}
-	}
-	
-	public void fail(Throwable t) {
-		
-		// damn, we failed. This means only that we keep our books and notify our parent JobExecutionVertex
-		// the actual computation on the task manager is cleaned up by the taskmanager that noticed the failure
-		
-		// we may need to loop multiple times (in the presence of concurrent calls) in order to
-		// atomically switch to failed 
-		while (true) {
-			ExecutionState2 current = this.state;
-			
-			if (current == FAILED) {
-				// concurrently set to failed. It is enough to remember once that we failed (its sad enough)
-				return;
-			}
-			
-			if (current == CANCELED) {
-				// we already aborting
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(String.format("Ignoring transition of vertex %s to %s while being %s",
-							getSimpleName(), FAILED, current));
-				}
-				return;
-			}
-			
-			// we should be in DEPLOYING or RUNNING when a regular failure happens
-			if (current != DEPLOYING && current != RUNNING && current != CANCELING) {
-				// this should not happen. still, what else to do but to comply and go to the FAILED state
-				// at least we should complain loudly to the log
-				LOG.error(String.format("Vertex %s unexpectedly went from state %s to %s with error: %s",
-						getSimpleName(), CREATED, FAILED, t.getMessage()), t);
-			}
-			
-			if (STATE_UPDATER.compareAndSet(this, current, FAILED)) {
-				// success (in a manner of speaking)
-				this.failureCause = t;
-				
-				getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, FAILED, StringUtils.stringifyException(t));
-				
-				// release the slot (concurrency safe)
-				setAssignedSlot(null);
-				
-				this.jobVertex.vertexFailed(subTaskIndex);
-				
-				// leave the loop
-				return;
-			}
-		}
-	}
-	
-	private void sendCancelRpcCall() {
-		// first of all, copy a reference to the stack. any concurrent change to the
-		// field does not affect us now
-		final AllocatedSlot slot = this.assignedSlot;
-		if (slot == null) {
-			throw new IllegalStateException("Cannot cancel when task was not running or deployed.");
-		}
-		
-		Runnable cancelAction = new Runnable() {
-			
-			@Override
-			public void run() {
-				Throwable exception = null;
-				
-				for (int triesLeft = NUM_CANCEL_CALL_TRIES; triesLeft > 0; --triesLeft) {
-					
-					try {
-						// send the call. it may be that the task is not really there (asynchronous / overtaking messages)
-						// in which case it is fine (the deployer catches it)
-						TaskOperationResult result = slot.getInstance().getTaskManagerProxy().cancelTask(getJobvertexId(), subTaskIndex);
-						
-						if (result.isSuccess()) {
-							
-							// make sure that we release the slot
-							try {
-								// found and canceled
-								if (STATE_UPDATER.compareAndSet(ExecutionVertex2.this, CANCELING, CANCELED)) {
-									// we completed the call. 
-									// release the slot resource and let the parent know we have cancelled
-									ExecutionVertex2.this.jobVertex.vertexCancelled(ExecutionVertex2.this.subTaskIndex);
-								}
-								else {
-									ExecutionState2 foundState = ExecutionVertex2.this.state;
-									// failing in the meantime may happen and is no problem
-									if (foundState != FAILED) {
-										// corner case? log at least
-										LOG.error(String.format("Asynchronous race: Found state %s after successful cancel call.", foundState));
-									}
-									
-								}
-							} finally {
-								slot.releaseSlot();
-							}
-						}
-						else {
-							// the task was not found, which may be when the task concurrently finishes or fails, or
-							// when the cancel call overtakes the deployment call
-							if (LOG.isDebugEnabled()) {
-								LOG.debug("Cancel task call did not find task. Probably cause: Acceptable asynchronous race.");
-							}
-						}
-						
-						// in any case, we need not call multiple times, so we quit
-						return;
-					}
-					catch (Throwable t) {
-						if (exception == null) {
-							exception = t;
-						}
-						LOG.error("Canceling vertex " + getSimpleName() + " failed (" + triesLeft + " tries left): " + t.getMessage() , t);
-					}
-				}
-				
-				// dang, utterly unsuccessful - the target node must be down, in which case the tasks are lost anyways
-				fail(new Exception("Task could not be canceled.", exception));
-			}
-		};
-		
-		execute(cancelAction);
-	}
-	
-	public Iterable<Instance> getPreferredLocations() {
-		return null;
-	}
-	
-	private void setAssignedSlot(AllocatedSlot slot) {
-		
-		while (true) {
-			AllocatedSlot previous = this.assignedSlot;
-			if (ASSIGNED_SLOT_UPDATER.compareAndSet(this, previous, slot)) {
-				// successfully swapped
-				// release the predecessor, if it was not null. this call is idempotent, so it does not matter if it is
-				// called more than once
-				try {
-					if (previous != null) {
-						previous.releaseSlot();
-					}
-				} catch (Throwable t) {
-					LOG.debug("Error releasing slot " + slot, t);
-				}
-				return;
-			}
-		}
-	}
-	
-	
-	private TaskDeploymentDescriptor createDeploymentDescriptor() {
-		//  create the input gate deployment descriptors
-		List<GateDeploymentDescriptor> inputGates = new ArrayList<GateDeploymentDescriptor>(inputEdges.length);
-		for (ExecutionEdge2[] channels : inputEdges) {
-			inputGates.add(GateDeploymentDescriptor.fromEdges(channels));
-		}
-		
-		// create the output gate deployment descriptors
-		List<GateDeploymentDescriptor> outputGates = new ArrayList<GateDeploymentDescriptor>(resultPartitions.length);
-		for (IntermediateResultPartition partition : resultPartitions) {
-			for (List<ExecutionEdge2> channels : partition.getConsumers()) {
-				outputGates.add(GateDeploymentDescriptor.fromEdges(channels));
-			}
-		}
-		
-		String[] jarFiles = getExecutionGraph().getUserCodeJarFiles();
-		
-		return new TaskDeploymentDescriptor(getJobId(), getJobvertexId(), getTaskName(), 
-				subTaskIndex, getTotalNumberOfParallelSubtasks(), 
-				getExecutionGraph().getJobConfiguration(), jobVertex.getJobVertex().getConfiguration(),
-				jobVertex.getJobVertex().getInvokableClassName(), outputGates, inputGates, jarFiles);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  Utilities
-	// --------------------------------------------------------------------------------------------
-	
-	public void execute(Runnable action) {
-		this.jobVertex.execute(action);
-	}
-	
-	/**
-	 * Creates a simple name representation in the style 'taskname (x/y)', where
-	 * 'taskname' is the name as returned by {@link #getTaskName()}, 'x' is the parallel
-	 * subtask index as returned by {@link #getParallelSubtaskIndex()}{@code + 1}, and 'y' is the total
-	 * number of tasks, as returned by {@link #getTotalNumberOfParallelSubtasks()}.
-	 * 
-	 * @return A simple name representation.
-	 */
-	public String getSimpleName() {
-		return getTaskName() + " (" + (getParallelSubtaskIndex()+1) + '/' + getTotalNumberOfParallelSubtasks() + ')';
-	}
-	
-	@Override
-	public String toString() {
-		return getSimpleName() + " [" + state + ']';
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
index b733baa..1c4e1fb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
@@ -25,21 +25,21 @@ public class IntermediateResultPartition {
 	
 	private final IntermediateResult totalResut;
 	
-	private final ExecutionVertex2 producer;
+	private final ExecutionVertex producer;
 	
 	private final int partition;
 	
-	private List<List<ExecutionEdge2>> consumers;
+	private List<List<ExecutionEdge>> consumers;
 	
 	
-	public IntermediateResultPartition(IntermediateResult totalResut, ExecutionVertex2 producer, int partition) {
+	public IntermediateResultPartition(IntermediateResult totalResut, ExecutionVertex producer, int partition) {
 		this.totalResut = totalResut;
 		this.producer = producer;
 		this.partition = partition;
-		this.consumers = new ArrayList<List<ExecutionEdge2>>(0);
+		this.consumers = new ArrayList<List<ExecutionEdge>>(0);
 	}
 	
-	public ExecutionVertex2 getProducer() {
+	public ExecutionVertex getProducer() {
 		return producer;
 	}
 	
@@ -51,17 +51,23 @@ public class IntermediateResultPartition {
 		return totalResut;
 	}
 	
-	public List<List<ExecutionEdge2>> getConsumers() {
+	public List<List<ExecutionEdge>> getConsumers() {
 		return consumers;
 	}
 	
 	int addConsumerGroup() {
 		int pos = consumers.size();
-		consumers.add(new ArrayList<ExecutionEdge2>());
+		
+		// NOTE: currently we support only one consumer per result!!!
+		if (pos != 0) {
+			throw new RuntimeException("Currenty, each intermediate result can only have one consumer.");
+		}
+		
+		consumers.add(new ArrayList<ExecutionEdge>());
 		return pos;
 	}
 	
-	public void addConsumer(ExecutionEdge2 edge, int consumerNumber) {
+	void addConsumer(ExecutionEdge edge, int consumerNumber) {
 		consumers.get(consumerNumber).add(edge);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
index f3f489b..2f3fa10 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.instance;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.jobgraph.JobID;
 
 /**
@@ -32,8 +32,8 @@ public class AllocatedSlot {
 	private static final AtomicIntegerFieldUpdater<AllocatedSlot> STATUS_UPDATER = 
 			AtomicIntegerFieldUpdater.newUpdater(AllocatedSlot.class, "status");
 	
-	private static final AtomicReferenceFieldUpdater<AllocatedSlot, ExecutionVertex2> VERTEX_UPDATER =
-			AtomicReferenceFieldUpdater.newUpdater(AllocatedSlot.class, ExecutionVertex2.class, "executedVertex");
+	private static final AtomicReferenceFieldUpdater<AllocatedSlot, Execution> VERTEX_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(AllocatedSlot.class, Execution.class, "executedTask");
 	
 	private static final int ALLOCATED_AND_ALIVE = 0;		// tasks may be added and might be running
 	private static final int CANCELLED = 1;					// no more tasks may run
@@ -49,8 +49,8 @@ public class AllocatedSlot {
 	/** The number of the slot on which the task is deployed */
 	private final int slotNumber;
 	
-	/** Vertex being executed in the slot. Volatile to force a memory barrier and allow for correct double-checking */
-	private volatile ExecutionVertex2 executedVertex;
+	/** Task being executed in the slot. Volatile to force a memory barrier and allow for correct double-checking */
+	private volatile Execution executedTask;
 	
 	/** The state of the vertex, only atomically updated */
 	private volatile int status = ALLOCATED_AND_ALIVE;
@@ -85,7 +85,11 @@ public class AllocatedSlot {
 		return slotNumber;
 	}
 	
-	public boolean setExecutedVertex(ExecutionVertex2 executedVertex) {
+	public Execution getExecutedVertex() {
+		return executedTask;
+	}
+	
+	public boolean setExecutedVertex(Execution executedVertex) {
 		if (executedVertex == null) {
 			throw new NullPointerException();
 		}
@@ -102,17 +106,13 @@ public class AllocatedSlot {
 
 		// we need to do a double check that we were not cancelled in the meantime
 		if (status != ALLOCATED_AND_ALIVE) {
-			this.executedVertex = null;
+			this.executedTask = null;
 			return false;
 		}
 		
 		return true;
 	}
 	
-	public ExecutionVertex2 getExecutedVertex() {
-		return executedVertex;
-	}
-	
 	// --------------------------------------------------------------------------------------------
 	//  Status and life cycle
 	// --------------------------------------------------------------------------------------------
@@ -133,8 +133,9 @@ public class AllocatedSlot {
 	public void cancel() {
 		if (STATUS_UPDATER.compareAndSet(this, ALLOCATED_AND_ALIVE, CANCELLED)) {
 			// kill all tasks currently running in this slot
-			if (this.executedVertex != null) {
-				this.executedVertex.fail(new Exception("The slot in which the task was scheduled has been cancelled."));
+			Execution exec = this.executedTask;
+			if (exec != null && !exec.isFinished()) {
+				exec.fail(new Exception("The slot in which the task was scheduled has been cancelled."));
 			}
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
index 543ae86..bc30254 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
@@ -197,19 +197,21 @@ public class Instance {
 			throw new IOException("No entry of required libraries for job " + jobID);
 		}
 
-		LibraryCacheProfileRequest request = new LibraryCacheProfileRequest();
-		request.setRequiredLibraries(requiredLibraries);
-
-		// Send the request
-		LibraryCacheProfileResponse response = getTaskManagerProxy().getLibraryCacheProfile(request);
-
-		// Check response and transfer libraries if necessary
-		for (int k = 0; k < requiredLibraries.length; k++) {
-			if (!response.isCached(k)) {
-				LibraryCacheUpdate update = new LibraryCacheUpdate(requiredLibraries[k]);
-				getTaskManagerProxy().updateLibraryCache(update);
+//		if (requiredLibraries.length > 0) {
+			LibraryCacheProfileRequest request = new LibraryCacheProfileRequest();
+			request.setRequiredLibraries(requiredLibraries);
+	
+			// Send the request
+			LibraryCacheProfileResponse response = getTaskManagerProxy().getLibraryCacheProfile(request);
+	
+			// Check response and transfer libraries if necessary
+			for (int k = 0; k < requiredLibraries.length; k++) {
+				if (!response.isCached(k)) {
+					LibraryCacheUpdate update = new LibraryCacheUpdate(requiredLibraries[k]);
+					getTaskManagerProxy().updateLibraryCache(update);
+				}
 			}
-		}
+//		}
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
index ec63c00..c17e631 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
@@ -23,9 +23,9 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 
 import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.StringUtils;
 
 /**
  * This class encapsulates all connection information necessary to connect to the instance's task manager.
@@ -154,9 +154,7 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 		this.ipcPort = in.readInt();
 		this.dataPort = in.readInt();
 		
-		if (in.readBoolean()) {
-			this.hostName = StringRecord.readString(in);
-		}
+		this.hostName = StringUtils.readNullableString(in);
 
 		try {
 			this.inetAddress = InetAddress.getByAddress(address);
@@ -174,12 +172,7 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 		out.writeInt(this.ipcPort);
 		out.writeInt(this.dataPort);
 		
-		if (this.hostName != null) {
-			out.writeBoolean(true);
-			StringRecord.writeString(out, this.hostName);
-		} else {
-			out.writeBoolean(false);
-		}
+		StringUtils.writeNullableString(hostName, out);
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
index 5571ccb..3066bb5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
@@ -110,6 +110,10 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 		this.globalBufferPool.destroy();
 	}
 
+	public GlobalBufferPool getGlobalBufferPool() {
+		return globalBufferPool;
+	}
+	
 	// -----------------------------------------------------------------------------------------------------------------
 	//                                               Task registration
 	// -----------------------------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java
index 23d1205..cdf9b87 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/RemoteReceiver.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.network;
 
 import java.io.IOException;
@@ -28,18 +27,17 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
 /**
- * Objects of this class uniquely identify a connection to a remote {@link TaskManager}.
- * 
+ * Objects of this class uniquely identify a connection to a remote {@link org.apache.flink.runtime.taskmanager.TaskManager}.
  */
 public final class RemoteReceiver implements IOReadableWritable {
 
 	/**
-	 * The address of the connection to the remote {@link TaskManager}.
+	 * The address of the connection to the remote TaskManager.
 	 */
 	private InetSocketAddress connectionAddress;
 
 	/**
-	 * The index of the connection to the remote {@link TaskManager}.
+	 * The index of the connection to the remote TaskManager.
 	 */
 	private int connectionIndex;
 
@@ -47,9 +45,9 @@ public final class RemoteReceiver implements IOReadableWritable {
 	 * Constructs a new remote receiver object.
 	 * 
 	 * @param connectionAddress
-	 *        the address of the connection to the remote {@link TaskManager}
+	 *        the address of the connection to the remote TaskManager
 	 * @param connectionIndex
-	 *        the index of the connection to the remote {@link TaskManager}
+	 *        the index of the connection to the remote TaskManager
 	 */
 	public RemoteReceiver(final InetSocketAddress connectionAddress, final int connectionIndex) {
 		if (connectionAddress == null) {
@@ -72,18 +70,18 @@ public final class RemoteReceiver implements IOReadableWritable {
 	}
 
 	/**
-	 * Returns the address of the connection to the remote {@link TaskManager}.
+	 * Returns the address of the connection to the remote TaskManager.
 	 * 
-	 * @return the address of the connection to the remote {@link TaskManager}
+	 * @return the address of the connection to the remote TaskManager
 	 */
 	public InetSocketAddress getConnectionAddress() {
 		return this.connectionAddress;
 	}
 
 	/**
-	 * Returns the index of the connection to the remote {@link TaskManager}.
+	 * Returns the index of the connection to the remote TaskManager.
 	 * 
-	 * @return the index of the connection to the remote {@link TaskManager}
+	 * @return the index of the connection to the remote TaskManager
 	 */
 	public int getConnectionIndex() {
 		return this.connectionIndex;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/MutableReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/MutableReader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/MutableReader.java
index 6f494eb..5a0fbb5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/MutableReader.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/MutableReader.java
@@ -16,23 +16,13 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.network.api;
 
 import java.io.IOException;
 
 import org.apache.flink.core.io.IOReadableWritable;
 
-/**
- * 
- */
 public interface MutableReader<T extends IOReadableWritable> extends ReaderBase {
 	
-	/**
-	 * @param target
-	 * @return
-	 * @throws IOException
-	 * @throws InterruptedException
-	 */
 	boolean next(T target) throws IOException, InterruptedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
index 27658f1..1fecdbd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
@@ -21,7 +21,9 @@ package org.apache.flink.runtime.iterative.concurrent;
 import java.util.concurrent.CountDownLatch;
 
 /**
- * Resettable barrier to synchronize {@link IterationHeadPactTask} and {@link IterationTailPactTask} in case of
+ * Resettable barrier to synchronize the
+ * {@link org.apache.flink.runtime.iterative.task.IterationHeadPactTask} and
+ * the {@link org.apache.flink.runtime.iterative.task.IterationTailPactTask} in case of
  * iterations that contain a separate solution set tail.
  */
 public class SolutionSetUpdateBarrier {


[03/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
index c28f946..768ac82 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
@@ -18,98 +18,218 @@
 
 package org.apache.flink.runtime.jobgraph;
 
-import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.*;
 
-import java.util.logging.Level;
-import java.util.logging.Logger;
+import java.util.List;
 
+import org.apache.flink.api.common.InvalidProgramException;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-/**
- * This class contains tests related to the JobGraph
- */
 public class JobGraphTest {
 
-	public JobGraphTest() {
-	}
-
-	@BeforeClass
-	public static void setUpClass() throws Exception {
-	}
-
-	@AfterClass
-	public static void tearDownClass() throws Exception {
-	}
-
-	@Before
-	public void setUp() {
+	@Test
+	public void testSerialization() {
+		try {
+			JobGraph jg = new JobGraph("The graph");
+			
+			// add some configuration values
+			{
+				jg.getJobConfiguration().setString("some key", "some value");
+				jg.getJobConfiguration().setDouble("Life of ", Math.PI);
+			}
+			
+			// add some vertices
+			{
+				AbstractJobVertex source1 = new AbstractJobVertex("source1");
+				AbstractJobVertex source2 = new AbstractJobVertex("source2");
+				AbstractJobVertex target = new AbstractJobVertex("target");
+				target.connectNewDataSetAsInput(source1, DistributionPattern.POINTWISE);
+				target.connectNewDataSetAsInput(source2, DistributionPattern.BIPARTITE);
+				
+				jg.addVertex(source1);
+				jg.addVertex(source2);
+				jg.addVertex(target);
+			}
+			
+			// de-/serialize and compare
+			JobGraph copy = CommonTestUtils.createCopyWritable(jg);
+			
+			assertEquals(jg.getName(), copy.getName());
+			assertEquals(jg.getJobID(), copy.getJobID());
+			assertEquals(jg.getJobConfiguration(), copy.getJobConfiguration());
+			assertEquals(jg.getNumberOfVertices(), copy.getNumberOfVertices());
+			
+			for (AbstractJobVertex vertex : copy.getVertices()) {
+				AbstractJobVertex original = jg.findVertexByID(vertex.getID());
+				assertNotNull(original);
+				assertEquals(original.getName(), vertex.getName());
+				assertEquals(original.getNumberOfInputs(), vertex.getNumberOfInputs());
+				assertEquals(original.getNumberOfProducedIntermediateDataSets(), vertex.getNumberOfProducedIntermediateDataSets());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
 	}
-
-	@After
-	public void tearDown() {
+	
+	@Test
+	public void testTopologicalSort1() {
+		try {
+			AbstractJobVertex source1 = new AbstractJobVertex("source1");
+			AbstractJobVertex source2 = new AbstractJobVertex("source2");
+			AbstractJobVertex target1 = new AbstractJobVertex("target1");
+			AbstractJobVertex target2 = new AbstractJobVertex("target2");
+			AbstractJobVertex intermediate1 = new AbstractJobVertex("intermediate1");
+			AbstractJobVertex intermediate2 = new AbstractJobVertex("intermediate2");
+			
+			target1.connectNewDataSetAsInput(source1, DistributionPattern.POINTWISE);
+			target2.connectNewDataSetAsInput(source1, DistributionPattern.POINTWISE);
+			target2.connectNewDataSetAsInput(intermediate2, DistributionPattern.POINTWISE);
+			intermediate2.connectNewDataSetAsInput(intermediate1, DistributionPattern.POINTWISE);
+			intermediate1.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE);
+			
+			JobGraph graph = new JobGraph("TestGraph", source1, source2, intermediate1, intermediate2, target1, target2);
+			List<AbstractJobVertex> sorted = graph.getVerticesSortedTopologicallyFromSources();
+			
+			assertBefore(source1, target1, sorted);
+			assertBefore(source1, target2, sorted);
+			assertBefore(source2, target2, sorted);
+			assertBefore(source2, intermediate1, sorted);
+			assertBefore(source2, intermediate2, sorted);
+			assertBefore(intermediate1, target2, sorted);
+			assertBefore(intermediate2, target2, sorted);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
 	}
-
+	
 	@Test
-	/**
-	 * This test ensures that the JobGraph edges are correctly set (forward/backward edges)
-	 */
-	public void testJobGraph() {
-		// check if the backward edge really points to the preceding vertex
-		final JobGraph jg = new JobGraph();
-
-		final JobTaskVertex v1 = new JobTaskVertex(jg);
-		final JobTaskVertex v2 = new JobTaskVertex(jg);
-
+	public void testTopologicalSort2() {
 		try {
-			v1.connectTo(v2);
-		} catch (JobGraphDefinitionException ex) {
-			Logger.getLogger(JobGraphTest.class.getName()).log(Level.SEVERE, null, ex);
+			AbstractJobVertex source1 = new AbstractJobVertex("source1");
+			AbstractJobVertex source2 = new AbstractJobVertex("source2");
+			AbstractJobVertex root = new AbstractJobVertex("root");
+			AbstractJobVertex l11 = new AbstractJobVertex("layer 1 - 1");
+			AbstractJobVertex l12 = new AbstractJobVertex("layer 1 - 2");
+			AbstractJobVertex l13 = new AbstractJobVertex("layer 1 - 3");
+			AbstractJobVertex l2 = new AbstractJobVertex("layer 2");
+			
+			root.connectNewDataSetAsInput(l13, DistributionPattern.POINTWISE);
+			root.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE);
+			root.connectNewDataSetAsInput(l2, DistributionPattern.POINTWISE);
+			
+			l2.connectNewDataSetAsInput(l11, DistributionPattern.POINTWISE);
+			l2.connectNewDataSetAsInput(l12, DistributionPattern.POINTWISE);
+			
+			l11.connectNewDataSetAsInput(source1, DistributionPattern.POINTWISE);
+			
+			l12.connectNewDataSetAsInput(source1, DistributionPattern.POINTWISE);
+			l12.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE);
+			
+			l13.connectNewDataSetAsInput(source2, DistributionPattern.POINTWISE);
+			
+			JobGraph graph = new JobGraph("TestGraph", source1, source2, root, l11, l13, l12, l2);
+			List<AbstractJobVertex> sorted = graph.getVerticesSortedTopologicallyFromSources();
+			
+			assertBefore(source1, root, sorted);
+			assertBefore(source2, root, sorted);
+			assertBefore(l11, root, sorted);
+			assertBefore(l12, root, sorted);
+			assertBefore(l13, root, sorted);
+			assertBefore(l2, root, sorted);
+			
+			assertBefore(l11, l2, sorted);
+			assertBefore(l12, l2, sorted);
+			assertBefore(l2, root, sorted);
+			
+			assertBefore(source1, l2, sorted);
+			assertBefore(source2, l2, sorted);
+			
+			assertBefore(source2, l13, sorted);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
 		}
-
-		assertEquals(v1, v2.getBackwardConnection(0).getConnectedVertex());
-
 	}
-
-	/**
-	 * In this test we construct a job graph and set the dependency chain for instance sharing in a way that a cycle is
-	 * created. The test is considered successful if the cycle is detected.
-	 */
+	
 	@Test
-	public void detectCycleInInstanceSharingDependencyChain() {
-
-		final JobGraph jg = new JobGraph();
-
-		final JobTaskVertex v1 = new JobTaskVertex("v1", jg);
-		final JobTaskVertex v2 = new JobTaskVertex("v2", jg);
-		final JobTaskVertex v3 = new JobTaskVertex("v3", jg);
-		final JobTaskVertex v4 = new JobTaskVertex("v4", jg);
-
+	public void testTopoSortCyclicGraphNoSources() {
 		try {
-			v1.connectTo(v2);
-			v2.connectTo(v3);
-			v3.connectTo(v4);
-		} catch (JobGraphDefinitionException ex) {
-			Logger.getLogger(JobGraphTest.class.getName()).log(Level.SEVERE, null, ex);
+			AbstractJobVertex v1 = new AbstractJobVertex("1");
+			AbstractJobVertex v2 = new AbstractJobVertex("2");
+			AbstractJobVertex v3 = new AbstractJobVertex("3");
+			AbstractJobVertex v4 = new AbstractJobVertex("4");
+			
+			v1.connectNewDataSetAsInput(v4, DistributionPattern.POINTWISE);
+			v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+			v3.connectNewDataSetAsInput(v2, DistributionPattern.POINTWISE);
+			v4.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE);
+			
+			JobGraph jg = new JobGraph("Cyclic Graph", v1, v2, v3, v4);
+			try {
+				jg.getVerticesSortedTopologicallyFromSources();
+				fail("Failed to raise error on topologically sorting cyclic graph.");
+			}
+			catch (InvalidProgramException e) {
+				// that what we wanted
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTopoSortCyclicGraphIntermediateCycle() {
+		try{ 
+			AbstractJobVertex source = new AbstractJobVertex("source");
+			AbstractJobVertex v1 = new AbstractJobVertex("1");
+			AbstractJobVertex v2 = new AbstractJobVertex("2");
+			AbstractJobVertex v3 = new AbstractJobVertex("3");
+			AbstractJobVertex v4 = new AbstractJobVertex("4");
+			AbstractJobVertex target = new AbstractJobVertex("target");
+			
+			v1.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE);
+			v1.connectNewDataSetAsInput(v4, DistributionPattern.POINTWISE);
+			v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+			v3.connectNewDataSetAsInput(v2, DistributionPattern.POINTWISE);
+			v4.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE);
+			target.connectNewDataSetAsInput(v3, DistributionPattern.POINTWISE);
+			
+			JobGraph jg = new JobGraph("Cyclic Graph", v1, v2, v3, v4, source, target);
+			try {
+				jg.getVerticesSortedTopologicallyFromSources();
+				fail("Failed to raise error on topologically sorting cyclic graph.");
+			}
+			catch (InvalidProgramException e) {
+				// that what we wanted
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private static final void assertBefore(AbstractJobVertex v1, AbstractJobVertex v2, List<AbstractJobVertex> list) {
+		boolean seenFirst = false;
+		for (AbstractJobVertex v : list) {
+			if (v == v1) {
+				seenFirst = true;
+			}
+			else if (v == v2) {
+				if (!seenFirst) {
+					fail("The first vertex (" + v1 + ") is not before the second vertex (" + v2 + ")");
+				}
+				break;
+			}
 		}
-
-		// Dependency chain is acyclic
-		v1.setVertexToShareInstancesWith(v2);
-		v3.setVertexToShareInstancesWith(v2);
-		v4.setVertexToShareInstancesWith(v1);
-
-		assertEquals(jg.isInstanceDependencyChainAcyclic(), true);
-
-		// Create a cycle v4 -> v1 -> v2 -> v4
-		v2.setVertexToShareInstancesWith(v4);
-
-		assertEquals(jg.isInstanceDependencyChainAcyclic(), false);
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java
new file mode 100644
index 0000000..e1d862e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobTaskVertexTest.java
@@ -0,0 +1,166 @@
+/**
+ * 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.jobgraph;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.SerializationUtils;
+import org.apache.flink.api.common.io.GenericInputFormat;
+import org.apache.flink.api.common.io.InitializeOnMaster;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
+import org.apache.flink.api.java.io.DiscardingOuputFormat;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+
+@SuppressWarnings("serial")
+public class JobTaskVertexTest {
+
+	@Test
+	public void testConnectDirectly() {
+		AbstractJobVertex source = new AbstractJobVertex("source");
+		AbstractJobVertex target = new AbstractJobVertex("target");
+		target.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE);
+		
+		assertTrue(source.isInputVertex());
+		assertFalse(source.isOutputVertex());
+		assertFalse(target.isInputVertex());
+		assertTrue(target.isOutputVertex());
+		
+		assertEquals(1, source.getNumberOfProducedIntermediateDataSets());
+		assertEquals(1, target.getNumberOfInputs());
+		
+		assertEquals(target.getInputs().get(0).getSource(), source.getProducedDataSets().get(0));
+		
+		assertEquals(1, source.getProducedDataSets().get(0).getConsumers().size());
+		assertEquals(target, source.getProducedDataSets().get(0).getConsumers().get(0).getTarget());
+	}
+	
+	@Test
+	public void testConnectMultipleTargets() {
+		AbstractJobVertex source = new AbstractJobVertex("source");
+		AbstractJobVertex target1= new AbstractJobVertex("target1");
+		AbstractJobVertex target2 = new AbstractJobVertex("target2");
+		target1.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE);
+		target2.connectDataSetAsInput(source.getProducedDataSets().get(0), DistributionPattern.BIPARTITE);
+		
+		assertTrue(source.isInputVertex());
+		assertFalse(source.isOutputVertex());
+		assertFalse(target1.isInputVertex());
+		assertTrue(target1.isOutputVertex());
+		assertFalse(target2.isInputVertex());
+		assertTrue(target2.isOutputVertex());
+		
+		assertEquals(1, source.getNumberOfProducedIntermediateDataSets());
+		assertEquals(2, source.getProducedDataSets().get(0).getConsumers().size());
+		
+		assertEquals(target1.getInputs().get(0).getSource(), source.getProducedDataSets().get(0));
+		assertEquals(target2.getInputs().get(0).getSource(), source.getProducedDataSets().get(0));
+	}
+	
+	@Test
+	public void testOutputFormatVertex() {
+		try {
+			final TestingOutputFormat outputFormat = new TestingOutputFormat();
+			final OutputFormatVertex of = new OutputFormatVertex("Name");
+			new TaskConfig(of.getConfiguration()).setStubWrapper(new UserCodeObjectWrapper<OutputFormat<?>>(outputFormat));
+			final ClassLoader cl = getClass().getClassLoader();
+			
+			try {
+				of.initializeOnMaster(cl);
+				fail("Did not throw expected exception.");
+			} catch (TestException e) {
+				// all good
+			}
+			
+			OutputFormatVertex copy = SerializationUtils.clone(of);
+			try {
+				copy.initializeOnMaster(cl);
+				fail("Did not throw expected exception.");
+			} catch (TestException e) {
+				// all good
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testInputFormatVertex() {
+		try {
+			final TestInputFormat inputFormat = new TestInputFormat();
+			final InputFormatVertex vertex = new InputFormatVertex("Name");
+			new TaskConfig(vertex.getConfiguration()).setStubWrapper(new UserCodeObjectWrapper<InputFormat<?, ?>>(inputFormat));
+			
+			final ClassLoader cl = getClass().getClassLoader();
+			
+			vertex.initializeOnMaster(cl);
+			InputSplit[] splits = vertex.getInputSplitSource().createInputSplits(77);
+			
+			assertNotNull(splits);
+			assertEquals(1, splits.length);
+			assertEquals(TestSplit.class, splits[0].getClass());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static final class TestingOutputFormat extends DiscardingOuputFormat<Object> implements InitializeOnMaster {
+		@Override
+		public void initializeGlobal(int parallelism) throws IOException {
+			throw new TestException();
+		}
+	}
+	
+	private static final class TestException extends IOException {}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static final class TestSplit extends GenericInputSplit {}
+	
+	private static final class TestInputFormat extends GenericInputFormat<Object> {
+
+		@Override
+		public boolean reachedEnd()  {
+			return false;
+		}
+
+		@Override
+		public Object nextRecord(Object reuse) {
+			return null;
+		}
+		
+		@Override
+		public GenericInputSplit[] createInputSplits(int numSplits) throws IOException {
+			return new GenericInputSplit[] { new TestSplit() };
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleSourceTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleSourceTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleSourceTask.java
deleted file mode 100644
index f021de7..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleSourceTask.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/**
- * 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.jobmanager;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.fs.LineReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-
-public class DoubleSourceTask extends AbstractInvokable {
-
-	private RecordWriter<StringRecord> output1 = null;
-
-	private RecordWriter<StringRecord> output2 = null;
-
-	@Override
-	public void invoke() throws Exception {
-		this.output1.initializeSerializers();
-		this.output2.initializeSerializers();
-
-		final Iterator<FileInputSplit> splitIterator = getInputSplits();
-
-		while (splitIterator.hasNext()) {
-
-			final FileInputSplit split = splitIterator.next();
-
-			final long start = split.getStart();
-			final long length = split.getLength();
-
-			final FileSystem fs = FileSystem.get(split.getPath().toUri());
-
-			final FSDataInputStream fdis = fs.open(split.getPath());
-
-			final LineReader lineReader = new LineReader(fdis, start, length, (1024 * 1024));
-
-			byte[] line = lineReader.readLine();
-
-			while (line != null) {
-
-				// Create a string object from the data read
-				StringRecord str = new StringRecord();
-				str.set(line);
-
-				// Send out string
-				output1.emit(str);
-				output2.emit(str);
-
-				line = lineReader.readLine();
-			}
-
-			// Close the stream;
-			lineReader.close();
-		}
-
-		this.output1.flush();
-		this.output2.flush();
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.output1 = new RecordWriter<StringRecord>(this);
-		this.output2 = new RecordWriter<StringRecord>(this);
-	}
-
-	private Iterator<FileInputSplit> getInputSplits() {
-
-		final InputSplitProvider provider = getEnvironment().getInputSplitProvider();
-
-		return new Iterator<FileInputSplit>() {
-
-			private FileInputSplit nextSplit;
-			
-			private boolean exhausted;
-
-			@Override
-			public boolean hasNext() {
-				if (exhausted) {
-					return false;
-				}
-				
-				if (nextSplit != null) {
-					return true;
-				}
-				
-				FileInputSplit split = (FileInputSplit) provider.getNextInputSplit();
-				
-				if (split != null) {
-					this.nextSplit = split;
-					return true;
-				}
-				else {
-					exhausted = true;
-					return false;
-				}
-			}
-
-			@Override
-			public FileInputSplit next() {
-				if (this.nextSplit == null && !hasNext()) {
-					throw new NoSuchElementException();
-				}
-
-				final FileInputSplit tmp = this.nextSplit;
-				this.nextSplit = null;
-				return tmp;
-			}
-
-			@Override
-			public void remove() {
-				throw new UnsupportedOperationException();
-			}
-		};
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleTargetTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleTargetTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleTargetTask.java
deleted file mode 100644
index 4caf479..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DoubleTargetTask.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * 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.jobmanager;
-
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-public class DoubleTargetTask extends AbstractInvokable {
-
-	private RecordReader<StringRecord> input1 = null;
-
-	private RecordReader<StringRecord> input2 = null;
-
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void invoke() throws Exception {
-
-		this.output.initializeSerializers();
-
-		while (this.input1.hasNext()) {
-
-			StringRecord s = input1.next();
-			this.output.emit(s);
-		}
-
-		while (this.input2.hasNext()) {
-
-			StringRecord s = input2.next();
-			this.output.emit(s);
-		}
-
-		this.output.flush();
-
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.input1 = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.input2 = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this);
-	}
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ForwardTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ForwardTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ForwardTask.java
deleted file mode 100644
index 2d0d8aa..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ForwardTask.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * 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.jobmanager;
-
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-public class ForwardTask extends AbstractInvokable {
-
-	private RecordReader<StringRecord> input = null;
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void invoke() throws Exception {
-
-		this.output.initializeSerializers();
-
-		while (this.input.hasNext()) {
-
-			StringRecord s = input.next();
-			this.output.emit(s);
-		}
-
-		this.output.flush();
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index 0b222f5..b8aac10 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -16,48 +16,27 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobmanager;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileReader;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
+import java.net.ServerSocket;
 
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.ExecutionMode;
-import org.apache.flink.runtime.client.JobClient;
-import org.apache.flink.runtime.client.JobExecutionException;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.client.AbstractJobResult;
+import org.apache.flink.runtime.client.JobSubmissionResult;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.operators.DataSinkTask;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.runtime.taskmanager.Task;
-import org.apache.flink.runtime.taskmanager.TaskManager;
-import org.apache.flink.runtime.testutils.ServerTestUtils;
-import org.apache.flink.runtime.testutils.tasks.DoubleSourceTask;
-import org.apache.flink.runtime.testutils.tasks.FileLineReader;
-import org.apache.flink.runtime.testutils.tasks.FileLineWriter;
-import org.apache.flink.runtime.testutils.tasks.JobFileInputVertex;
-import org.apache.flink.runtime.testutils.tasks.JobFileOutputVertex;
-import org.apache.flink.runtime.util.JarFileCreator;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobmanager.tasks.NoOpInvokable;
 import org.junit.Test;
 
 /**
@@ -65,990 +44,92 @@ import org.junit.Test;
  */
 public class JobManagerITCase {
 
-	/**
-	 * The name of the test directory some tests read their input from.
-	 */
-	private static final String INPUT_DIRECTORY = "testDirectory";
-
-	private static Configuration configuration;
-
-	private static JobManager jobManager;
-
-	/**
-	 * Starts the JobManager in local mode.
-	 */
-	@BeforeClass
-	public static void startNephele() {
-		try {
-			Configuration cfg = new Configuration();
-			cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "127.0.0.1");
-			cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, 6123);
-			cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 1);
-			cfg.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, 1);
-			
-			GlobalConfiguration.includeConfiguration(cfg);
-			
-			configuration = GlobalConfiguration.getConfiguration(new String[] { ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY });
-			
-			jobManager = new JobManager(ExecutionMode.LOCAL);
-
-			// Wait for the local task manager to arrive
-			ServerTestUtils.waitForJobManagerToBecomeReady(jobManager);
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			fail("Could not start job manager: " + e.getMessage());
-		}
-	}
-
-	/**
-	 * Stops the JobManager
-	 */
-	@AfterClass
-	public static void stopNephele() {
-		jobManager.shutdown();
-		jobManager = null;
-	}
-	
-	/**
-	 * Tests the correctness of the union record reader with non-empty inputs.
-	 */
-	@Test
-	public void testUnionWithNonEmptyInput() {
-		testUnion(1000000);
-	}
-
-	/**
-	 * Tests of the Nephele channels with a large (> 1 MB) file.
-	 */
 	@Test
-	public void testExecutionWithLargeInputFile() {
-		test(1000000);
-	}
-
-	/**
-	 * Tests of the Nephele channels with a file of zero bytes size.
-	 */
-	@Test
-	public void testExecutionWithZeroSizeInputFile() {
-		test(0);
-	}
-
-	/**
-	 * Tests the execution of a job with a directory as input. The test directory contains files of different length.
-	 */
-	@Test
-	public void testExecutionWithDirectoryInput() {
-
-		// Define size of input
-		final int sizeOfInput = 100;
-
-		// Create test directory
-		final String testDirectory = ServerTestUtils.getTempDir() + File.separator + INPUT_DIRECTORY;
-		final File td = new File(testDirectory);
-		if (!td.exists()) {
-			td.mkdir();
-		}
-
-		File inputFile1 = null;
-		File inputFile2 = null;
-		File outputFile = null;
-		File jarFile = null;
-		JobClient jobClient = null;
-
+	public void testSingleVertexJob() {
 		try {
-			// Get name of the forward class
-			final String forwardClassName = ForwardTask.class.getSimpleName();
-
-			// Create input and jar files
-			inputFile1 = ServerTestUtils.createInputFile(INPUT_DIRECTORY, 0);
-			inputFile2 = ServerTestUtils.createInputFile(INPUT_DIRECTORY, sizeOfInput);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-			jarFile = ServerTestUtils.createJarFile(forwardClassName);
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Job Graph 1");
-
-			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(new File(testDirectory).toURI()));
-			i1.setNumberOfSubtasks(1);
-
-			// task vertex 1
-			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
-			t1.setInvokableClass(ForwardTask.class);
-			t1.setNumberOfSubtasks(1);
-
-			// task vertex 2
-			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setInvokableClass(ForwardTask.class);
-			t2.setNumberOfSubtasks(1);
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-			o1.setNumberOfSubtasks(1);
-
-			t1.setVertexToShareInstancesWith(i1);
-			t2.setVertexToShareInstancesWith(i1);
-			o1.setVertexToShareInstancesWith(i1);
-
-			// connect vertices
-			try {
-				i1.connectTo(t1, ChannelType.NETWORK);
-				t1.connectTo(t2, ChannelType.IN_MEMORY);
-				t2.connectTo(o1, ChannelType.IN_MEMORY);
-			} catch (JobGraphDefinitionException e) {
-				e.printStackTrace();
-			}
-
-			// add jar
-			jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + forwardClassName + ".jar").toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
-			jobClient.submitJobAndWait();
-
-			// Finally, compare output file to initial number sequence
-			final BufferedReader bufferedReader = new BufferedReader(new FileReader(outputFile));
-			for (int i = 0; i < sizeOfInput; i++) {
-				final String number = bufferedReader.readLine();
-				try {
-					assertEquals(i, Integer.parseInt(number));
-				} catch (NumberFormatException e) {
-					fail(e.getMessage());
-				}
-			}
-
-			bufferedReader.close();
-
-		} catch (NumberFormatException e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		} catch (JobExecutionException e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		} catch (IOException ioe) {
-			ioe.printStackTrace();
-			fail(ioe.getMessage());
-		} finally {
-			// Remove temporary files
-			if (inputFile1 != null) {
-				inputFile1.delete();
-			}
-			if (inputFile2 != null) {
-				inputFile2.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			// Remove test directory
-			if (td != null) {
-				td.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
-			}
-		}
-	}
-
-	/**
-	 * Tests the Nephele execution when an exception occurs. In particular, it is tested if the information that is
-	 * wrapped by the exception is correctly passed on to the client.
-	 */
-	@Test
-	public void testExecutionWithException() {
-
-		final String exceptionClassName = ExceptionTask.class.getSimpleName();
-		File inputFile = null;
-		File outputFile = null;
-		File jarFile = null;
-		JobClient jobClient = null;
-
-		try {
-
-			inputFile = ServerTestUtils.createInputFile(0);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-			jarFile = ServerTestUtils.createJarFile(exceptionClassName);
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Job Graph for Exception Test");
-
-			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
-
-			// task vertex 1
-			final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg);
-			t1.setInvokableClass(ExceptionTask.class);
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-
-			t1.setVertexToShareInstancesWith(i1);
-			o1.setVertexToShareInstancesWith(i1);
-
-			// connect vertices
-			i1.connectTo(t1, ChannelType.IN_MEMORY);
-			t1.connectTo(o1, ChannelType.IN_MEMORY);
-
-			// add jar
-			jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + exceptionClassName + ".jar")
-				.toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
+			final AbstractJobVertex vertex = new AbstractJobVertex("Test Vertex");
+			vertex.setParallelism(3);
+			vertex.setInvokableClass(NoOpInvokable.class);
 			
-			try {
-				jobClient.submitJobAndWait();
-			} catch (JobExecutionException e) {
-				// Check if the correct error message is encapsulated in the exception
-				if (e.getMessage() == null) {
-					fail("JobExecutionException does not contain an error message");
-				}
-				if (!e.getMessage().contains(ExceptionTask.ERROR_MESSAGE)) {
-					fail("JobExecutionException does not contain the expected error message");
-				}
-
-				return;
-			}
-
-			fail("Expected exception but did not receive it");
-
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		} finally {
-
-			// Remove temporary files
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
-			}
-		}
-	}
-
-	/**
-	 * Tests the Nephele execution when a runtime exception during the registration of the input/output gates occurs.
-	 */
-	@Test
-	public void testExecutionWithRuntimeException() {
-
-		final String runtimeExceptionClassName = RuntimeExceptionTask.class.getSimpleName();
-		File inputFile = null;
-		File outputFile = null;
-		File jarFile = null;
-		JobClient jobClient = null;
-
-		try {
-
-			inputFile = ServerTestUtils.createInputFile(100);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-			jarFile = ServerTestUtils.createJarFile(runtimeExceptionClassName);
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Job Graph for Exception Test");
-
-			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
-
-			// task vertex 1
-			final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg);
-			t1.setInvokableClass(RuntimeExceptionTask.class);
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-
-			t1.setVertexToShareInstancesWith(i1);
-			o1.setVertexToShareInstancesWith(i1);
-
-			// connect vertices
-			i1.connectTo(t1, ChannelType.IN_MEMORY);
-			t1.connectTo(o1, ChannelType.IN_MEMORY);
-
-			// add jar
-			jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + runtimeExceptionClassName
-				+ ".jar").toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
+			final JobGraph jobGraph = new JobGraph("Test Job", vertex);
 			
+			JobManager jm = startJobManager();
 			try {
-				jobClient.submitJobAndWait();
-			} catch (JobExecutionException e) {
-
-				// Check if the correct error message is encapsulated in the exception
-				if (e.getMessage() == null) {
-					fail("JobExecutionException does not contain an error message");
-				}
-				if (!e.getMessage().contains(RuntimeExceptionTask.RUNTIME_EXCEPTION_MESSAGE)) {
-					fail("JobExecutionException does not contain the expected error message");
-				}
-
-				// Check if the correct error message is encapsulated in the exception
-				return;
-			}
-
-			fail("Expected exception but did not receive it");
-
-		} catch (JobGraphDefinitionException jgde) {
-			fail(jgde.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		} finally {
-
-			// Remove temporary files
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
-			}
-		}
-	}
-
-	/**
-	 * Tests the Nephele execution when a runtime exception in the output format occurs.
-	 */
-	@Test
-	public void testExecutionWithRuntimeExceptionInOutputFormat() {
-
-		final String runtimeExceptionClassName = RuntimeExceptionTask.class.getSimpleName();
-		File inputFile = null;
-		File outputFile = null;
-		File jarFile = null;
-		JobClient jobClient = null;
-
-		try {
-
-			inputFile = ServerTestUtils.createInputFile(100);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-			jarFile = ServerTestUtils.createJarFile(runtimeExceptionClassName);
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Job Graph for Exception Test");
-
-			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
-			i1.setNumberOfSubtasks(1);
-
-			// task vertex 1
-			final JobTaskVertex t1 = new JobTaskVertex("Task with Exception", jg);
-			t1.setInvokableClass(ForwardTask.class);
-
-			// output vertex
-			JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
-			o1.setNumberOfSubtasks(1);
-			o1.setInvokableClass(DataSinkTask.class);
-			ExceptionOutputFormat outputFormat = new ExceptionOutputFormat();
-			o1.setOutputFormat(outputFormat);
-			TaskConfig outputConfig = new TaskConfig(o1.getConfiguration());
-			outputConfig.setStubWrapper(new UserCodeObjectWrapper<OutputFormat<?>>(outputFormat));
-//			outputConfig.addInputToGroup(0);
-//			
-//			ValueSerializer<StringRecord> serializer = new ValueSerializer<StringRecord>(StringRecord.class);
-//			RuntimeStatefulSerializerFactory<StringRecord> serializerFactory = new RuntimeStatefulSerializerFactory<StringRecord>(serializer, StringRecord.class);
-//			outputConfig.setInputSerializer(serializerFactory, 0);
-
-			t1.setVertexToShareInstancesWith(i1);
-			o1.setVertexToShareInstancesWith(i1);
-
-			// connect vertices
-			i1.connectTo(t1, ChannelType.IN_MEMORY);
-			t1.connectTo(o1, ChannelType.IN_MEMORY);
-
-			// add jar
-			jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + runtimeExceptionClassName
-					+ ".jar").toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
-
-			try {
-				jobClient.submitJobAndWait();
-			} catch (JobExecutionException e) {
-
-				// Check if the correct error message is encapsulated in the exception
-				if (e.getMessage() == null) {
-					fail("JobExecutionException does not contain an error message");
-				}
-				if (!e.getMessage().contains(RuntimeExceptionTask.RUNTIME_EXCEPTION_MESSAGE)) {
-					fail("JobExecutionException does not contain the expected error message, " +
-							"but instead: " + e.getMessage());
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+				
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					
+					long deadline = System.currentTimeMillis() + 60*1000;
+					boolean success = false;
+					
+					while (System.currentTimeMillis() < deadline) {
+						JobStatus state = eg.getState();
+						if (state == JobStatus.FINISHED) {
+							success = true;
+							break;
+						}
+						
+						else if (state == JobStatus.FAILED || state == JobStatus.CANCELED) {
+							break;
+						}
+						else {
+							Thread.sleep(200);
+						}
+					}
+					
+					assertTrue("The job did not finish successfully.", success);
 				}
-
-				// Check if the correct error message is encapsulated in the exception
-				return;
-			}
-
-			fail("Expected exception but did not receive it");
-
-		} catch (JobGraphDefinitionException jgde) {
-			fail(jgde.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		} finally {
-
-			// Remove temporary files
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
-			}
-		}
-	}
-
-	/**
-	 * Creates a file with a sequence of 0 to <code>limit</code> integer numbers
-	 * and triggers a sample job. The sample reads all the numbers from the input file and pushes them through a
-	 * network, a file, and an in-memory channel. Eventually, the numbers are written back to an output file. The test
-	 * is considered successful if the input file equals the output file.
-	 * 
-	 * @param limit
-	 *        the upper bound for the sequence of numbers to be generated
-	 */
-	private void test(final int limit) {
-
-		JobClient jobClient = null;
-
-		try {
-
-			// Get name of the forward class
-			final String forwardClassName = ForwardTask.class.getSimpleName();
-
-			// Create input and jar files
-			final File inputFile = ServerTestUtils.createInputFile(limit);
-			final File outputFile = new File(ServerTestUtils.getTempDir() + File.separator
-				+ ServerTestUtils.getRandomFilename());
-			final File jarFile = ServerTestUtils.createJarFile(forwardClassName);
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Job Graph 1");
-
-			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
-			i1.setNumberOfSubtasks(1);
-
-			// task vertex 1
-			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
-			t1.setInvokableClass(ForwardTask.class);
-			t1.setNumberOfSubtasks(1);
-
-			// task vertex 2
-			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setInvokableClass(ForwardTask.class);
-			t2.setNumberOfSubtasks(1);
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-			o1.setNumberOfSubtasks(1);
-
-			t1.setVertexToShareInstancesWith(i1);
-			t2.setVertexToShareInstancesWith(i1);
-			o1.setVertexToShareInstancesWith(i1);
-
-			// connect vertices
-			try {
-				i1.connectTo(t1, ChannelType.NETWORK);
-				t1.connectTo(t2, ChannelType.IN_MEMORY);
-				t2.connectTo(o1, ChannelType.IN_MEMORY);
-			} catch (Exception e) {
-				e.printStackTrace();
-				fail(e.getMessage());
-			}
-
-			// add jar
-			jg.addJar(new Path(new File(ServerTestUtils.getTempDir() + File.separator + forwardClassName + ".jar")
-				.toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
-			
-			try {
-				jobClient.submitJobAndWait();
-			} catch (JobExecutionException e) {
-				fail(e.getMessage());
-			}
-
-			// Finally, compare output file to initial number sequence
-			final BufferedReader bufferedReader = new BufferedReader(new FileReader(outputFile));
-			for (int i = 0; i < limit; i++) {
-				final String number = bufferedReader.readLine();
-				try {
-					assertEquals(i, Integer.parseInt(number));
-				} catch (NumberFormatException e) {
-					fail(e.getMessage());
+				else {
+					// already done, that was fast;
 				}
 			}
-
-			bufferedReader.close();
-
-			// Remove temporary files
-			inputFile.delete();
-			outputFile.delete();
-			jarFile.delete();
-
-		} catch (IOException ioe) {
-			ioe.printStackTrace();
-			fail(ioe.getMessage());
-		} finally {
-			if (jobClient != null) {
-				jobClient.close();
+			finally {
+				jm.shutdown();
 			}
 		}
-	}
-
-	/**
-	 * Tests the Nephele execution with a job that has two vertices, that are connected twice with each other with
-	 * different channel types.
-	 */
-	@Test
-	public void testExecutionDoubleConnection() {
-
-		File inputFile = null;
-		File outputFile = null;
-		File jarFile = new File(ServerTestUtils.getTempDir() + File.separator + "doubleConnection.jar");
-		JobClient jobClient = null;
-
-		try {
-
-			inputFile = ServerTestUtils.createInputFile(0);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-
-			// Create required jar file
-			JarFileCreator jfc = new JarFileCreator(jarFile);
-			jfc.addClass(DoubleSourceTask.class);
-			jfc.addClass(DoubleTargetTask.class);
-			jfc.createJarFile();
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Job Graph for Double Connection Test");
-
-			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input with two Outputs", jg);
-			i1.setInvokableClass(DoubleSourceTask.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
-
-			// task vertex 1
-			final JobTaskVertex t1 = new JobTaskVertex("Task with two Inputs", jg);
-			t1.setInvokableClass(DoubleTargetTask.class);
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex("Output 1", jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-
-			t1.setVertexToShareInstancesWith(i1);
-			o1.setVertexToShareInstancesWith(i1);
-
-			// connect vertices
-			i1.connectTo(t1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-			i1.connectTo(t1, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-			t1.connectTo(o1, ChannelType.IN_MEMORY);
-
-			// add jar
-			jg.addJar(new Path(jarFile.toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
-			jobClient.submitJobAndWait();
-
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		} finally {
-
-			// Remove temporary files
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
-			}
-		}
-	}
-
-	/**
-	 * Tests the Nephele job execution when the graph and the tasks are given no specific name.
-	 */
-	@Test
-	public void testEmptyTaskNames() {
-
-		File inputFile = null;
-		File outputFile = null;
-		File jarFile = new File(ServerTestUtils.getTempDir() + File.separator + "emptyNames.jar");
-		JobClient jobClient = null;
-
-		try {
-
-			inputFile = ServerTestUtils.createInputFile(0);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-
-			// Create required jar file
-			JarFileCreator jfc = new JarFileCreator(jarFile);
-			jfc.addClass(DoubleSourceTask.class);
-			jfc.addClass(DoubleTargetTask.class);
-			jfc.createJarFile();
-
-			// Create job graph
-			final JobGraph jg = new JobGraph();
-
-			// input vertex
-			final JobFileInputVertex i1 = new JobFileInputVertex(jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile.toURI()));
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex(jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-
-			o1.setVertexToShareInstancesWith(i1);
-
-			// connect vertices
-			i1.connectTo(o1, ChannelType.IN_MEMORY);
-
-			// add jar
-			jg.addJar(new Path(jarFile.toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
-			jobClient.submitJobAndWait();
-		} catch (Exception e) {
+		catch (Exception e) {
 			e.printStackTrace();
 			fail(e.getMessage());
-		} finally {
-
-			// Remove temporary files
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
-			}
 		}
 	}
-
-	/**
-	 * Tests the correctness of the union record reader with empty inputs.
-	 */
-	@Test
-	public void testUnionWithEmptyInput() {
-		testUnion(0);
-	}
-
-	/**
-	 * Tests the correctness of the union reader for different input sizes.
-	 * 
-	 * @param limit
-	 *        the upper bound for the sequence of numbers to be generated
-	 */
-	private void testUnion(final int limit) {
-
-		File inputFile1 = null;
-		File inputFile2 = null;
-		File outputFile = null;
-		File jarFile = new File(ServerTestUtils.getTempDir() + File.separator + "unionWithEmptyInput.jar");
-		JobClient jobClient = null;
-
-		try {
-
-			inputFile1 = ServerTestUtils.createInputFile(limit);
-			inputFile2 = ServerTestUtils.createInputFile(limit);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-
-			// Create required jar file
-			JarFileCreator jfc = new JarFileCreator(jarFile);
-			jfc.addClass(UnionTask.class);
-			jfc.createJarFile();
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Union job " + limit);
-
-			// input vertex 1
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile1.toURI()));
-
-			// input vertex 2
-			final JobFileInputVertex i2 = new JobFileInputVertex("Input 2", jg);
-			i2.setInvokableClass(FileLineReader.class);
-			i2.setFilePath(new Path(inputFile2.toURI()));
-
-			// union task
-			final JobTaskVertex u1 = new JobTaskVertex("Union", jg);
-			u1.setInvokableClass(UnionTask.class);
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex("Output", jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-			o1.setNumberOfSubtasks(1);
-
-			i1.setVertexToShareInstancesWith(o1);
-			i2.setVertexToShareInstancesWith(o1);
-			u1.setVertexToShareInstancesWith(o1);
-
-			// connect vertices
-			i1.connectTo(u1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-			i2.connectTo(u1, ChannelType.IN_MEMORY);
-			u1.connectTo(o1, ChannelType.IN_MEMORY);
-
-			// add jar
-			jg.addJar(new Path(jarFile.toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
-
-			try {
-				jobClient.submitJobAndWait();
-			} catch (JobExecutionException e) {
-				fail(e.getMessage());
-			}
-
-			// Finally, check the output
-			final Map<Integer, Integer> expectedNumbers = new HashMap<Integer, Integer>();
-			final Integer two = Integer.valueOf(2);
-			for (int i = 0; i < limit; ++i) {
-				expectedNumbers.put(Integer.valueOf(i), two);
-			}
-
-			final BufferedReader bufferedReader = new BufferedReader(new FileReader(outputFile));
-			String line = bufferedReader.readLine();
-			while (line != null) {
-
-				final Integer number = Integer.valueOf(Integer.parseInt(line));
-				Integer val = expectedNumbers.get(number);
-				if (val == null) {
-					fail("Found unexpected number in union output: " + number);
-				} else {
-					val = Integer.valueOf(val.intValue() - 1);
-					if (val.intValue() < 0) {
-						fail(val + " occurred more than twice in union output");
-					}
-					if (val.intValue() == 0) {
-						expectedNumbers.remove(number);
-					} else {
-						expectedNumbers.put(number, val);
-					}
-				}
-
-				line = bufferedReader.readLine();
-			}
-
-			bufferedReader.close();
-
-			if (!expectedNumbers.isEmpty()) {
-				final StringBuilder str = new StringBuilder();
-				str.append("The following numbers have not been found in the union output:\n");
-				final Iterator<Map.Entry<Integer, Integer>> it = expectedNumbers.entrySet().iterator();
-				while (it.hasNext()) {
-					final Map.Entry<Integer, Integer> entry = it.next();
-					str.append(entry.getKey().toString());
-					str.append(" (");
-					str.append(entry.getValue().toString());
-					str.append("x)\n");
-				}
-
-				fail(str.toString());
-			}
-
-		} catch (JobGraphDefinitionException jgde) {
-			fail(jgde.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		} finally {
-
-			// Remove temporary files
-			if (inputFile1 != null) {
-				inputFile1.delete();
-			}
-			if (inputFile2 != null) {
-				inputFile2.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
-			}
-		}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static final JobManager startJobManager() throws Exception {
+		Configuration cfg = new Configuration();
+		cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+		cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, getAvailablePort());
+		cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10);
+		
+		GlobalConfiguration.includeConfiguration(cfg);
+		
+		JobManager jm = new JobManager(ExecutionMode.LOCAL);
+		return jm;
 	}
-
-	/**
-	 * Tests the execution of a job with a large degree of parallelism. In particular, the tests checks that the overall
-	 * runtime of the test does not exceed a certain time limit.
-	 */
-	@Test
-	public void testExecutionWithLargeDoP() {
-
-		// The degree of parallelism to be used by tasks in this job.
-		final int numberOfSubtasks = 64;
-
-		File inputFile1 = null;
-		File inputFile2 = null;
-		File outputFile = null;
-		File jarFile = new File(ServerTestUtils.getTempDir() + File.separator + "largeDoP.jar");
-		JobClient jobClient = null;
-
-		try {
-
-			inputFile1 = ServerTestUtils.createInputFile(0);
-			inputFile2 = ServerTestUtils.createInputFile(0);
-			outputFile = new File(ServerTestUtils.getTempDir() + File.separator + ServerTestUtils.getRandomFilename());
-
-			// Create required jar file
-			JarFileCreator jfc = new JarFileCreator(jarFile);
-			jfc.addClass(UnionTask.class);
-			jfc.createJarFile();
-
-			// Create job graph
-			final JobGraph jg = new JobGraph("Job with large DoP (" + numberOfSubtasks + ")");
-
-			// input vertex 1
-			final JobFileInputVertex i1 = new JobFileInputVertex("Input 1", jg);
-			i1.setInvokableClass(FileLineReader.class);
-			i1.setFilePath(new Path(inputFile1.toURI()));
-			i1.setNumberOfSubtasks(numberOfSubtasks);
-
-			// input vertex 2
-			final JobFileInputVertex i2 = new JobFileInputVertex("Input 2", jg);
-			i2.setInvokableClass(FileLineReader.class);
-			i2.setFilePath(new Path(inputFile2.toURI()));
-			i2.setNumberOfSubtasks(numberOfSubtasks);
-
-			// union task
-			final JobTaskVertex f1 = new JobTaskVertex("Forward 1", jg);
-			f1.setInvokableClass(DoubleTargetTask.class);
-			f1.setNumberOfSubtasks(numberOfSubtasks);
-
-			// output vertex
-			JobFileOutputVertex o1 = new JobFileOutputVertex("Output", jg);
-			o1.setInvokableClass(FileLineWriter.class);
-			o1.setFilePath(new Path(outputFile.toURI()));
-			o1.setNumberOfSubtasks(numberOfSubtasks);
-
-			i1.setVertexToShareInstancesWith(o1);
-			i2.setVertexToShareInstancesWith(o1);
-			f1.setVertexToShareInstancesWith(o1);
-
-			// connect vertices
-			i1.connectTo(f1, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-			i2.connectTo(f1, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-			f1.connectTo(o1, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-
-			// add jar
-			jg.addJar(new Path(jarFile.toURI()));
-
-			// Create job client and launch job
-			jobClient = new JobClient(jg, configuration, getClass().getClassLoader());
-			
+	
+	private static int getAvailablePort() throws IOException {
+		for (int i = 0; i < 50; i++) {
+			ServerSocket serverSocket = null;
 			try {
-				
-				jobClient.submitJobAndWait();
-			} catch (JobExecutionException e) {
-				// Job execution should lead to an error due to lack of resources
-				return;
-			} catch (Exception e) {
-				e.printStackTrace();
-				fail(e.getMessage());
-			}
-
-			fail("Undetected lack of resources");
-
-		} catch (JobGraphDefinitionException jgde) {
-			fail(jgde.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		} finally {
-
-			// Remove temporary files
-			if (inputFile1 != null) {
-				inputFile1.delete();
-			}
-			if (inputFile2 != null) {
-				inputFile2.delete();
-			}
-			if (outputFile != null) {
-				if (outputFile.isDirectory()) {
-					final String[] files = outputFile.list();
-					final String outputDir = outputFile.getAbsolutePath();
-					for (final String file : files) {
-						new File(outputDir + File.separator + file).delete();
-					}
+				serverSocket = new ServerSocket(0);
+				int port = serverSocket.getLocalPort();
+				if (port != 0) {
+					return port;
 				}
-				outputFile.delete();
-			}
-			if (jarFile != null) {
-				jarFile.delete();
-			}
-
-			if (jobClient != null) {
-				jobClient.close();
+			} finally {
+				serverSocket.close();
 			}
 		}
+		
+		throw new IOException("could not find free port");
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
new file mode 100644
index 0000000..835af95
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
@@ -0,0 +1,23 @@
+/**
+ * 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.jobmanager;
+
+public class JobManagerTest {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/RuntimeExceptionTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/RuntimeExceptionTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/RuntimeExceptionTask.java
deleted file mode 100644
index 6004bc1..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/RuntimeExceptionTask.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * 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.jobmanager;
-
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * This task throws a {@link RuntimeException} when the method <code>registerInputOutput</code> is called.
- */
-public class RuntimeExceptionTask extends AbstractInvokable {
-
-	/**
-	 * The message which is used for the test runtime exception.
-	 */
-	public static final String RUNTIME_EXCEPTION_MESSAGE = "This is a test runtime exception";
-
-
-	@Override
-	public void registerInputOutput() {
-		throw new RuntimeException(RUNTIME_EXCEPTION_MESSAGE);
-	}
-
-	@Override
-	public void invoke() {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/UnionTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/UnionTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/UnionTask.java
deleted file mode 100644
index 9d01907..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/UnionTask.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * 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.jobmanager;
-
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.MutableRecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.io.network.api.UnionRecordReader;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * A simple implementation of a task using a {@link UnionRecordReader}.
- */
-public class UnionTask extends AbstractInvokable {
-
-	/**
-	 * The union record reader to be used during the tests.
-	 */
-	private UnionRecordReader<StringRecord> unionReader;
-
-	private RecordWriter<StringRecord> writer;
-	
-	
-	@Override
-	public void registerInputOutput() {
-
-		@SuppressWarnings("unchecked")
-		MutableRecordReader<StringRecord>[] recordReaders = (MutableRecordReader<StringRecord>[]) new MutableRecordReader<?>[2];
-		recordReaders[0] = new MutableRecordReader<StringRecord>(this);
-		recordReaders[1] = new MutableRecordReader<StringRecord>(this);
-		this.unionReader = new UnionRecordReader<StringRecord>(recordReaders, StringRecord.class);
-		
-		this.writer = new RecordWriter<StringRecord>(this);
-	}
-
-	@Override
-	public void invoke() throws Exception {
-		this.writer.initializeSerializers();
-
-		while (this.unionReader.hasNext()) {
-			this.writer.emit(this.unionReader.next());
-		}
-
-		this.writer.flush();
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
deleted file mode 100644
index 2b2c81d..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import static org.junit.Assert.*;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.core.io.InputSplit;
-import org.junit.Test;
-
-
-public class DefaultSplitAssignerTest {
-
-	@Test
-	public void testSerialSplitAssignment() {
-		try {
-			final int NUM_SPLITS = 50;
-			
-			Set<InputSplit> splits = new HashSet<InputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new GenericInputSplit(i, NUM_SPLITS));
-			}
-			
-			DefaultInputSplitAssigner ia = new DefaultInputSplitAssigner(splits);
-			InputSplit is = null;
-			while ((is = ia.getNextInputSplit("")) != null) {
-				assertTrue(splits.remove(is));
-			}
-			
-			assertTrue(splits.isEmpty());
-			assertNull(ia.getNextInputSplit(""));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testConcurrentSplitAssignment() {
-		try {
-			final int NUM_THREADS = 10;
-			final int NUM_SPLITS = 500;
-			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
-			
-			Set<InputSplit> splits = new HashSet<InputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new GenericInputSplit(i, NUM_SPLITS));
-			}
-			
-			final DefaultInputSplitAssigner ia = new DefaultInputSplitAssigner(splits);
-			
-			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
-			final AtomicInteger sumOfIds = new AtomicInteger(0);
-			
-			Runnable retriever = new Runnable() {
-				
-				@Override
-				public void run() {
-					String host = "";
-					GenericInputSplit split;
-					while ((split = (GenericInputSplit) ia.getNextInputSplit(host)) != null) {
-						splitsRetrieved.incrementAndGet();
-						sumOfIds.addAndGet(split.getSplitNumber());
-					}
-				}
-			};
-			
-			// create the threads
-			Thread[] threads = new Thread[NUM_THREADS];
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i] = new Thread(retriever);
-				threads[i].setDaemon(true);
-			}
-			
-			// launch concurrently
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].start();
-			}
-			
-			// sync
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].join(5000);
-			}
-			
-			// verify
-			for (int i = 0; i < NUM_THREADS; i++) {
-				if (threads[i].isAlive()) {
-					fail("The concurrency test case is erroneous, the thread did not respond in time.");
-				}
-			}
-			
-			assertEquals(NUM_SPLITS, splitsRetrieved.get());
-			assertEquals(SUM_OF_IDS, sumOfIds.get());
-			
-			// nothing left
-			assertNull(ia.getNextInputSplit(""));
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
deleted file mode 100644
index fdc1cf4..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
+++ /dev/null
@@ -1,382 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import static org.junit.Assert.*;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.io.LocatableInputSplit;
-
-import org.junit.Test;
-
-
-public class LocatableSplitAssignerTest {
-	
-	@Test
-	public void testSerialSplitAssignmentWithNullHost() {
-		try {
-			final int NUM_SPLITS = 50;
-			final String[][] hosts = new String[][] {
-					new String[] { "localhost" },
-					new String[0],
-					null
-			};
-			
-			// load some splits
-			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new LocatableInputSplit(i, hosts[i%3]));
-			}
-			
-			// get all available splits
-			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
-			InputSplit is = null;
-			while ((is = ia.getNextInputSplit(null)) != null) {
-				assertTrue(splits.remove(is));
-			}
-			
-			// check we had all
-			assertTrue(splits.isEmpty());
-			assertNull(ia.getNextInputSplit(""));
-			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
-			assertEquals(0, ia.getNumberOfLocalAssignments());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSerialSplitAssignmentAllForSameHost() {
-		try {
-			final int NUM_SPLITS = 50;
-			
-			// load some splits
-			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new LocatableInputSplit(i, "testhost"));
-			}
-			
-			// get all available splits
-			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
-			InputSplit is = null;
-			while ((is = ia.getNextInputSplit("testhost")) != null) {
-				assertTrue(splits.remove(is));
-			}
-			
-			// check we had all
-			assertTrue(splits.isEmpty());
-			assertNull(ia.getNextInputSplit(""));
-			
-			assertEquals(0, ia.getNumberOfRemoteAssignments());
-			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSerialSplitAssignmentAllForRemoteHost() {
-		try {
-			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
-			final int NUM_SPLITS = 10 * hosts.length;
-			
-			// load some splits
-			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new LocatableInputSplit(i, hosts[i % hosts.length]));
-			}
-			
-			// get all available splits
-			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
-			InputSplit is = null;
-			while ((is = ia.getNextInputSplit("testhost")) != null) {
-				assertTrue(splits.remove(is));
-			}
-			
-			// check we had all
-			assertTrue(splits.isEmpty());
-			assertNull(ia.getNextInputSplit("anotherHost"));
-			
-			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
-			assertEquals(0, ia.getNumberOfLocalAssignments());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testSerialSplitAssignmentMixedLocalHost() {
-		try {
-			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
-			final int NUM_SPLITS = 10 * hosts.length;
-			
-			// load some splits
-			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new LocatableInputSplit(i, hosts[i % hosts.length]));
-			}
-			
-			// get all available splits
-			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
-			InputSplit is = null;
-			int i = 0;
-			while ((is = ia.getNextInputSplit(hosts[i++ % hosts.length])) != null) {
-				assertTrue(splits.remove(is));
-			}
-			
-			// check we had all
-			assertTrue(splits.isEmpty());
-			assertNull(ia.getNextInputSplit("anotherHost"));
-			
-			assertEquals(0, ia.getNumberOfRemoteAssignments());
-			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testConcurrentSplitAssignmentNullHost() {
-		try {
-			final int NUM_THREADS = 10;
-			final int NUM_SPLITS = 500;
-			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
-			
-			final String[][] hosts = new String[][] {
-					new String[] { "localhost" },
-					new String[0],
-					null
-			};
-			
-			// load some splits
-			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new LocatableInputSplit(i, hosts[i%3]));
-			}
-			
-			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
-			
-			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
-			final AtomicInteger sumOfIds = new AtomicInteger(0);
-			
-			Runnable retriever = new Runnable() {
-				
-				@Override
-				public void run() {
-					LocatableInputSplit split;
-					while ((split = ia.getNextInputSplit(null)) != null) {
-						splitsRetrieved.incrementAndGet();
-						sumOfIds.addAndGet(split.getSplitNumber());
-					}
-				}
-			};
-			
-			// create the threads
-			Thread[] threads = new Thread[NUM_THREADS];
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i] = new Thread(retriever);
-				threads[i].setDaemon(true);
-			}
-			
-			// launch concurrently
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].start();
-			}
-			
-			// sync
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].join(5000);
-			}
-			
-			// verify
-			for (int i = 0; i < NUM_THREADS; i++) {
-				if (threads[i].isAlive()) {
-					fail("The concurrency test case is erroneous, the thread did not respond in time.");
-				}
-			}
-			
-			assertEquals(NUM_SPLITS, splitsRetrieved.get());
-			assertEquals(SUM_OF_IDS, sumOfIds.get());
-			
-			// nothing left
-			assertNull(ia.getNextInputSplit(""));
-			
-			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
-			assertEquals(0, ia.getNumberOfLocalAssignments());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testConcurrentSplitAssignmentForSingleHost() {
-		try {
-			final int NUM_THREADS = 10;
-			final int NUM_SPLITS = 500;
-			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
-			
-			// load some splits
-			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new LocatableInputSplit(i, "testhost"));
-			}
-			
-			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
-			
-			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
-			final AtomicInteger sumOfIds = new AtomicInteger(0);
-			
-			Runnable retriever = new Runnable() {
-				
-				@Override
-				public void run() {
-					LocatableInputSplit split;
-					while ((split = ia.getNextInputSplit("testhost")) != null) {
-						splitsRetrieved.incrementAndGet();
-						sumOfIds.addAndGet(split.getSplitNumber());
-					}
-				}
-			};
-			
-			// create the threads
-			Thread[] threads = new Thread[NUM_THREADS];
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i] = new Thread(retriever);
-				threads[i].setDaemon(true);
-			}
-			
-			// launch concurrently
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].start();
-			}
-			
-			// sync
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].join(5000);
-			}
-			
-			// verify
-			for (int i = 0; i < NUM_THREADS; i++) {
-				if (threads[i].isAlive()) {
-					fail("The concurrency test case is erroneous, the thread did not respond in time.");
-				}
-			}
-			
-			assertEquals(NUM_SPLITS, splitsRetrieved.get());
-			assertEquals(SUM_OF_IDS, sumOfIds.get());
-			
-			// nothing left
-			assertNull(ia.getNextInputSplit("testhost"));
-			
-			assertEquals(0, ia.getNumberOfRemoteAssignments());
-			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testConcurrentSplitAssignmentForMultipleHosts() {
-		try {
-			final int NUM_THREADS = 10;
-			final int NUM_SPLITS = 500;
-			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
-			
-			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
-			
-			// load some splits
-			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
-			for (int i = 0; i < NUM_SPLITS; i++) {
-				splits.add(new LocatableInputSplit(i, hosts[i%hosts.length]));
-			}
-			
-			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
-			
-			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
-			final AtomicInteger sumOfIds = new AtomicInteger(0);
-			
-			Runnable retriever = new Runnable() {
-				
-				@Override
-				public void run() {
-					final String threadHost = hosts[(int) (Math.random() * hosts.length)];
-					
-					LocatableInputSplit split;
-					while ((split = ia.getNextInputSplit(threadHost)) != null) {
-						splitsRetrieved.incrementAndGet();
-						sumOfIds.addAndGet(split.getSplitNumber());
-					}
-				}
-			};
-			
-			// create the threads
-			Thread[] threads = new Thread[NUM_THREADS];
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i] = new Thread(retriever);
-				threads[i].setDaemon(true);
-			}
-			
-			// launch concurrently
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].start();
-			}
-			
-			// sync
-			for (int i = 0; i < NUM_THREADS; i++) {
-				threads[i].join(5000);
-			}
-			
-			// verify
-			for (int i = 0; i < NUM_THREADS; i++) {
-				if (threads[i].isAlive()) {
-					fail("The concurrency test case is erroneous, the thread did not respond in time.");
-				}
-			}
-			
-			assertEquals(NUM_SPLITS, splitsRetrieved.get());
-			assertEquals(SUM_OF_IDS, sumOfIds.get());
-			
-			// nothing left
-			assertNull(ia.getNextInputSplit("testhost"));
-			
-			// at least one fraction of hosts needs be local, no matter how bad the thread races
-			assertTrue(ia.getNumberOfLocalAssignments() >= NUM_SPLITS / hosts.length);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/NoOpInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/NoOpInvokable.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/NoOpInvokable.java
new file mode 100644
index 0000000..c58da13
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/NoOpInvokable.java
@@ -0,0 +1,33 @@
+/**
+ * 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.jobmanager.tasks;
+
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+
+/**
+ * An invokable that does nothing.
+ */
+public class NoOpInvokable extends AbstractInvokable {
+
+	@Override
+	public void registerInputOutput() {}
+
+	@Override
+	public void invoke() {}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java
index 93e52f6..c20e5dc 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/SortMergeMatchIteratorITCase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.sort;
 
 import java.util.ArrayList;
@@ -37,7 +36,6 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
 import org.apache.flink.runtime.operators.sort.MergeIterator;
 import org.apache.flink.runtime.operators.sort.MergeMatchIterator;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
index ccb5215..86c32ac 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.testutils;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
index aed4c78..a4d0f32 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
@@ -250,16 +250,6 @@ public class MockEnvironment implements Environment, BufferProvider, LocalBuffer
 	}
 
 	@Override
-	public void userThreadStarted(final Thread userThread) {
-		// Nothing to do here
-	}
-
-	@Override
-	public void userThreadFinished(final Thread userThread) {
-		// Nothing to do here
-	}
-
-	@Override
 	public InputSplitProvider getInputSplitProvider() {
 		return this.inputSplitProvider;
 	}


[18/63] [abbrv] git commit: Stubs for intermediate data set and related classes

Posted by se...@apache.org.
Stubs for intermediate data set and related classes


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/c16f6d81
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/c16f6d81
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/c16f6d81

Branch: refs/heads/master
Commit: c16f6d816899d8db7ff7c809d0c26be611b4d561
Parents: 9035b6d
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Jun 30 17:12:10 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../runtime/jobgraph/AbstractJobVertex.java     | 13 +++--
 .../apache/flink/runtime/jobgraph/JobGraph.java | 22 ---------
 .../nephele/jobgraph/IntermediateDataSet.java   | 42 ++++++++++++++++
 .../nephele/jobgraph/IntermediateDataSetID.java | 50 ++++++++++++++++++++
 4 files changed, 100 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c16f6d81/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index 7df76c3..cc7a5d0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -42,19 +42,19 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	private static final String DEFAULT_NAME = "(unnamed vertex)";
 	
 	/**
-	 * List of outgoing edges.
+	 * List of produced data sets, one per writer
 	 */
-	private final ArrayList<JobEdge> forwardEdges = new ArrayList<JobEdge>();
+	private final ArrayList<IntermediateDataSet> results = new ArrayList<IntermediateDataSet>();
 
 	/**
-	 * List of incoming edges.
+	 * List of edges with incoming data. One per Reader.
 	 */
-	private final ArrayList<JobEdge> backwardEdges = new ArrayList<JobEdge>();
+	private final ArrayList<JobEdge> inputs = new ArrayList<JobEdge>();
 
 	/**
 	 * The name of the vertex or task, respectively.
 	 */
-	private String name;
+	private final String name;
 
 	/**
 	 * The ID of the vertex.
@@ -86,6 +86,7 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	 */
 	protected Class<? extends AbstractInvokable> invokableClass;
 
+	// --------------------------------------------------------------------------------------------
 	
 	/**
 	 * Constructs a new job vertex and assigns it with the given name.
@@ -112,6 +113,8 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 		this.id = id == null ? new JobVertexID() : id;
 		this.jobGraph = jobGraph;
 	}
+	
+	// --------------------------------------------------------------------------------------------
 
 	/**
 	 * Connects the job vertex to the specified job vertex.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c16f6d81/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 48d858a..39dc382 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -192,30 +192,8 @@ public class JobGraph implements IOReadableWritable {
 	 */
 	public AbstractJobVertex[] getAllJobVertices() {
 		return this.taskVertices.values().toArray(new AbstractJobVertex[this.taskVertices.size()]);
-		
-		int i = 0;
-		final AbstractJobVertex[] vertices = new AbstractJobVertex[inputVertices.size() + outputVertices.size()
-			+ taskVertices.size()];
-
-		final Iterator<AbstractJobInputVertex> iv = getInputVertices();
-		while (iv.hasNext()) {
-			vertices[i++] = iv.next();
-		}
-
-		final Iterator<AbstractJobOutputVertex> ov = getOutputVertices();
-		while (ov.hasNext()) {
-			vertices[i++] = ov.next();
-		}
-
-		final Iterator<JobTaskVertex> tv = getTaskVertices();
-		while (tv.hasNext()) {
-			vertices[i++] = tv.next();
-		}
-
-		return vertices;
 	}
 
-
 	/**
 	 * Returns the ID of the job.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c16f6d81/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java
new file mode 100644
index 0000000..fdc3375
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSet.java
@@ -0,0 +1,42 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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 eu.stratosphere.nephele.jobgraph;
+
+/**
+ * An intermediate data set is the data set produced by an operator - either a
+ * source or any intermediate operation.
+ * 
+ * Intermediate data sets may be read by other operators, materialized, or
+ * discarded.
+ */
+public class IntermediateDataSet {
+	
+	private final IntermediateDataSetID id; 		// the identifier
+	
+	private final AbstractJobVertex producer;		// the operation that produced this data set
+
+	
+	public IntermediateDataSet(AbstractJobVertex producer) {
+		this(new IntermediateDataSetID(), producer);
+	}
+	
+	public IntermediateDataSet(IntermediateDataSetID id, AbstractJobVertex producer) {
+		this.id = id;
+		this.producer = producer;
+	}
+	
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c16f6d81/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java
----------------------------------------------------------------------
diff --git a/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java
new file mode 100644
index 0000000..ac12be9
--- /dev/null
+++ b/stratosphere-runtime/src/main/java/eu/stratosphere/nephele/jobgraph/IntermediateDataSetID.java
@@ -0,0 +1,50 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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 eu.stratosphere.nephele.jobgraph;
+
+import java.util.UUID;
+
+import eu.stratosphere.nephele.AbstractID;
+
+public class IntermediateDataSetID extends AbstractID {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Creates an new random intermediate data set ID.
+	 */
+	public IntermediateDataSetID() {
+		super();
+	}
+	
+	/**
+	 * Creates a new intermediate data set ID with the bytes of the given ID.
+	 * 
+	 * @param from The ID to create this ID from.
+	 */
+	public IntermediateDataSetID(AbstractID from) {
+		super(from);
+	}
+	
+	/**
+	 * Creates a new intermediate data set ID with the bytes of the given UUID.
+	 * 
+	 * @param from The UUID to create this ID from.
+	 */
+	public IntermediateDataSetID(UUID from) {
+		super(from.getLeastSignificantBits(), from.getMostSignificantBits());
+	}
+}


[49/63] [abbrv] git commit: Fix serializability for SlotSharingGroup

Posted by se...@apache.org.
Fix serializability for SlotSharingGroup


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/e56d883f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/e56d883f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/e56d883f

Branch: refs/heads/master
Commit: e56d883faa6b94b9b2895643f818679fd242d7fe
Parents: ef7957f
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 16:54:14 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:50 2014 +0200

----------------------------------------------------------------------
 .../flink/runtime/jobmanager/scheduler/SlotSharingGroup.java | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e56d883f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
index b8b2ed1..30fe7c8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
@@ -29,11 +29,15 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
  * deployed together within a slot. This is a soft permission, in contrast to the hard constraint
  * defined by a co-location hint.
  */
-public class SlotSharingGroup {
+public class SlotSharingGroup implements java.io.Serializable {
 	
+	private static final long serialVersionUID = 1L;
+	
+
 	private final Set<JobVertexID> ids = new TreeSet<JobVertexID>();
 	
-	private SlotSharingGroupAssignment taskAssignment;
+	/** Mapping of tasks to subslots. This field is only needed inside the JobManager, and is not RPCed. */
+	private transient SlotSharingGroupAssignment taskAssignment;
 	
 	
 	public SlotSharingGroup() {}


[17/63] [abbrv] git commit: Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
Refactor job graph construction to incremental attachment based


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/b32e77a2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/b32e77a2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/b32e77a2

Branch: refs/heads/master
Commit: b32e77a2d8be76aeafa28b94fd7cfbb8de80f4cb
Parents: c16f6d8
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Jun 30 21:16:15 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 docs/img/state_machine.svg                      |  235 ++
 .../mapred/HadoopInputFormat.java               |    7 +-
 .../mapred/record/HadoopRecordInputFormat.java  |    8 +-
 .../mapred/record/HadoopRecordOutputFormat.java |    2 +-
 .../mapred/wrapper/HadoopInputSplit.java        |    6 +
 .../mapreduce/HadoopInputFormat.java            |    7 +-
 .../flink/addons/hbase/TableInputFormat.java    |   20 +-
 .../flink/addons/hbase/TableInputSplit.java     |    2 +
 .../flink/api/java/io/jdbc/JDBCInputFormat.java |   11 +-
 .../org/apache/flink/client/CliFrontend.java    |    2 +-
 .../flink/client/CliFrontendListCancelTest.java |   11 -
 .../plantranslate/NepheleJobGraphGenerator.java |   10 +-
 .../common/io/DefaultInputSplitAssigner.java    |   75 +
 .../flink/api/common/io/FileInputFormat.java    |    7 +-
 .../flink/api/common/io/FileOutputFormat.java   |    1 -
 .../apache/flink/api/common/io/FormatUtil.java  |  186 --
 .../flink/api/common/io/GenericInputFormat.java |    5 +-
 .../flink/api/common/io/InitializeOnMaster.java |    1 -
 .../apache/flink/api/common/io/InputFormat.java |    9 +-
 .../common/io/LocatableInputSplitAssigner.java  |  197 ++
 .../flink/api/common/io/NonParallelInput.java   |    3 +-
 .../flink/api/common/io/OutputFormat.java       |    2 -
 .../apache/flink/core/fs/FileInputSplit.java    |   15 +-
 .../apache/flink/core/io/GenericInputSplit.java |    2 +-
 .../flink/core/io/IOReadableWritable.java       |    1 -
 .../apache/flink/core/io/InputSplitSource.java  |   48 +
 .../flink/core/memory/DataInputViewStream.java  |   43 +
 .../flink/core/memory/DataOutputViewStream.java |   43 +
 .../java/org/apache/flink/util/ClassUtils.java  |    1 -
 .../org/apache/flink/util/ExceptionUtils.java   |   78 +
 .../flink/util/NumberSequenceIterator.java      |   24 +-
 .../apache/flink/util/SimpleStringUtils.java    |    2 -
 .../flink/util/TraversableOnceException.java    |    7 +
 .../apache/flink/api/common/io/FormatUtil.java  |  186 ++
 .../flink/core/io/DefaultSplitAssignerTest.java |  125 +
 .../core/io/LocatableSplitAssignerTest.java     |  385 +++
 ...ternalProcessFixedLengthInputFormatTest.java |    6 +-
 .../io/ExternalProcessInputFormatTest.java      |    9 +-
 .../TypeExtractorInputFormatsTest.java          |    5 +-
 .../org/apache/flink/runtime/AbstractID.java    |   20 +-
 .../org/apache/flink/runtime/JobException.java  |   32 +
 .../flink/runtime/JobSubmissionException.java   |   44 -
 .../flink/runtime/client/AbstractJobResult.java |   16 +-
 .../apache/flink/runtime/client/JobClient.java  |   27 +-
 .../runtime/client/JobSubmissionResult.java     |   19 -
 .../deployment/ChannelDeploymentDescriptor.java |   45 +-
 .../deployment/GateDeploymentDescriptor.java    |  122 +-
 .../deployment/TaskDeploymentDescriptor.java    |  367 +--
 .../flink/runtime/event/job/AbstractEvent.java  |   43 +-
 .../event/job/ExecutionStateChangeEvent.java    |   27 +-
 .../flink/runtime/event/job/JobEvent.java       |   17 +-
 .../runtime/event/job/ManagementEvent.java      |    8 +-
 .../event/job/VertexAssignmentEvent.java        |  151 -
 .../flink/runtime/event/job/VertexEvent.java    |   51 +-
 .../flink/runtime/event/task/AbstractEvent.java |   10 +-
 .../flink/runtime/execution/Environment.java    |   14 -
 .../runtime/execution/ExecutionAttempt.java     |  100 +
 .../runtime/execution/ExecutionListener.java    |   49 +-
 .../runtime/execution/ExecutionObserver.java    |   19 +-
 .../runtime/execution/ExecutionState2.java      |   38 +
 .../execution/ExecutionStateTransition.java     |  118 -
 .../runtime/execution/RuntimeEnvironment.java   |  379 +--
 .../librarycache/LibraryCacheManager.java       |    6 +-
 .../librarycache/LibraryCacheUpdate.java        |   26 +-
 .../executiongraph/AllVerticesIterator.java     |   71 +
 .../DistributionPatternProvider.java            |   83 -
 .../executiongraph/ExecutionAttemptID.java      |   30 +
 .../runtime/executiongraph/ExecutionEdge2.java  |   74 +
 .../runtime/executiongraph/ExecutionGraph.java  | 2688 ++++++++++--------
 .../executiongraph/ExecutionGraphIterator.java  |  444 ---
 .../ExecutionGroupVertexIterator.java           |  331 ---
 .../executiongraph/ExecutionJobVertex.java      |  220 ++
 .../executiongraph/ExecutionPipeline.java       |  129 -
 .../executiongraph/ExecutionSignature.java      |  158 -
 .../runtime/executiongraph/ExecutionStage.java  |  411 ---
 .../executiongraph/ExecutionStageListener.java  |   43 -
 .../runtime/executiongraph/ExecutionState.java  |   34 +
 .../runtime/executiongraph/ExecutionVertex.java |   34 +-
 .../executiongraph/ExecutionVertex2.java        |  638 ++++-
 .../executiongraph/ExecutionVertexID.java       |   74 -
 .../GraphConversionException.java               |   51 -
 .../executiongraph/IntermediateResult.java      |   82 +
 .../IntermediateResultPartition.java            |   68 +
 .../executiongraph/InternalJobStatus.java       |  104 -
 .../executiongraph/JobStatusListener.java       |   22 +-
 .../flink/runtime/instance/AllocatedSlot.java   |   48 +-
 .../instance/DefaultInstanceManager.java        |  314 --
 .../apache/flink/runtime/instance/Instance.java |   26 +
 .../flink/runtime/instance/InstanceManager.java |  281 +-
 .../runtime/instance/LocalInstanceManager.java  |    9 +-
 .../disk/iomanager/ChannelReaderInputView.java  |   11 +-
 .../disk/iomanager/ChannelWriterOutputView.java |   10 +-
 .../runtime/io/disk/iomanager/Deserializer.java |   61 -
 .../HeaderlessChannelReaderInputView.java       |   11 +-
 .../runtime/io/disk/iomanager/IOManager.java    |   21 +-
 .../runtime/io/network/ChannelManager.java      |   22 +-
 .../runtime/io/network/gates/InputGate.java     |   16 +-
 .../runtime/io/network/gates/OutputGate.java    |    9 +-
 .../concurrent/SolutionSetUpdateBarrier.java    |    1 -
 .../jobgraph/AbstractJobInputVertex.java        |   72 -
 .../jobgraph/AbstractJobOutputVertex.java       |   42 -
 .../runtime/jobgraph/AbstractJobVertex.java     |  623 ++--
 .../runtime/jobgraph/DistributionPattern.java   |    3 -
 .../jobgraph/InputFormatInputVertex.java        |  103 -
 .../runtime/jobgraph/InputFormatVertex.java     |   58 +
 .../runtime/jobgraph/IntermediateDataSet.java   |   83 +
 .../runtime/jobgraph/IntermediateDataSetID.java |   53 +
 .../apache/flink/runtime/jobgraph/JobEdge.java  |  137 +-
 .../apache/flink/runtime/jobgraph/JobGraph.java |  662 ++---
 .../jobgraph/JobGraphDefinitionException.java   |   33 -
 .../flink/runtime/jobgraph/JobStatus.java       |   36 +-
 .../flink/runtime/jobgraph/JobTaskVertex.java   |   55 -
 .../flink/runtime/jobgraph/JobVertexID.java     |    6 +
 .../jobgraph/OutputFormatOutputVertex.java      |   83 -
 .../runtime/jobgraph/OutputFormatVertex.java    |   66 +
 .../runtime/jobgraph/SimpleInputVertex.java     |   61 -
 .../runtime/jobgraph/SimpleOutputVertex.java    |   53 -
 .../jobgraph/tasks/AbstractInvokable.java       |   37 +-
 .../jobgraph/tasks/InputSplitIterator.java      |    5 +-
 .../jobgraph/tasks/InputSplitProvider.java      |   12 +-
 .../runtime/jobmanager/DeploymentManager.java   |   47 -
 .../runtime/jobmanager/EventCollector.java      |  253 +-
 .../flink/runtime/jobmanager/JobManager.java    |  999 ++-----
 .../runtime/jobmanager/JobManagerUtils.java     |  119 -
 .../jobmanager/scheduler/DefaultScheduler.java  |   18 +-
 .../jobmanager/scheduler/LifoSetQueue.java      |   25 +-
 .../scheduler/NoResourceAvailableException.java |   31 +-
 .../scheduler/SlotAvailablilityListener.java    |    3 +-
 .../jobmanager/scheduler/SlotSharingGroup.java  |   13 +-
 .../DefaultInputSplitAssigner.java              |   75 -
 .../splitassigner/InputSplitWrapper.java        |  144 -
 .../LocatableInputSplitAssigner.java            |  197 --
 .../managementgraph/ManagementVertex.java       |   30 +-
 .../managementgraph/ManagementVertexID.java     |    4 +
 .../runtime/operators/CollectorMapDriver.java   |    1 +
 .../flink/runtime/operators/CrossDriver.java    |    2 +-
 .../operators/sort/MergeMatchIterator.java      |    4 +-
 .../operators/sort/UnilateralSortMerger.java    |    6 -
 .../runtime/profiling/TaskManagerProfiler.java  |   12 +-
 .../profiling/impl/EnvironmentListenerImpl.java |   55 +-
 .../profiling/impl/EnvironmentThreadSet.java    |   11 +-
 .../profiling/impl/JobProfilingData.java        |   10 +-
 .../profiling/impl/TaskManagerProfilerImpl.java |   13 +-
 .../InternalExecutionVertexProfilingData.java   |   24 +-
 ...ernalExecutionVertexThreadProfilingData.java |   21 +-
 .../types/InternalInputGateProfilingData.java   |   24 +-
 .../types/InternalOutputGateProfilingData.java  |   21 +-
 .../impl/types/InternalProfilingData.java       |    5 +-
 .../protocols/ExtendedManagementProtocol.java   |   26 -
 .../protocols/InputSplitProviderProtocol.java   |   27 +-
 .../protocols/JobManagementProtocol.java        |    5 +-
 .../protocols/TaskOperationProtocol.java        |   88 +-
 .../runtime/taskmanager/AbstractTaskResult.java |  144 -
 .../apache/flink/runtime/taskmanager/Task.java  |  426 +--
 .../runtime/taskmanager/TaskCancelResult.java   |   50 -
 .../runtime/taskmanager/TaskExecutionState.java |  130 +-
 .../taskmanager/TaskInputSplitProvider.java     |   56 +-
 .../runtime/taskmanager/TaskKillResult.java     |   50 -
 .../flink/runtime/taskmanager/TaskManager.java  | 1280 ++++-----
 .../taskmanager/TaskOperationResult.java        |  110 +
 .../taskmanager/TaskSubmissionResult.java       |   50 -
 .../apache/flink/runtime/util/EnumUtils.java    |    8 +-
 .../runtime/util/ExecutorThreadFactory.java     |   32 +-
 .../runtime/util/SerializableArrayList.java     |   38 +-
 .../flink/runtime/client/JobResultTest.java     |    2 -
 .../ChannelDeploymentDescriptorTest.java        |    6 +-
 .../GateDeploymentDescriptorTest.java           |  138 -
 .../TaskDeploymentDescriptorTest.java           |  291 +-
 .../flink/runtime/event/job/JobEventTest.java   |   32 +-
 .../runtime/event/job/ManagementEventTest.java  |   28 +-
 .../executiongraph/AllVerticesIteratorTest.java |   67 +
 .../ExecutionGraphConstructionTest.java         |  570 ++++
 .../ExecutionGraphDeploymentTest.java           |  145 +
 .../executiongraph/ExecutionGraphTest.java      |  955 -------
 .../executiongraph/ExecutionGraphTestUtils.java |  179 ++
 .../ExecutionVertexCancelTest.java              |  466 +++
 .../ExecutionVertexDeploymentTest.java          |  219 ++
 .../ForwardTask1Input1Output.java               |   52 -
 .../ForwardTask1Input2Outputs.java              |   58 -
 .../ForwardTask2Inputs1Output.java              |   64 -
 .../executiongraph/PointwisePatternTest.java    |  330 +++
 .../executiongraph/SelfCrossForwardTask.java    |   40 -
 .../executiongraph/VertexSlotSharingTest.java   |  103 +
 .../runtime/instance/AllocatedSlotTest.java     |  140 +
 .../instance/DefaultInstanceManagerTest.java    |  275 --
 .../runtime/instance/InstanceManagerTest.java   |  276 ++
 .../instance/LocalInstanceManagerTest.java      |   25 +-
 .../flink/runtime/instance/SlotTestUtils.java   |   23 +
 .../flink/runtime/jobgraph/JobGraphTest.java    |  274 +-
 .../runtime/jobgraph/JobTaskVertexTest.java     |  166 ++
 .../runtime/jobmanager/DoubleSourceTask.java    |  138 -
 .../runtime/jobmanager/DoubleTargetTask.java    |   63 -
 .../flink/runtime/jobmanager/ForwardTask.java   |   51 -
 .../runtime/jobmanager/JobManagerITCase.java    | 1073 +------
 .../runtime/jobmanager/JobManagerTest.java      |   23 +
 .../jobmanager/RuntimeExceptionTask.java        |   42 -
 .../flink/runtime/jobmanager/UnionTask.java     |   63 -
 .../splitassigner/DefaultSplitAssignerTest.java |  124 -
 .../LocatableSplitAssignerTest.java             |  382 ---
 .../runtime/jobmanager/tasks/NoOpInvokable.java |   33 +
 .../sort/SortMergeMatchIteratorITCase.java      |    2 -
 .../operators/testutils/DriverTestBase.java     |    1 -
 .../operators/testutils/MockEnvironment.java    |   10 -
 .../operators/util/HashVsSortMiniBenchmark.java |    2 -
 .../flink/runtime/taskmanager/OneShotLatch.java |   41 +
 .../taskmanager/TaskExecutionStateTest.java     |   72 +
 .../runtime/taskmanager/TaskManagerTest.java    |  390 +++
 .../flink/runtime/taskmanager/TaskTest.java     |  346 +++
 .../testutils/tasks/DoubleSourceTask.java       |  138 -
 .../runtime/testutils/tasks/FileLineReader.java |  137 -
 .../runtime/testutils/tasks/FileLineWriter.java |   76 -
 .../testutils/tasks/JobFileInputVertex.java     |  259 --
 .../testutils/tasks/JobFileOutputVertex.java    |  113 -
 .../util/DelegatingConfigurationTest.java       |   93 +
 .../util/TestDelegatingConfiguration.java       |   93 -
 .../BroadcastVarsNepheleITCase.java             |   18 +-
 .../KMeansIterativeNepheleITCase.java           |   33 +-
 .../ConnectedComponentsNepheleITCase.java       |   58 +-
 .../IterationWithChainingNepheleITCase.java     |    8 +-
 .../test/iterative/nephele/JobGraphUtils.java   |   33 +-
 .../CustomCompensatableDanglingPageRank.java    |    6 +-
 ...mpensatableDanglingPageRankWithCombiner.java |   11 +-
 .../CompensatableDanglingPageRank.java          |   10 +-
 .../nephele/jobgraph/IntermediateDataSet.java   |   42 -
 .../nephele/jobgraph/IntermediateDataSetID.java |   50 -
 225 files changed, 11600 insertions(+), 14492 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/docs/img/state_machine.svg
----------------------------------------------------------------------
diff --git a/docs/img/state_machine.svg b/docs/img/state_machine.svg
new file mode 100644
index 0000000..6aadb50
--- /dev/null
+++ b/docs/img/state_machine.svg
@@ -0,0 +1,235 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!-- Created with Inkscape (http://www.inkscape.org/) -->
+
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   version="1.0"
+   width="230.41mm"
+   height="154.87mm"
+   id="svg2985">
+  <metadata
+     id="metadata3079">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+        <dc:title></dc:title>
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <defs
+     id="defs2987" />
+  <g
+     id="g2989">
+    <path
+       d="m 146.66237,340.91829 c 0,-20.55261 16.68962,-37.20472 37.27974,-37.20472 20.59011,0 37.26098,16.65211 37.26098,37.20472 0,20.53386 -16.67087,37.18597 -37.26098,37.18597 -20.59012,0 -37.27974,-16.65211 -37.27974,-37.18597"
+       id="path2991"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 146.66237,340.91829 c 0,-20.55261 16.68962,-37.20472 37.27974,-37.20472 20.59011,0 37.26098,16.65211 37.26098,37.20472 0,20.53386 -16.67087,37.18597 -37.26098,37.18597 -20.59012,0 -37.27974,-16.65211 -37.27974,-37.18597"
+       id="path2993"
+       style="fill:none;stroke:#41719c;stroke-width:1.25640953px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="148.53239"
+       y="346.52737"
+       id="text2995"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">CREATED</text>
+    <path
+       d="m 321.52833,238.17399 c 0,-20.59011 16.68962,-37.27973 37.27973,-37.27973 20.59012,0 37.26098,16.68962 37.26098,37.27973 0,20.57137 -16.67086,37.26099 -37.26098,37.26099 -20.59011,0 -37.27973,-16.68962 -37.27973,-37.26099"
+       id="path2997"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 321.52833,238.17399 c 0,-20.59011 16.68962,-37.27973 37.27973,-37.27973 20.59012,0 37.26098,16.68962 37.26098,37.27973 0,20.57137 -16.67086,37.26099 -37.26098,37.26099 -20.59011,0 -37.27973,-16.68962 -37.27973,-37.26099"
+       id="path2999"
+       style="fill:none;stroke:#41719c;stroke-width:1.25640953px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="335.59247"
+       y="234.72913"
+       id="text3001"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">SCHE</text>
+    <text
+       x="377.29776"
+       y="234.72913"
+       id="text3003"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">-</text>
+    <text
+       x="333.49219"
+       y="252.73141"
+       id="text3005"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">DULED</text>
+    <path
+       d="m 295.91258,509.1459 c 0,-20.59011 16.68961,-37.27973 37.26098,-37.27973 20.59011,0 37.27973,16.68962 37.27973,37.27973 0,20.57136 -16.68962,37.26098 -37.27973,37.26098 -20.57137,0 -37.26098,-16.68962 -37.26098,-37.26098"
+       id="path3007"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 294.07484,511.09615 -0.0375,-1.95025 0.0563,-2.02526 0.15002,-1.98775 0.24378,-1.95025 0.33755,-1.91274 0.41255,-1.78147 3.65671,0.8251 -0.3938,1.74397 0.0188,-0.0938 -0.33755,1.83773 0.0188,-0.11251 -0.22503,1.87524 0,-0.0938 -0.15002,1.89399 0,-0.0938 -0.0375,1.91275 0,-0.075 0.0375,1.91274 -3.75048,0.075 z m 2.3628,-15.45196 0.67509,-1.74397 0.7876,-1.74398 0.86261,-1.66896 0.93762,-1.63145 1.01263,-1.59396 1.10639,-1.53769 0.28128,-0.37505 2.96288,2.30654 -0.26254,0.33755 0.0375,-0.075 -1.03138,1.46268 0.0563,-0.075 -0.97513,1.51895 0.0563,-0.0563 -0.90012,1.55645 0.0375,-0.0938 -0.82511,1.61271 0.0375,-0.0938 -0.7501,1.65021 0.0375,-0.0938 -0.65633,1.70647 -3.48795,-1.36892 z m 8.21355,-13.29544 0.84385,-0.88136 1.36893,-1.31267 1.42518,-1.2189 1.48144,-1.16265 1.51894,-1.08764 1.59395,-1.01263 1.01263,-0.58132 1.85648,3.24416 -0.95637,0.56257 0.075,-0.0563 -1.50019,0.97513 0.0563,-0.0375 -1.46269,1.03138 0.075,-0.0563 -1.40643,1.10639 0.075,-0.0563 -1.36892,1.16265
  0.075,-0.0563 -1.29391,1.23766 0.0562,-0.075 -0.80635,0.86261 -2.71909,-2.58783 z m 12.73286,-9.0199 0.56257,-0.26253 1.78148,-0.69384 1.81898,-0.61883 1.85649,-0.52506 1.89399,-0.43131 1.91274,-0.33754 1.53769,-0.20628 0.48757,3.71297 -1.50019,0.20628 0.0938,-0.0188 -1.81898,0.31879 0.075,-0.0188 -1.80023,0.41255 0.0938,-0.0188 -1.76272,0.50631 0.075,-0.0188 -1.72522,0.58132 0.0938,-0.0375 -1.68772,0.65633 0.075,-0.0375 -0.50631,0.24378 -1.55645,-3.41294 z m 15.30194,-3.31917 0.48757,-0.0188 2.02525,0.0563 1.98776,0.15002 1.95024,0.26253 1.9315,0.33754 1.89399,0.43131 1.38768,0.3938 -1.03138,3.60045 -1.35018,-0.37504 0.11252,0.0188 -1.80023,-0.41255 0.075,0.0188 -1.83773,-0.31879 0.0938,0.0188 -1.85649,-0.24378 0.0938,0.0188 -1.89399,-0.15002 0.11252,0 -1.9315,-0.0563 0.0938,0 -0.45005,0.0188 -0.0938,-3.75048 z m 15.37696,2.92537 0.35629,0.13127 1.72522,0.7876 1.70647,0.88136 1.63146,0.93762 1.5752,1.01263 1.53769,1.08764 1.48144,1.16265 0.11251,0.0938 -2.43781,2.85036 -0.0938,-0.
 075 0.075,0.0563 -1.40643,-1.10639 0.075,0.0563 -1.46268,-1.03138 0.0562,0.0375 -1.50019,-0.97513 0.075,0.0563 -1.55644,-0.90011 0.075,0.0375 -1.6127,-0.8251 0.0938,0.0375 -1.65021,-0.75009 0.075,0.0375 -0.31879,-0.11251 1.38768,-3.48795 z m 13.0329,8.75737 1.06889,1.12514 1.2189,1.42518 1.16265,1.48144 1.08764,1.53769 1.01263,1.59396 0.95637,1.63145 0.37505,0.71259 -3.33793,1.72522 -0.35629,-0.69384 0.0375,0.0938 -0.90011,-1.55645 0.0562,0.0563 -0.97512,-1.51895 0.0563,0.075 -1.05014,-1.46268 0.0563,0.075 -1.10639,-1.40643 0.0563,0.075 -1.16265,-1.35017 0.0563,0.0563 -1.03138,-1.08764 2.71909,-2.58782 z m 8.49483,13.08916 0.3563,0.90011 0.61883,1.81898 0.52506,1.85649 0.43131,1.91274 0.35629,1.91274 0.24378,1.95025 0.0938,1.12515 -3.75047,0.30003 -0.075,-1.08763 0,0.0938 -0.22503,-1.87524 0.0188,0.11251 -0.33754,-1.83773 0.0188,0.0938 -0.41255,-1.81898 0.0188,0.11252 -0.50631,-1.76273 0.0375,0.075 -0.58132,-1.74397 0.0375,0.0938 -0.33754,-0.84386 3.46919,-1.38767 z m 2.7191,15.4144
 6 -0.0375,0.97512 -0.15002,1.969 -0.24378,1.95025 -0.35629,1.93149 -0.43131,1.89399 -0.52506,1.87524 -0.33755,0.99388 -3.5442,-1.20016 0.31879,-0.95637 -0.0375,0.075 0.50631,-1.76272 -0.0188,0.0938 0.41255,-1.80023 -0.0188,0.0938 0.33754,-1.83773 -0.0188,0.0938 0.22503,-1.87524 0,0.0938 0.15002,-1.87524 -0.0188,0.0938 0.0375,-0.91887 3.75048,0.0938 z m -3.5442,15.24568 -0.30004,0.67509 -0.86261,1.68771 -0.95637,1.63146 -1.01263,1.59395 -1.08764,1.5377 -1.16265,1.48143 -1.05013,1.21891 -2.83161,-2.45656 1.01263,-1.1814 -0.0563,0.0562 1.10639,-1.40642 -0.0563,0.075 1.05014,-1.46269 -0.0563,0.075 0.97512,-1.50019 -0.0562,0.0563 0.90011,-1.55645 -0.0375,0.0938 0.8251,-1.6127 -0.0375,0.075 0.28128,-0.63758 3.41294,1.55645 z m -9.18867,12.62035 -0.075,0.0563 -1.42518,1.23766 -1.48144,1.16265 -1.53769,1.08763 -1.5752,1.01263 -1.65021,0.93762 -1.68772,0.88136 -0.4313,0.18753 -1.5377,-3.41294 0.3938,-0.18752 -0.0938,0.0563 1.6127,-0.82511 -0.075,0.0375 1.55644,-0.90012 -0.075,0.0375 1.51895,
 -0.95637 -0.0938,0.0375 1.46269,-1.03138 -0.0563,0.0563 1.40643,-1.10639 -0.075,0.0562 1.36893,-1.1814 -0.075,0.0563 0.0375,-0.0187 2.58783,2.71909 z m -13.50171,8.04478 -1.25641,0.41255 -1.85649,0.54382 -1.89399,0.4313 -1.9315,0.33754 -1.95024,0.24379 -1.98776,0.16877 -0.69383,0.0187 -0.0938,-3.75048 0.63758,-0.0187 -0.0938,0 1.89399,-0.15002 -0.0938,0.0188 1.85649,-0.24378 -0.0938,0.0188 1.83773,-0.3188 -0.075,0 1.80023,-0.41255 -0.11252,0.0375 1.78148,-0.50632 -0.075,0.0188 1.20015,-0.41256 1.20016,3.56296 z m -15.50822,2.08151 -1.38768,-0.11251 -1.95025,-0.24379 -1.93149,-0.33754 -1.89399,-0.4313 -1.85649,-0.54382 -1.81898,-0.60008 -0.65633,-0.26253 1.38767,-3.48794 0.60008,0.24378 -0.0938,-0.0375 1.72522,0.58133 -0.075,-0.0188 1.76272,0.50632 -0.0938,-0.0375 1.80023,0.41255 -0.075,0 1.83773,0.3188 -0.11251,-0.0188 1.87523,0.24378 -0.0938,-0.0188 1.35017,0.11252 -0.30003,3.73172 z m -15.07692,-4.12552 -0.97512,-0.50632 -1.63146,-0.93762 -1.59395,-1.01263 -1.51894,-1.08763 -1.481
 44,-1.16265 -1.42518,-1.23766 -0.93762,-0.88136 2.58783,-2.7191 0.90011,0.84386 -0.075,-0.0563 1.36892,1.1814 -0.075,-0.0562 1.40643,1.10639 -0.075,-0.0563 1.46269,1.03138 -0.075,-0.0375 1.51894,0.95637 -0.075,-0.0375 1.55645,0.90012 -0.075,-0.0563 0.91887,0.48756 -1.70647,3.33793 z m -12.2453,-9.71374 -0.26254,-0.30004 -1.16264,-1.48143 -1.08764,-1.5377 -1.03138,-1.59395 -0.93762,-1.63146 -0.86261,-1.68771 -0.7876,-1.74397 -0.0563,-0.11252 3.5067,-1.36892 0.0188,0.075 -0.0375,-0.0938 0.7501,1.6502 -0.0375,-0.075 0.82511,1.59395 -0.0375,-0.0563 0.88137,1.55645 -0.0563,-0.0938 0.97512,1.51894 -0.0375,-0.075 1.03138,1.46269 -0.0375,-0.075 1.10639,1.40642 -0.0563,-0.0562 0.24378,0.28128 -2.85036,2.43781 z m -7.50096,-13.783 -0.46881,-1.65021 -0.45005,-1.89399 -0.33755,-1.91274 -0.26253,-1.969 -0.13127,-1.95025 3.73173,-0.26253 0.15002,1.89399 -0.0188,-0.11252 0.24379,1.87524 -0.0188,-0.0938 0.33755,1.83773 -0.0188,-0.0938 0.41256,1.80023 -0.0188,-0.0938 0.45006,1.59395 -3.60046,1.03138
  z"
+       id="path3009"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="300.41989"
+       y="505.78619"
+       id="text3011"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">CANCEL</text>
+    <text
+       x="361.32761"
+       y="505.78619"
+       id="text3013"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">-</text>
+    <text
+       x="322.77271"
+       y="523.78845"
+       id="text3015"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">ED</text>
+    <path
+       d="m 681.76157,143.15568 c 0,-20.59012 16.65212,-37.27974 37.16722,-37.27974 20.55261,0 37.20472,16.68962 37.20472,37.27974 0,20.59011 -16.65211,37.27973 -37.20472,37.27973 -20.5151,0 -37.16722,-16.68962 -37.16722,-37.27973"
+       id="path3017"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 679.92384,145.14343 -0.0375,-1.98775 0.0375,-2.02526 0.15002,-1.95025 0.26253,-1.98775 0.33754,-1.91274 0.41256,-1.76273 3.63796,0.82511 -0.37505,1.72522 0,-0.075 -0.33754,1.83773 0.0375,-0.11251 -0.26253,1.87524 0.0375,-0.11252 -0.15002,1.91274 0,-0.11251 -0.0375,1.91274 0,-0.075 0.0375,1.91275 -3.75047,0.11251 z m 2.32529,-15.41446 0,-0.0375 0.71259,-1.76273 0.7501,-1.72521 0.86261,-1.68772 0.93762,-1.65021 1.05013,-1.5752 1.08764,-1.53769 0.30004,-0.37505 2.92537,2.32529 -0.26253,0.33755 0.0375,-0.075 -1.01263,1.46268 0.0375,-0.075 -0.97513,1.50019 0.075,-0.075 -0.90011,1.5377 0.0375,-0.075 -0.8251,1.61271 0.0375,-0.075 -0.75009,1.65021 0.0375,-0.075 -0.67508,1.68771 0.0375,-0.075 0,0 -3.52545,-1.23766 z m 8.21355,-13.35169 0.86261,-0.90012 1.35017,-1.27516 1.42518,-1.23766 1.46268,-1.16264 1.5377,-1.08764 1.5752,-1.05013 1.01263,-0.56258 1.87524,3.26292 -0.97513,0.56257 0.075,-0.075 -1.50019,0.97512 0.075,-0.0375 -1.46269,1.05013 0.075,-0.075 -1.38767,1.12515 0.0375,
 -0.075 -1.35017,1.16264 0.075,-0.0375 -1.27516,1.23766 0.0375,-0.075 -0.7876,0.86261 -2.73784,-2.58783 z m 12.75162,-9.03865 0.52506,-0.26253 1.76273,-0.67509 1.83773,-0.63758 1.83773,-0.52507 1.87524,-0.41255 1.95025,-0.33754 1.53769,-0.22503 0.48757,3.71297 -1.5002,0.22503 0.075,-0.0375 -1.83773,0.33755 0.11252,-0.0375 -1.80023,0.41256 0.075,0 -1.76273,0.48756 0.075,0 -1.72522,0.56257 0.11252,-0.0375 -1.68772,0.67509 0.075,-0.0375 -0.48756,0.22503 -1.53769,-3.41293 z m 15.26443,-3.30042 0.45006,-0.0375 2.02526,0.075 1.98775,0.15001 1.95025,0.26254 1.91274,0.33754 1.91274,0.41255 1.42518,0.41256 -1.01262,3.60045 -1.42518,-0.37504 0.11251,0 -1.80023,-0.41256 0.075,0.0375 -1.80023,-0.33755 0.075,0.0375 -1.83773,-0.26253 0.075,0.0375 -1.87524,-0.15002 0.075,0 -1.91274,-0.075 0.11251,0 -0.41255,0.0375 -0.11252,-3.75048 z m 15.37696,2.92537 0.30003,0.11252 1.72522,0.7876 1.68772,0.90011 1.65021,0.93762 1.5752,1.01263 1.53769,1.08764 1.46269,1.16264 0.18752,0.15002 -2.47531,2.85037 -0.15
 002,-0.15002 0.075,0.075 -1.38768,-1.12515 0.075,0.075 -1.46268,-1.05013 0.075,0.0375 -1.5002,-0.93762 0.075,0.0375 -1.5752,-0.90011 0.075,0.0375 -1.5752,-0.8251 0.075,0.0375 -1.65021,-0.7501 0.075,0.0375 -0.26253,-0.11251 1.38768,-3.48795 z m 13.01415,8.81362 1.01263,1.05013 1.20015,1.42519 1.16265,1.50019 1.08764,1.53769 1.01262,1.5752 0.93762,1.65021 0.45006,0.82511 -3.37543,1.68771 -0.37505,-0.7876 0.0375,0.075 -0.90012,-1.5377 0.0375,0.075 -0.93762,-1.50019 0.0375,0.075 -1.05013,-1.46268 0.075,0.075 -1.12514,-1.42518 0.0375,0.075 -1.16264,-1.38768 0.075,0.075 -0.97512,-1.01263 2.73785,-2.58783 z m 8.43857,13.12667 0.33754,0.7876 0.60008,1.83773 0.52507,1.83774 0.45005,1.91274 0.33755,1.91274 0.22502,1.95025 0.11252,1.23766 -3.75048,0.30003 -0.075,-1.20015 0,0.11252 -0.22503,-1.87524 0,0.11251 -0.33754,-1.83773 0.0375,0.075 -0.41256,-1.80023 0.0375,0.075 -0.52507,-1.76272 0.0375,0.11251 -0.60008,-1.72522 0.0375,0.075 -0.30004,-0.7501 3.48794,-1.38767 z m 2.66284,15.41446 0,0.862
 61 -0.15002,1.98775 -0.26253,1.95025 -0.33755,1.91274 -0.45005,1.91274 -0.52507,1.87524 -0.37505,1.08764 -3.56295,-1.20016 0.37505,-1.05013 -0.0375,0.075 0.52507,-1.76272 -0.0375,0.11251 0.41256,-1.83773 -0.0375,0.11251 0.33754,-1.83773 0,0.075 0.22503,-1.87524 0,0.11251 0.15002,-1.87523 0,0.075 0,-0.7876 3.75048,0.075 z m -3.56295,15.22693 -0.26254,0.60008 -0.86261,1.68771 -0.93762,1.6127 -1.01262,1.61271 -1.08764,1.53769 -1.16265,1.46269 -1.12514,1.31267 -2.85037,-2.43781 1.12515,-1.27516 -0.075,0.075 1.12514,-1.42519 -0.075,0.075 1.05013,-1.46268 -0.0375,0.075 0.93762,-1.50019 -0.0375,0.075 0.90012,-1.5752 -0.0375,0.075 0.82511,-1.61271 -0.0375,0.11252 0.22503,-0.56258 3.41294,1.5377 z m -9.30119,12.67661 -1.31266,1.16265 -1.46269,1.16264 -1.53769,1.08764 -1.5752,1.01263 -1.65021,0.93762 -1.68772,0.86261 -0.56257,0.26253 -1.53769,-3.41293 0.52506,-0.26253 -0.075,0.075 1.5752,-0.82511 -0.075,0.0375 1.5752,-0.90012 -0.075,0.0375 1.5002,-0.97513 -0.075,0.075 1.46268,-1.05013 -0.075,
 0.0375 1.38768,-1.08763 -0.075,0.0375 1.31267,-1.08764 2.4378,2.81286 z m -13.4267,7.95101 -1.12514,0.37505 -1.83774,0.52506 -1.91274,0.45006 -1.91274,0.33754 -1.95025,0.22503 -1.98775,0.18753 -0.86261,0 -0.075,-3.75048 0.7876,0 -0.11252,0 1.91275,-0.15002 -0.11252,0 1.87524,-0.22503 -0.075,0 1.80023,-0.30004 -0.075,0 1.80023,-0.41255 -0.11251,0.0375 1.76272,-0.52507 -0.075,0.0375 1.08764,-0.37504 1.20015,3.56295 z m -15.52697,2.02526 -1.20015,-0.11252 -1.95025,-0.22503 -1.95025,-0.33754 -1.87524,-0.45006 -1.83773,-0.52506 -1.83773,-0.60008 -0.82511,-0.33754 1.38768,-3.48795 0.7876,0.30004 -0.11252,-0.0375 1.72522,0.60007 -0.075,-0.0375 1.76273,0.52507 -0.075,-0.0375 1.80023,0.41255 -0.11252,-0.0375 1.83773,0.33754 -0.075,0 1.83774,0.22503 -0.075,0 1.16265,0.075 -0.30004,3.75048 z m -15.03941,-4.20054 -0.7876,-0.41255 -1.65021,-0.93762 -1.5752,-1.01263 -1.5377,-1.08764 -1.46268,-1.16264 -1.42518,-1.23766 -1.08764,-1.05013 2.58783,-2.70035 1.05013,1.01263 -0.075,-0.075 1.35017,1.1626
 5 -0.0375,-0.0375 1.38767,1.08763 -0.075,-0.0375 1.46269,1.01263 -0.075,-0.0375 1.50019,0.97513 -0.075,-0.0375 1.5752,0.90012 -0.075,-0.0375 0.71259,0.37505 -1.68771,3.33792 z m -12.18905,-9.78874 -0.11251,-0.15002 -1.16265,-1.46269 -1.12514,-1.53769 -1.01263,-1.61271 -0.90012,-1.6127 -0.90011,-1.68771 -0.7501,-1.72522 -0.15002,-0.33755 3.48795,-1.38767 0.11251,0.30004 -0.0375,-0.075 0.75009,1.65021 -0.0375,-0.11252 0.8251,1.61271 -0.0375,-0.075 0.90011,1.5752 -0.075,-0.075 0.97513,1.50019 -0.0375,-0.075 1.01263,1.46268 -0.0375,-0.075 1.12514,1.42519 -0.075,-0.075 0.11251,0.11251 -2.85036,2.43781 z m -7.42594,-13.83926 -0.37505,-1.42518 -0.45006,-1.87524 -0.33754,-1.95024 -0.26253,-1.95025 -0.15002,-1.95025 3.75047,-0.26253 0.15002,1.87523 -0.0375,-0.075 0.26253,1.83773 -0.0375,-0.075 0.33754,1.83773 -0.0375,-0.11251 0.45006,1.80023 -0.0375,-0.075 0.37504,1.35017 -3.60045,1.05013 z"
+       id="path3019"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750476px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="693.01093"
+       y="148.82382"
+       id="text3021"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">FAILED</text>
+    <path
+       d="m 213.98342,303.26351 3.76923,-6.82587 3.86299,-6.75086 1.969,-3.31917 2.04401,-3.26291 2.10027,-3.22541 2.17527,-3.1129 2.25029,-3.03788 2.34405,-2.94413 2.43781,-2.8316 2.56907,-2.70035 2.68159,-2.58783 2.83161,-2.41905 2.96288,-2.28779 3.11289,-2.10027 3.26292,-1.91274 3.41293,-1.76273 3.56295,-1.59395 3.67547,-1.42518 3.80673,-1.31267 3.88175,-1.16265 3.99425,-1.06888 4.08802,-0.93762 4.16303,-0.88136 4.21929,-0.76885 4.27554,-0.71259 4.3318,-0.67509 8.73861,-1.20015 6.97589,-0.86261 0.22503,1.85649 -6.95714,0.88136 0,-0.0188 -8.71986,1.20015 0,0 -4.31304,0.65633 0.0188,0 -4.25679,0.71259 0.0188,0 -4.20053,0.7876 0.0188,-0.0188 -4.14428,0.86261 0.0188,0 -4.05052,0.95637 0.0188,-0.0188 -3.95675,1.05013 0.0188,0 -3.86299,1.16265 0.0375,-0.0188 -3.75048,1.29391 0.0375,0 -3.63797,1.40643 0.0375,-0.0188 -3.52545,1.5752 0.0563,-0.0187 -3.37543,1.72522 0.0563,-0.0188 -3.22541,1.89399 0.0375,-0.0375 -3.05664,2.06276 0.0375,-0.0188 -2.90662,2.23153 0.0375,-0.0375 -2.77536,2.400
 3 0.0375,-0.0375 -2.66284,2.55032 0.0375,-0.0375 -2.53157,2.68159 0.0188,-0.0375 -2.41906,2.81286 0.0375,-0.0375 -2.32529,2.92538 0.0188,-0.0375 -2.23153,3.01913 0.0188,-0.0375 -2.15652,3.1129 0,-0.0188 -2.08152,3.18791 0,-0.0187 -2.02526,3.24416 0.0188,0 -1.969,3.30042 0,-0.0188 -3.84424,6.73211 0.0188,0 -3.76923,6.82587 z m 91.99918,-67.02101 9.46996,4.03176 -8.19479,6.24454 c -0.41256,0.31879 -0.99388,0.24378 -1.31267,-0.16877 -0.30004,-0.41255 -0.22503,-0.99387 0.18752,-1.31266 l 6.90088,-5.26942 0.20628,1.6127 -7.98852,-3.41293 c -0.48756,-0.20628 -0.69384,-0.76885 -0.50631,-1.23766 0.20627,-0.48756 0.76884,-0.69384 1.23765,-0.48756 z"
+       id="path3023"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 217.90267,373.6412 8.32606,6.91962 8.2698,6.88213 4.06926,3.43168 4.01301,3.41294 3.95676,3.41293 3.90049,3.35668 3.80673,3.35667 3.71298,3.31917 3.61921,3.30042 3.50669,3.24417 3.39418,3.22541 3.26292,3.16915 3.11289,3.13165 2.96288,3.09414 2.81285,3.03788 2.66284,3.00038 2.51282,2.94413 2.38155,2.90662 2.26904,2.86911 2.15653,2.83161 2.04401,2.81286 1.95024,2.7566 1.87524,2.7566 1.80023,2.71909 1.74397,2.70035 1.68772,2.68159 3.26291,5.34443 2.21278,3.69422 -1.59395,0.95637 -2.23153,-3.69422 0,0.0188 -3.24417,-5.32568 0,0 -1.68771,-2.66283 0,0 -1.72522,-2.68159 0,0 -1.78148,-2.70035 0,0 -1.85648,-2.73785 0,0.0188 -1.95025,-2.7566 0.0188,0 -2.04401,-2.79411 0.0188,0.0187 -2.13778,-2.81285 0,0.0188 -2.25028,-2.86911 0,0.0188 -2.3628,-2.88787 0,0.0188 -2.49407,-2.94412 0.0188,0.0188 -2.64408,-2.98163 0,0.0188 -2.79411,-3.03789 0.0188,0.0188 -2.96288,-3.07539 0.0187,0.0187 -3.11289,-3.13164 0.0188,0.0188 -3.24416,-3.16915 0,0 -3.37543,-3.20666 0,0 -3.5067,-3.24416 0.0188,0
  -3.61921,-3.28167 0,0 -3.71298,-3.30042 0,0 -3.78798,-3.35667 0,0.0188 -3.88174,-3.37543 0,0 -3.95675,-3.39418 0,0.0187 -4.01301,-3.41293 0,0 -4.06927,-3.43169 0,0 -8.23229,-6.88212 -8.32606,-6.91963 z m 91.30534,88.09868 0.15002,10.29506 -9.03864,-4.95063 c -0.45006,-0.24378 -0.61883,-0.8251 -0.3563,-1.27516 0.24378,-0.45006 0.80635,-0.61883 1.25641,-0.37505 l 7.63222,4.18178 -1.40643,0.82511 -0.11251,-8.68236 c 0,-0.50631 0.41255,-0.93761 0.93761,-0.93761 0.50632,-0.0188 0.93762,0.3938 0.93762,0.91886 z"
+       id="path3025"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 451.38857,238.17399 c 0,-20.59011 16.68962,-37.27973 37.27973,-37.27973 20.59012,0 37.26098,16.68962 37.26098,37.27973 0,20.57137 -16.67086,37.26099 -37.26098,37.26099 -20.59011,0 -37.27973,-16.68962 -37.27973,-37.26099"
+       id="path3027"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 451.38857,238.17399 c 0,-20.59011 16.68962,-37.27973 37.27973,-37.27973 20.59012,0 37.26098,16.68962 37.26098,37.27973 0,20.57137 -16.67086,37.26099 -37.26098,37.26099 -20.59011,0 -37.27973,-16.68962 -37.27973,-37.26099"
+       id="path3029"
+       style="fill:none;stroke:#41719c;stroke-width:1.25640953px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="456.49106"
+       y="234.72913"
+       id="text3031"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">DEPLOY</text>
+    <text
+       x="516.04858"
+       y="234.72913"
+       id="text3033"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">-</text>
+    <text
+       x="475.39346"
+       y="252.73141"
+       id="text3035"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">ING</text>
+    <path
+       d="m 734.56828,356.7828 c 0,-20.59011 16.65211,-37.27973 37.20472,-37.27973 20.51511,0 37.16722,16.68962 37.16722,37.27973 0,20.59012 -16.65211,37.27974 -37.16722,37.27974 -20.55261,0 -37.20472,-16.68962 -37.20472,-37.27974"
+       id="path3037"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 732.73054,358.73305 -0.0375,-1.95025 0.0375,-1.98775 0.15002,-2.02525 0.26254,-1.95025 0.33754,-1.91275 0.41255,-1.76272 3.63796,0.82511 -0.37504,1.72521 0,-0.075 -0.30004,1.83773 0,-0.11252 -0.22503,1.87524 0,-0.11251 -0.15002,1.91274 0,-0.11251 -0.0375,1.91274 0,-0.075 0.0375,1.91274 -3.75048,0.075 z m 2.3628,-15.45196 0.67509,-1.72522 0.7876,-1.72522 0.86261,-1.68771 0.93762,-1.65021 1.01263,-1.5752 1.08763,-1.5752 0.30004,-0.37505 2.92537,2.3253 -0.26253,0.33754 0.075,-0.075 -1.05013,1.46269 0.0375,-0.075 -0.93762,1.53769 0.0375,-0.075 -0.90012,1.5377 0.0375,-0.075 -0.82511,1.6127 0.0375,-0.075 -0.71259,1.65021 0.0375,-0.11251 -0.67508,1.72522 -3.48795,-1.38768 z m 8.21355,-13.27669 0.8251,-0.90011 1.35017,-1.31267 1.42518,-1.20015 1.46269,-1.16265 1.53769,-1.08763 1.5752,-1.05014 1.05014,-0.56257 1.83773,3.26291 -0.97512,0.56258 0.11251,-0.075 -1.53769,0.97512 0.075,-0.0375 -1.46269,1.05013 0.075,-0.075 -1.38768,1.08764 0.075,-0.0375 -1.35017,1.16265 0.075,-0.0375 -
 1.31267,1.23765 0.075,-0.075 -0.8251,0.86261 -2.70034,-2.58783 z m 12.71411,-9.03864 0.52507,-0.26254 1.80023,-0.67508 1.80022,-0.63758 1.83774,-0.52507 1.91274,-0.41255 1.91274,-0.33755 1.5752,-0.22502 0.48757,3.71297 -1.5377,0.22503 0.075,-0.0375 -1.83773,0.33754 0.11251,-0.0375 -1.80023,0.41255 0.11252,0 -1.76273,0.48756 0.075,0 -1.72522,0.56258 0.075,-0.0375 -1.68771,0.67509 0.075,-0.0375 -0.45006,0.22503 -1.5752,-3.41293 z m 15.30194,-3.30042 0.45006,-0.0375 1.98775,0.075 1.98776,0.15002 1.95024,0.26253 1.95025,0.33755 1.87524,0.41255 1.46268,0.41255 -1.05013,3.63796 -1.38767,-0.41255 0.075,0 -1.80022,-0.41255 0.11251,0.0375 -1.83773,-0.33754 0.075,0.0375 -1.83774,-0.26254 0.075,0.0375 -1.87524,-0.15002 0.075,0 -1.91274,-0.075 0.11252,0 -0.41256,0.0375 -0.075,-3.75047 z m 15.37696,2.92537 0.26253,0.11251 1.76272,0.7876 1.65021,0.90012 1.65021,0.90011 1.5752,1.05014 1.5377,1.08763 1.46268,1.16265 0.22503,0.15002 -2.47531,2.85036 -0.15002,-0.15002 0.0375,0.0375 -1.38767,-1.08764 
 0.075,0.075 -1.46269,-1.05013 0.075,0.0375 -1.50019,-0.97512 0.075,0.075 -1.5377,-0.90012 0.075,0.0375 -1.6127,-0.82511 0.075,0.0375 -1.65021,-0.7501 0.075,0.0375 -0.22503,-0.11252 1.38768,-3.48794 z m 12.97664,8.81362 1.01263,1.05013 1.23766,1.42518 1.16265,1.50019 1.05013,1.5377 1.05013,1.5752 0.93762,1.65021 0.41256,0.8251 -3.33793,1.68772 -0.41255,-0.7876 0.0375,0.075 -0.86261,-1.5377 0.0375,0.075 -0.97512,-1.53769 0.0375,0.075 -1.01263,-1.46269 0.0375,0.075 -1.08764,-1.38768 0.0375,0.075 -1.16264,-1.35017 0.0375,0.0375 -0.93762,-1.01263 2.70034,-2.58783 z m 8.47608,13.12667 0.30004,0.7876 0.60007,1.83773 0.56258,1.83773 0.41255,1.91274 0.33754,1.91275 0.26253,1.95025 0.11252,1.23765 -3.75048,0.30004 -0.11251,-1.20015 0.0375,0.11251 -0.26253,-1.87524 0.0375,0.11252 -0.33754,-1.83773 0,0.075 -0.41255,-1.80022 0.0375,0.11251 -0.48756,-1.76272 0,0.075 -0.56257,-1.76273 0.0375,0.11252 -0.30004,-0.7501 3.48794,-1.38767 z m 2.66284,15.41445 -0.0375,0.86261 -0.15002,1.98775 -0.26253,1.
 95025 -0.33754,1.91274 -0.41255,1.91275 -0.52507,1.87524 -0.37505,1.08763 -3.56295,-1.20015 0.37505,-1.05013 -0.0375,0.075 0.48756,-1.76273 -0.0375,0.075 0.41255,-1.80023 0,0.11252 0.33754,-1.83773 -0.0375,0.075 0.26253,-1.87524 0,0.11251 0.11252,-1.87524 0,0.075 0.0375,-0.7876 3.75048,0.075 z m -3.56295,15.22694 -0.26254,0.60007 -0.86261,1.68772 -0.93762,1.6127 -1.05013,1.61271 -1.05013,1.53769 -1.16265,1.46269 -1.16265,1.31266 -2.81285,-2.43781 1.08763,-1.27516 -0.0375,0.0375 1.08764,-1.38768 -0.0375,0.075 1.01263,-1.46269 -0.0375,0.075 0.97512,-1.53769 -0.0375,0.11251 0.86261,-1.5752 -0.0375,0.075 0.8251,-1.6127 -0.0375,0.075 0.22503,-0.52507 3.45044,1.5377 z m -9.30118,12.67661 -1.35018,1.16264 -1.46268,1.12515 -1.5377,1.12514 -1.5752,1.01263 -1.65021,0.93762 -1.68771,0.86261 -0.56257,0.26253 -1.5377,-3.41293 0.52507,-0.26254 -0.075,0.075 1.6127,-0.8251 -0.11251,0.0375 1.5752,-0.90011 -0.075,0.0375 1.50019,-0.97512 -0.075,0.075 1.46269,-1.05013 -0.075,0.0375 1.42518,-1.08764 -0.
 075,0.0375 1.27516,-1.12515 2.47532,2.85037 z m -13.46421,7.95101 -1.08764,0.37504 -1.87524,0.52507 -1.91274,0.45006 -1.91275,0.33754 -1.95024,0.22503 -1.98776,0.18752 -0.86261,0 -0.075,-3.75047 0.7876,0 -0.075,0 1.87524,-0.15002 -0.075,0 1.83774,-0.22503 -0.075,0 1.83773,-0.33754 -0.11251,0.0375 1.80022,-0.41255 -0.075,0.0375 1.76272,-0.52506 -0.075,0.0375 1.05013,-0.37505 1.20016,3.56296 z m -15.52698,2.02525 -1.20015,-0.11251 -1.95025,-0.22503 -1.91274,-0.33754 -1.91274,-0.45006 -1.83774,-0.52507 -1.80022,-0.60007 -0.86261,-0.33755 1.38767,-3.48794 0.7876,0.30004 -0.075,-0.0375 1.72522,0.60008 -0.11251,-0.0375 1.80023,0.52506 -0.11252,-0.0375 1.80023,0.41255 -0.11251,-0.0375 1.83773,0.33754 -0.075,0 1.83773,0.22503 -0.075,0 1.16265,0.075 -0.30004,3.75047 z m -15.03941,-4.20053 -0.7876,-0.41255 -1.65021,-0.93762 -1.5752,-1.01263 -1.53769,-1.12514 -1.46269,-1.16265 -1.42518,-1.20015 -1.08764,-1.05014 2.58783,-2.70034 1.05014,0.97512 -0.0375,-0.0375 1.35017,1.16265 -0.075,-0.0375 1.
 38768,1.08764 -0.075,-0.0375 1.46269,1.01262 -0.075,-0.0375 1.50019,0.97512 -0.075,-0.0375 1.5752,0.90011 -0.075,-0.0375 0.75009,0.37505 -1.72522,3.33792 z m -12.18904,-9.78874 -0.11252,-0.15002 -1.16265,-1.46269 -1.08763,-1.53769 -1.01263,-1.61271 -0.93762,-1.6127 -0.86261,-1.68772 -0.7876,-1.76272 -0.15002,-0.30004 3.48794,-1.38768 0.15002,0.30004 -0.0375,-0.11251 0.71259,1.65021 -0.0375,-0.075 0.82511,1.6127 -0.0375,-0.075 0.90012,1.5752 -0.0375,-0.075 0.93762,1.50019 -0.0375,-0.075 1.05013,1.46269 -0.075,-0.075 1.12514,1.38768 -0.075,-0.0375 0.11252,0.11252 -2.85036,2.43781 z m -7.42595,-13.83926 -0.37504,-1.42518 -0.45006,-1.87524 -0.33754,-1.95025 -0.26254,-1.95025 -0.15002,-1.95024 3.75048,-0.26254 0.15002,1.87524 0,-0.11251 0.22503,1.87524 -0.0375,-0.075 0.33755,1.83773 0,-0.11252 0.41255,1.80023 -0.0375,-0.075 0.41256,1.38768 -3.63797,1.01263 z"
+       id="path3039"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750476px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="736.83307"
+       y="362.34521"
+       id="text3041"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">FINISHED</text>
+    <path
+       d="m 586.42447,238.19275 c 0,-20.59012 16.68961,-37.27974 37.27973,-37.27974 20.59011,0 37.24223,16.68962 37.24223,37.27974 0,20.55261 -16.65212,37.24223 -37.24223,37.24223 -20.59012,0 -37.27973,-16.68962 -37.27973,-37.24223"
+       id="path3043"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 586.42447,238.19275 c 0,-20.59012 16.68961,-37.27974 37.27973,-37.27974 20.59011,0 37.24223,16.68962 37.24223,37.27974 0,20.55261 -16.65212,37.24223 -37.24223,37.24223 -20.59012,0 -37.27973,-16.68962 -37.27973,-37.24223"
+       id="path3045"
+       style="fill:none;stroke:#41719c;stroke-width:1.23765719px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="588.8078"
+       y="243.73027"
+       id="text3047"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">RUNNING</text>
+    <path
+       d="m 400.92591,237.14261 38.40488,0 0,1.87524 -38.40488,0 z m 31.37273,-4.23804 8.88863,5.17566 -8.88863,5.19441 c -0.45005,0.26253 -1.03138,0.11252 -1.29391,-0.33754 -0.26253,-0.45006 -0.11251,-1.01263 0.33754,-1.27516 l 7.50095,-4.38806 0,1.63146 -7.50095,-4.38806 c -0.4313,-0.26253 -0.60007,-0.82511 -0.33754,-1.27516 0.26253,-0.45006 0.84386,-0.60008 1.29391,-0.33755 z"
+       id="path3049"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 535.47425,237.14261 38.40487,0 0,1.87524 -38.40487,0 z m 31.37273,-4.23804 8.88863,5.17566 -8.88863,5.19441 c -0.45006,0.26253 -1.03138,0.11252 -1.29392,-0.33754 -0.26253,-0.45006 -0.0938,-1.01263 0.33755,-1.27516 l 7.50095,-4.38806 0,1.63146 -7.50095,-4.38806 c -0.43131,-0.26253 -0.60008,-0.82511 -0.33755,-1.27516 0.26254,-0.45006 0.84386,-0.60008 1.29392,-0.33755 z"
+       id="path3051"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 672.49789,252.8196 6.60084,5.21317 6.52583,5.17565 6.41332,5.10065 3.1504,2.55032 3.07539,2.51282 3.03788,2.47532 2.96288,2.47531 2.88787,2.43781 2.81285,2.40031 2.70035,2.3628 2.58782,2.28779 2.51282,2.28779 2.40031,2.21278 2.25028,2.17527 2.17528,2.13778 2.02526,2.10026 1.95024,2.02526 1.87524,2.02526 1.76273,1.95024 1.68771,1.95025 1.61271,1.91275 1.5752,1.87523 1.50019,1.87524 2.85036,3.63796 2.77535,3.63797 1.5752,2.06276 -1.50019,1.12514 -1.5752,-2.06276 0,0 -2.73785,-3.60046 0,0 -2.85036,-3.63796 0,0 -1.50019,-1.83773 0,0 -1.5377,-1.87524 0,0 -1.6127,-1.87524 0,0 -1.68771,-1.95025 0.0375,0.0375 -1.76272,-1.98776 0,0.0375 -1.83774,-2.02526 0,0.0375 -1.95024,-2.06277 0.0375,0.0375 -2.06276,-2.10027 0.0375,0.0375 -2.17527,-2.13777 0.0375,0 -2.28779,-2.13778 0.0375,0 -2.40031,-2.21278 0,0 -2.47531,-2.28779 0,0.0375 -2.58783,-2.3253 0,0 -2.70034,-2.32529 0,0 -2.81286,-2.40031 0.0375,0 -2.88786,-2.43781 0,0 -2.96288,-2.43781 0,0 -3.00038,-2.47531 0,0 -3.1129,-2.51282 -3
 .1504,-2.55032 0.0375,0 -6.41332,-5.10065 0,0 -6.52582,-5.17566 -6.60084,-5.17566 z m 72.45921,60.68271 1.20015,10.2388 -9.48871,-3.97551 c -0.48756,-0.22502 -0.71259,-0.7876 -0.52506,-1.23765 0.22503,-0.48756 0.75009,-0.71259 1.23765,-0.52507 l 7.98852,3.37543 -1.27516,0.97512 -1.01263,-8.62609 c -0.0375,-0.48756 0.30004,-0.97513 0.8251,-1.01263 0.52507,-0.075 0.97513,0.30004 1.05014,0.7876 z"
+       id="path3053"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750476px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 350.93206,285.18621 -1.35017,12.71412 -1.33142,12.67661 -1.29391,12.56409 -1.27516,12.43283 -0.61883,6.15078 -0.58133,6.09453 -0.60007,6.03826 -0.56257,5.98201 -0.54382,5.88825 -0.52507,5.85074 -0.48756,5.73823 -0.48756,5.66322 -0.45006,5.56946 -0.43131,5.47569 -0.3938,5.40069 -0.3938,5.32568 -0.35629,5.25066 -0.33754,5.17566 -0.33755,5.13815 -0.31879,5.06315 -0.58132,10.03252 -0.56257,9.86375 -0.52507,9.78874 -0.41255,7.876 1.87524,0.0938 0.41255,-7.876 0.52507,-9.76999 0.54381,-9.88251 0.60008,-10.01377 0,0.0188 0.31879,-5.08189 0.31879,-5.1194 0.3563,-5.19441 0.35629,-5.23192 0,0 0.37505,-5.32567 0.41255,-5.38194 0,0 0.43131,-5.47569 0.45005,-5.56946 0,0 0.46881,-5.64447 0.50632,-5.75698 0,0 0.52506,-5.81324 0.54382,-5.907 0.56257,-5.96325 0,0 0.58133,-6.03827 0.60007,-6.09452 0,0 0.61883,-6.15078 1.25641,-12.43283 1.31267,-12.5641 1.33142,-12.65786 1.33142,-12.71411 z m -18.63987,170.51541 4.70685,9.15116 5.66322,-8.60735 c 0.28129,-0.4313 0.15002,-1.01262 -0.28128,-
 1.29391 -0.43131,-0.28129 -1.01263,-0.16877 -1.29392,0.26253 l -4.7631,7.25718 1.6127,0.0938 -3.9755,-7.72598 c -0.22503,-0.46881 -0.7876,-0.63758 -1.25641,-0.41256 -0.46881,0.24378 -0.63758,0.80636 -0.41256,1.27517 z"
+       id="path3055"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 452.32619,422.88495 c 0,-20.59012 16.68962,-37.27974 37.27973,-37.27974 20.59012,0 37.26098,16.68962 37.26098,37.27974 0,20.57136 -16.67086,37.26098 -37.26098,37.26098 -20.59011,0 -37.27973,-16.68962 -37.27973,-37.26098"
+       id="path3057"
+       style="fill:#5b9bd5;fill-opacity:1;fill-rule:evenodd;stroke:none" />
+    <path
+       d="m 452.32619,422.88495 c 0,-20.59012 16.68962,-37.27974 37.27973,-37.27974 20.59012,0 37.26098,16.68962 37.26098,37.27974 0,20.57136 -16.67086,37.26098 -37.26098,37.26098 -20.59011,0 -37.27973,-16.68962 -37.27973,-37.26098"
+       id="path3059"
+       style="fill:none;stroke:#41719c;stroke-width:1.25640953px;stroke-linecap:butt;stroke-linejoin:miter;stroke-miterlimit:4;stroke-opacity:1;stroke-dasharray:none" />
+    <text
+       x="456.80737"
+       y="419.42795"
+       id="text3061"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">CANCEL</text>
+    <text
+       x="517.71515"
+       y="419.42795"
+       id="text3063"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">-</text>
+    <text
+       x="476.30984"
+       y="437.43024"
+       id="text3065"
+       xml:space="preserve"
+       style="font-size:15.00190544px;font-style:normal;font-weight:normal;text-align:start;text-anchor:start;fill:#000000;font-family:Arial">ING</text>
+    <path
+       d="m 440.64345,443.55007 -69.85262,41.51777 0.95638,1.61271 69.85262,-41.51777 z m -65.98963,34.2606 -4.98813,9.00115 10.29506,-0.075 c 0.50631,0 0.93762,-0.43131 0.91886,-0.95637 0,-0.50632 -0.41255,-0.91887 -0.93761,-0.91887 l -8.68236,0.075 0.82511,1.38767 4.21928,-7.59471 c 0.24378,-0.45006 0.075,-1.03138 -0.37504,-1.27516 -0.45006,-0.24378 -1.01263,-0.0938 -1.27517,0.35629 z"
+       id="path3067"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 491.57492,285.20497 -1.01263,12.6016 -0.50631,6.22579 -0.46881,6.18828 -0.46881,6.05702 -0.43131,5.96326 -0.3938,5.83199 -0.37504,5.66322 -0.33755,5.47569 -0.30003,5.32568 -0.26254,5.17566 -0.24378,5.06314 -0.22503,4.96938 -0.20627,4.89437 -0.31879,7.8385 1.85648,0.075 0.31879,-7.81975 0.20628,-4.91312 0,0 0.22503,-4.95063 0.24378,-5.06314 0,0 0.28128,-5.17566 0,0 0.30004,-5.30692 0,0 0.31879,-5.4757 0,0 0.37505,-5.64446 0.41255,-5.81324 0.43131,-5.96326 0,0 0.45005,-6.07577 0.48757,-6.16953 0.48756,-6.2258 1.01263,-12.6016 z m -9.52621,80.07266 4.81936,9.09491 5.55071,-8.68235 c 0.28128,-0.43131 0.15002,-1.01263 -0.28129,-1.29392 -0.4313,-0.28128 -1.01263,-0.15002 -1.29391,0.28129 l -4.6881,7.33218 1.63146,0.0563 -4.06927,-7.66973 c -0.24378,-0.45006 -0.80635,-0.63758 -1.27516,-0.3938 -0.45006,0.24378 -0.61883,0.80635 -0.3938,1.27516 z"
+       id="path3069"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.01875238px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 609.93995,278.0228 -4.05051,8.73861 -4.05052,8.70111 -4.12552,8.55108 -2.10027,4.23804 -2.13777,4.16303 -2.13777,4.12552 -2.21278,4.05052 -2.21278,4.01301 -2.28779,3.90049 -2.3253,3.78798 -2.3628,3.75048 -2.4003,3.60046 -2.51282,3.52545 -2.51282,3.41293 -2.62534,3.30042 -2.62533,3.1879 -2.70034,3.07539 -2.73785,3.00038 -2.77535,2.92538 -2.81286,2.85036 -2.85036,2.77535 -2.88787,2.70034 -2.92537,2.66284 -5.85074,5.21316 -5.92575,5.06315 -4.53808,3.82548 -1.20015,-1.42518 4.50057,-3.82548 0,0 5.92575,-5.06315 0,0 5.85074,-5.17565 0,0 2.88787,-2.62534 0,0 2.85036,-2.70034 0,0 2.85036,-2.77535 -0.0375,0 2.81286,-2.81286 0,0 2.73784,-2.88787 0,0 2.73785,-3.00038 -0.0375,0.0375 2.70034,-3.07539 -0.0375,0 2.66283,-3.1504 -0.0375,0 2.58783,-3.26292 0,0.0375 2.51282,-3.37543 0,0 2.47531,-3.48794 -0.0375,0 2.4003,-3.60046 0,0.0375 2.3628,-3.71297 0,0 2.28779,-3.78798 0,0 2.25029,-3.86299 0,0 2.21278,-3.97551 0,0 2.17527,-4.013 0,0 2.17528,-4.12553 -0.0375,0.0375 2.13777,-4.16303 0
 ,0 2.10026,-4.23804 0,0.0375 4.12553,-8.58859 0,0.0375 4.05051,-8.70111 0,0 4.05052,-8.73861 z m -70.58396,107.8637 -10.12629,1.76272 3.45044,-9.67622 c 0.15002,-0.48757 0.71259,-0.7501 1.20015,-0.60008 0.48756,0.18752 0.7501,0.71259 0.56257,1.20015 l -2.92537,8.17604 -1.05013,-1.23766 8.58859,-1.46268 c 0.48756,-0.11252 0.97512,0.22503 1.08764,0.75009 0.075,0.52507 -0.26254,0.97513 -0.7876,1.08764 z"
+       id="path3071"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750476px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 637.39344,198.32518 2.70034,-3.56295 2.66284,-3.52545 2.70034,-3.41293 2.70034,-3.33792 2.70035,-3.18791 1.35017,-1.50019 1.38767,-1.46268 1.35018,-1.42519 1.38767,-1.35017 1.38768,-1.27516 1.38767,-1.23766 1.35018,-1.12514 1.42518,-1.05013 1.38767,-0.97513 1.38768,-0.93762 1.38768,-0.8626 1.42518,-0.82511 2.77535,-1.46269 2.81286,-1.31266 2.77535,-1.20015 2.81286,-1.12515 1.05013,-0.37504 0.63758,1.72521 -1.01263,0.41256 0,0 -2.77535,1.08763 0.0375,0 -2.77535,1.20016 0.0375,-0.0375 -2.77536,1.31267 0.0375,-0.0375 -2.73785,1.46269 0,0 -1.35017,0.7876 0,-0.0375 -1.35017,0.8626 0,0 -1.35017,0.90012 0,0 -1.35017,0.93762 0.0375,0 -1.38768,1.05013 0.0375,-0.0375 -1.35017,1.12514 0,-0.0375 -1.35017,1.20016 0,0 -1.35018,1.23765 0.0375,0 -1.38768,1.31267 0.0375,0 -1.35017,1.38768 0,0 -1.35018,1.42518 0.0375,0 -1.35017,1.50019 0,0 -2.70035,3.1504 0.0375,-0.0375 -2.70034,3.33793 0,0 -2.66284,3.41293 0,0 -2.70034,3.48794 0,0 -2.66284,3.56296 z m 34.20434,-37.99232 10.16379,1.65021 
 -6.45082,8.02602 c -0.33754,0.41255 -0.93762,0.48756 -1.31266,0.15002 -0.41256,-0.33755 -0.48757,-0.90012 -0.15002,-1.31267 l 5.43819,-6.78836 0.56257,1.53769 -8.55109,-1.42518 c -0.52506,-0.075 -0.86261,-0.56257 -0.7876,-1.08764 0.11252,-0.48756 0.56257,-0.8251 1.08764,-0.75009 z"
+       id="path3073"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750476px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+    <path
+       d="m 499.67595,198.06265 12.6016,-7.53846 6.3008,-3.75047 6.2633,-3.71297 6.26329,-3.63797 6.18829,-3.56295 6.18828,-3.48794 6.15078,-3.37543 6.07577,-3.30042 6.03827,-3.11289 6.00076,-3.00039 5.88825,-2.81285 5.85074,-2.66284 5.77574,-2.47532 5.66322,-2.25028 2.81285,-1.08764 2.77536,-0.97512 2.77535,-0.93762 2.73785,-0.86261 5.40068,-1.61271 5.28817,-1.38767 5.25067,-1.20016 5.13815,-1.01262 5.10065,-0.86261 5.02564,-0.71259 4.95063,-0.56258 4.91312,-0.45005 4.87562,-0.37505 4.83811,-0.26253 4.80061,-0.22503 4.76311,-0.11252 4.7631,-0.11251 7.57597,-0.075 0.0375,1.87524 -7.57596,0.075 0,0 -4.76311,0.11251 0,0 -4.7631,0.11252 0.0375,0 -4.80061,0.22502 0.0375,0 -4.83812,0.26254 0.0375,0 -4.87562,0.33754 0.0375,0 -4.91312,0.48756 0.0375,0 -4.95063,0.56257 0,0 -4.98814,0.71259 0.0375,0 -5.06314,0.82511 0.0375,0 -5.13815,1.01263 0.0375,0 -5.21316,1.20015 0.0375,0 -5.28818,1.38768 0,-0.0375 -5.36318,1.61271 0.0375,0 -2.73785,0.86261 0,0 -2.73785,0.90011 0.0375,0 -2.77536,1.01263 
 0,-0.0375 -2.77535,1.05014 0,0 -5.66322,2.25028 0.0375,0 -5.73823,2.43781 0,0 -5.81324,2.66284 0.0375,0 -5.88825,2.81286 0,0 -5.96326,2.96287 0,0 -6.03827,3.1504 0.0375,-0.0375 -6.07577,3.26291 0,0 -6.11328,3.37543 0,0 -6.18828,3.48795 0,0 -6.18829,3.56295 0,0 -6.2633,3.63796 0,0 -6.26329,3.71297 0.0375,0 -6.3008,3.75048 -12.6016,7.53846 z m 167.98383,-65.67084 8.92613,5.06314 -8.81362,5.28818 c -0.45005,0.26253 -1.01262,0.15001 -1.27516,-0.30004 -0.26253,-0.45006 -0.15002,-1.01263 0.30004,-1.31267 l 0,0 7.46345,-4.46307 0,1.65021 -7.53846,-4.31304 c -0.45006,-0.26254 -0.60007,-0.82511 -0.33754,-1.27517 0.22503,-0.45005 0.8251,-0.60007 1.27516,-0.33754 z"
+       id="path3075"
+       style="fill:#000000;fill-opacity:1;fill-rule:nonzero;stroke:#000000;stroke-width:0.03750476px;stroke-linecap:butt;stroke-linejoin:round;stroke-opacity:1;stroke-dasharray:none" />
+  </g>
+</svg>

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopInputFormat.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopInputFormat.java
index c224d64..f1edc26 100644
--- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopInputFormat.java
+++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopInputFormat.java
@@ -26,6 +26,8 @@ import java.util.ArrayList;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
@@ -37,6 +39,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.hadoopcompatibility.mapred.utils.HadoopUtils;
 import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopDummyReporter;
 import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopInputSplit;
@@ -145,8 +148,8 @@ public class HadoopInputFormat<K extends Writable, V extends Writable> implement
 	}
 	
 	@Override
-	public Class<? extends HadoopInputSplit> getInputSplitType() {
-		return HadoopInputSplit.class;
+	public InputSplitAssigner getInputSplitAssigner(HadoopInputSplit[] inputSplits) {
+		return new DefaultInputSplitAssigner(inputSplits);
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java
index dcf1952..f3065e7 100644
--- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java
+++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordInputFormat.java
@@ -23,9 +23,11 @@ import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.io.ObjectOutputStream;
 
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.hadoopcompatibility.mapred.record.datatypes.HadoopTypeConverter;
 import org.apache.flink.hadoopcompatibility.mapred.utils.HadoopUtils;
 import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopDummyReporter;
@@ -84,8 +86,8 @@ public class HadoopRecordInputFormat<K, V> implements InputFormat<Record, Hadoop
 	}
 
 	@Override
-	public Class<? extends HadoopInputSplit> getInputSplitType() {
-		return HadoopInputSplit.class;
+	public InputSplitAssigner getInputSplitAssigner(HadoopInputSplit[] inputSplits) {
+		return new DefaultInputSplitAssigner(inputSplits);
 	}
 
 	@Override
@@ -129,7 +131,7 @@ public class HadoopRecordInputFormat<K, V> implements InputFormat<Record, Hadoop
 	
 	/**
 	 * Custom serialization methods.
-	 *  @see http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html
+	 *  @see "http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html"
 	 */
 	private void writeObject(ObjectOutputStream out) throws IOException {
 		out.writeUTF(hadoopInputFormatName);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java
index 337b543..68fd93d 100644
--- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java
+++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/record/HadoopRecordOutputFormat.java
@@ -110,7 +110,7 @@ public class HadoopRecordOutputFormat<K,V> implements OutputFormat<Record> {
 
 	/**
 	 * Custom serialization methods.
-	 *  @see http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html
+	 *  @see "http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html"
 	 */
 	private void writeObject(ObjectOutputStream out) throws IOException {
 		out.writeUTF(hadoopOutputFormatName);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
index da46690..d132902 100644
--- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
+++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopInputSplit.java
@@ -30,8 +30,14 @@ import org.apache.hadoop.mapred.JobConf;
 
 public class HadoopInputSplit implements InputSplit {
 
+	private static final long serialVersionUID = 1L;
+
+	
 	private transient org.apache.hadoop.mapred.InputSplit hadoopInputSplit;
+	
+	@SuppressWarnings("unused")
 	private JobConf jobConf;
+	
 	private int splitNumber;
 	private String hadoopInputSplitTypeName;
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/HadoopInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/HadoopInputFormat.java b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/HadoopInputFormat.java
index de443c1..d9da2b4 100644
--- a/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/HadoopInputFormat.java
+++ b/flink-addons/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapreduce/HadoopInputFormat.java
@@ -27,6 +27,8 @@ import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.io.FileInputFormat.FileBaseStatistics;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
@@ -38,6 +40,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.hadoopcompatibility.mapreduce.utils.HadoopUtils;
 import org.apache.flink.hadoopcompatibility.mapreduce.wrapper.HadoopInputSplit;
 import org.apache.flink.types.TypeInformation;
@@ -166,8 +169,8 @@ public class HadoopInputFormat<K extends Writable, V extends Writable> implement
 	}
 	
 	@Override
-	public Class<? extends HadoopInputSplit> getInputSplitType() {
-		return HadoopInputSplit.class;
+	public InputSplitAssigner getInputSplitAssigner(HadoopInputSplit[] inputSplits) {
+		return new DefaultInputSplitAssigner(inputSplits);
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java b/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
index fdf3c6c..63362f8 100644
--- a/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
+++ b/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputFormat.java
@@ -29,8 +29,10 @@ import org.apache.flink.addons.hbase.common.HBaseKey;
 import org.apache.flink.addons.hbase.common.HBaseResult;
 import org.apache.flink.addons.hbase.common.HBaseUtil;
 import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.types.Record;
 import org.apache.flink.util.OperatingSystem;
 import org.apache.hadoop.fs.Path;
@@ -73,7 +75,7 @@ public class TableInputFormat implements InputFormat<Record, TableInputSplit> {
 
 	/**
 	 * Base-64 encoded scanner. All other SCAN_ confs are ignored if this is specified.
-	 * See {@link TableMapReduceUtil#convertScanToString(Scan)} for more details.
+	 * See TableMapReduceUtil.convertScanToString(Scan) for more details.
 	 */
 	public static final String SCAN = "hbase.scan";
 
@@ -120,7 +122,7 @@ public class TableInputFormat implements InputFormat<Record, TableInputSplit> {
 	 * Read the configuration and creates a {@link Scan} object.
 	 * 
 	 * @param parameters
-	 * @return
+	 * @return The scanner
 	 */
 	protected Scan createScanner(Configuration parameters) {
 		Scan scan = null;
@@ -380,15 +382,12 @@ public class TableInputFormat implements InputFormat<Record, TableInputSplit> {
 		return true;
 	}
 
-
 	@Override
-	public Class<TableInputSplit> getInputSplitType() {
-
-		return TableInputSplit.class;
+	public InputSplitAssigner getInputSplitAssigner(TableInputSplit[] inputSplits) {
+		return new LocatableInputSplitAssigner(inputSplits);
 	}
-
-	public void setTable(HTable table)
-	{
+	
+	public void setTable(HTable table) {
 		this.table = table;
 	}
 
@@ -396,8 +395,7 @@ public class TableInputFormat implements InputFormat<Record, TableInputSplit> {
 		return table;
 	}
 
-	public void setScan(Scan scan)
-	{
+	public void setScan(Scan scan) {
 		this.scan = scan;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java b/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
index a77402d..c984667 100644
--- a/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
+++ b/flink-addons/flink-hbase/src/main/java/org/apache/flink/addons/hbase/TableInputSplit.java
@@ -31,6 +31,8 @@ import org.apache.flink.core.memory.DataOutputView;
  */
 public class TableInputSplit extends LocatableInputSplit {
 
+	private static final long serialVersionUID = 1L;
+
 	/**
 	 * The name of the table to retrieve data from
 	 */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-addons/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java b/flink-addons/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
index b17f658..fc384f8 100644
--- a/flink-addons/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
+++ b/flink-addons/flink-jdbc/src/main/java/org/apache/flink/api/java/io/jdbc/JDBCInputFormat.java
@@ -28,12 +28,15 @@ import java.sql.Statement;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.GenericInputSplit;
 import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.types.NullValue;
 
 /**
@@ -47,7 +50,6 @@ import org.apache.flink.types.NullValue;
 public class JDBCInputFormat<OUT extends Tuple> implements InputFormat<OUT, InputSplit> {
 	private static final long serialVersionUID = 1L;
 
-	@SuppressWarnings("unused")
 	private static final Logger LOG = LoggerFactory.getLogger(JDBCInputFormat.class);
 
 	private String username;
@@ -184,8 +186,6 @@ public class JDBCInputFormat<OUT extends Tuple> implements InputFormat<OUT, Inpu
 	/**
 	 * Enters data value from the current resultSet into a Record.
 	 *
-	 * @param pos Tuple position to be set.
-	 * @param type SQL type of the resultSet value.
 	 * @param reuse Target Record.
 	 */
 	private void addValue(OUT reuse) throws SQLException {
@@ -289,9 +289,10 @@ public class JDBCInputFormat<OUT extends Tuple> implements InputFormat<OUT, Inpu
 	}
 
 	@Override
-	public Class<? extends InputSplit> getInputSplitType() {
-		return GenericInputSplit.class;
+	public InputSplitAssigner getInputSplitAssigner(InputSplit[] inputSplits) {
+		return new DefaultInputSplitAssigner(inputSplits);
 	}
+	
 
 	/**
 	 * A builder used to set parameters to the output format's configuration in a fluent way.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
index d15b0d0..d95b808 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/CliFrontend.java
@@ -541,7 +541,7 @@ public class CliFrontend {
 				if (running && rje.getJobStatus().equals(JobStatus.RUNNING)) {
 					runningJobs.add(rje);
 				}
-				if (scheduled && rje.getJobStatus().equals(JobStatus.SCHEDULED)) {
+				if (scheduled && rje.getJobStatus().equals(JobStatus.CREATED)) {
 					scheduledJobs.add(rje);
 				}
 			}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
index f335745..593916f 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
@@ -35,7 +35,6 @@ import org.apache.flink.runtime.event.job.RecentJobEvent;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.managementgraph.ManagementGraph;
-import org.apache.flink.runtime.managementgraph.ManagementVertexID;
 import org.apache.flink.runtime.protocols.ExtendedManagementProtocol;
 import org.apache.flink.runtime.types.IntegerRecord;
 import org.junit.Assert;
@@ -190,16 +189,6 @@ public class CliFrontendListCancelTest {
 		}
 
 		@Override
-		public void killTask(JobID jobID, ManagementVertexID id) throws IOException {
-			throw new UnsupportedOperationException();
-		}
-
-		@Override
-		public void logBufferUtilization(JobID jobID) throws IOException {
-			throw new UnsupportedOperationException();
-		}
-
-		@Override
 		public int getAvailableSlots() {
 			return 1;
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index cb912de..ad6bb70 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -66,12 +66,8 @@ import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.AbstractJobOutputVertex;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetFirstDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetSecondDriver;
@@ -809,8 +805,8 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		return vertex;
 	}
 
-	private AbstractJobVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException {
-		final InputFormatInputVertex vertex = new InputFormatInputVertex(node.getNodeName(), this.jobGraph);
+	private InputFormatVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException {
+		final InputFormatVertex vertex = new InputFormatVertex(node.getNodeName(), this.jobGraph);
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
 		vertex.setInvokableClass(DataSourceTask.class);
@@ -824,7 +820,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	}
 
 	private AbstractJobOutputVertex createDataSinkVertex(SinkPlanNode node) throws CompilerException {
-		final OutputFormatOutputVertex vertex = new OutputFormatOutputVertex(node.getNodeName(), this.jobGraph);
+		final OutputFormatVertex vertex = new OutputFormatVertex(node.getNodeName(), this.jobGraph);
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
 		vertex.setInvokableClass(DataSinkTask.class);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java b/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
new file mode 100644
index 0000000..10746af
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/DefaultInputSplitAssigner.java
@@ -0,0 +1,75 @@
+/**
+ * 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.api.common.io;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+
+/**
+ * This is the default implementation of the {@link InputSplitAssigner} interface. The default input split assigner
+ * simply returns all input splits of an input vertex in the order they were originally computed.
+ */
+public class DefaultInputSplitAssigner implements InputSplitAssigner {
+
+	/** The logging object used to report information and errors. */
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultInputSplitAssigner.class);
+
+	/** The list of all splits */
+	private final List<InputSplit> splits = new ArrayList<InputSplit>();
+
+
+	public DefaultInputSplitAssigner(InputSplit[] splits) {
+		Collections.addAll(this.splits, splits);
+	}
+	
+	public DefaultInputSplitAssigner(Collection<? extends InputSplit> splits) {
+		this.splits.addAll(splits);
+	}
+	
+	
+	@Override
+	public InputSplit getNextInputSplit(String host) {
+		InputSplit next = null;
+		
+		// keep the synchronized part short
+		synchronized (this.splits) {
+			if (this.splits.size() > 0) {
+				next = this.splits.remove(this.splits.size() - 1);
+			}
+		}
+		
+		if (LOG.isDebugEnabled()) {
+			if (next == null) {
+				LOG.debug("Assigning split " + next + " to " + host);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("No more input splits available");
+				}
+			}
+		}
+		return next;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
index ccd6c51..522e69a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileInputFormat.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.common.io;
 
 import java.io.IOException;
@@ -44,7 +43,7 @@ import org.apache.flink.core.fs.Path;
  * The base class for {@link InputFormat}s that read from files. For specific input types the 
  * <tt>nextRecord()</tt> and <tt>reachedEnd()</tt> methods need to be implemented.
  * Additionally, one may override {@link #open(FileInputSplit)} and {@link #close()} to
- * change the lifecycle behavior.
+ * change the life cycle behavior.
  * <p>
  * After the {@link #open(FileInputSplit)} method completed, the file input data is available
  * from the {@link #stream} field.
@@ -358,8 +357,8 @@ public abstract class FileInputFormat<OT> implements InputFormat<OT, FileInputSp
 	}
 
 	@Override
-	public Class<FileInputSplit> getInputSplitType() {
-		return FileInputSplit.class;
+	public LocatableInputSplitAssigner getInputSplitAssigner(FileInputSplit[] splits) {
+		return new LocatableInputSplitAssigner(splits);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
index 4331116..008493d 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/FileOutputFormat.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.common.io;
 
 import java.io.IOException;


[35/63] [abbrv] Finalize ExecutionGraph state machine and calls

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index d225483..4ad030b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
@@ -56,24 +56,24 @@ public class TaskTest {
 			Task task = new Task(jid, vid, 2, 7, eid, "TestTask", taskManager);
 			task.setEnvironment(env);
 			
-			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, task.getExecutionState());
 			
 			// cancel
 			task.cancelExecution();
-			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, task.getExecutionState());
 			
 			// cannot go into running or finished state
 			
 			assertFalse(task.startExecution());
-			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, task.getExecutionState());
 			
 			assertFalse(task.markAsFinished());
-			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, task.getExecutionState());
 			
 			task.markFailed(new Exception("test"));
-			assertTrue(ExecutionState2.CANCELED == task.getExecutionState());
+			assertTrue(ExecutionState.CANCELED == task.getExecutionState());
 			
-			verify(taskManager, times(1)).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELED, null);
+			verify(taskManager, times(1)).notifyExecutionStateChange(jid, eid, ExecutionState.CANCELED, null);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -109,7 +109,7 @@ public class TaskTest {
 			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
 			when(env.getExecutingThread()).thenReturn(operation);
 			
-			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, task.getExecutionState());
 			
 			// start the execution
 			task.setEnvironment(env);
@@ -122,9 +122,9 @@ public class TaskTest {
 				ExceptionUtils.rethrow(error.get());
 			}
 			
-			assertEquals(ExecutionState2.FINISHED, task.getExecutionState());
+			assertEquals(ExecutionState.FINISHED, task.getExecutionState());
 			
-			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState2.FINISHED, null);
+			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState.FINISHED, null);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -159,7 +159,7 @@ public class TaskTest {
 			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
 			when(env.getExecutingThread()).thenReturn(operation);
 			
-			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, task.getExecutionState());
 			
 			// start the execution
 			task.setEnvironment(env);
@@ -173,8 +173,8 @@ public class TaskTest {
 			}
 			
 			// make sure the final state is correct and the task manager knows the changes
-			assertEquals(ExecutionState2.FAILED, task.getExecutionState());
-			verify(taskManager).notifyExecutionStateChange(Matchers.eq(jid), Matchers.eq(eid), Matchers.eq(ExecutionState2.FAILED), Matchers.anyString());
+			assertEquals(ExecutionState.FAILED, task.getExecutionState());
+			verify(taskManager).notifyExecutionStateChange(Matchers.eq(jid), Matchers.eq(eid), Matchers.eq(ExecutionState.FAILED), Matchers.any(Throwable.class));
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -216,7 +216,7 @@ public class TaskTest {
 			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
 			when(env.getExecutingThread()).thenReturn(operation);
 			
-			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, task.getExecutionState());
 			
 			// start the execution
 			task.setEnvironment(env);
@@ -234,8 +234,8 @@ public class TaskTest {
 			}
 			
 			// make sure the final state is correct and the task manager knows the changes
-			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
-			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELED, null);
+			assertEquals(ExecutionState.CANCELED, task.getExecutionState());
+			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState.CANCELED, null);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -266,14 +266,14 @@ public class TaskTest {
 			
 			task.setEnvironment(env);
 			
-			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, task.getExecutionState());
 			
 			task.startExecution();
 			task.getEnvironment().getExecutingThread().join();
 			
-			assertEquals(ExecutionState2.FINISHED, task.getExecutionState());
+			assertEquals(ExecutionState.FINISHED, task.getExecutionState());
 			
-			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState2.FINISHED, null);
+			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState.FINISHED, null);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -304,17 +304,17 @@ public class TaskTest {
 			
 			task.setEnvironment(env);
 			
-			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, task.getExecutionState());
 			
 			task.startExecution();
 			task.getEnvironment().getExecutingThread().join();
 			
-			assertEquals(ExecutionState2.FAILED, task.getExecutionState());
+			assertEquals(ExecutionState.FAILED, task.getExecutionState());
 			
-			verify(taskManager).notifyExecutionStateChange(Matchers.eq(jid), Matchers.eq(eid), Matchers.eq(ExecutionState2.FAILED), Matchers.anyString());
-			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELING, null);
-			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELED, null);
-			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState2.FINISHED, null);
+			verify(taskManager).notifyExecutionStateChange(Matchers.eq(jid), Matchers.eq(eid), Matchers.eq(ExecutionState.FAILED), Matchers.any(Throwable.class));
+			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState.CANCELING, null);
+			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState.CANCELED, null);
+			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState.FINISHED, null);
 		}
 		catch (Exception e) {
 			e.printStackTrace();


[10/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/OutputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/OutputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/OutputGate.java
index f2a2381..73bb9b5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/OutputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/OutputGate.java
@@ -20,6 +20,7 @@
 package org.apache.flink.runtime.io.network.gates;
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.runtime.deployment.ChannelDeploymentDescriptor;
@@ -81,13 +82,13 @@ public class OutputGate extends Gate<IOReadableWritable> {
 	// -----------------------------------------------------------------------------------------------------------------
 
 	public void initializeChannels(GateDeploymentDescriptor descriptor) {
-		int numChannels = descriptor.getNumberOfChannelDescriptors();
+		List<ChannelDeploymentDescriptor> channelDescr = descriptor.getChannels();
+		
+		int numChannels = channelDescr.size();
 		this.channels = new OutputChannel[numChannels];
 
-		setChannelType(descriptor.getChannelType());
-
 		for (int i = 0; i < numChannels; i++) {
-			ChannelDeploymentDescriptor channelDescriptor = descriptor.getChannelDescriptor(i);
+			ChannelDeploymentDescriptor channelDescriptor = channelDescr.get(i);
 
 			ChannelID id = channelDescriptor.getOutputChannelID();
 			ChannelID connectedId = channelDescriptor.getInputChannelID();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
index 1a8601e..27658f1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrier.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.iterative.concurrent;
 
 import java.util.concurrent.CountDownLatch;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobInputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobInputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobInputVertex.java
deleted file mode 100644
index e64de0a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobInputVertex.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * 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.jobgraph;
-
-import org.apache.flink.core.io.InputSplit;
-
-/**
- * An abstract base class for input vertices.
- */
-public abstract class AbstractJobInputVertex extends AbstractJobVertex {
-
-	/**
-	 * Constructs a new job input vertex with the given name.
-	 * 
-	 * @param name
-	 *        the name of the new job input vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	protected AbstractJobInputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-	
-	/**
-	 * Constructs a new job input vertex with the given name.
-	 * 
-	 * @param name
-	 *        the name of the new job input vertex
-	 * @param id
-	 *        the ID of this vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	protected AbstractJobInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-
-		jobGraph.addVertex(this);
-	}
-
-	/**
-	 * Returns the input split type of the input splits created by this input vertex
-	 *
-	 * @return input split type class
-	 */
-	public abstract Class<? extends InputSplit> getInputSplitType();
-
-	/**
-	 * Computes the input splits created by this input vertex
-	 *
-	 * @param minNumSplits Number of minimal input splits
-	 * @return Array of input splits
-	 * @throws Exception
-	 */
-	public abstract InputSplit[] getInputSplits(int minNumSplits) throws Exception;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java
deleted file mode 100644
index edb8d0e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * 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.jobgraph;
-
-/**
- * An abstract base class for output vertices in Nephele.
- */
-public abstract class AbstractJobOutputVertex extends AbstractJobVertex {
-
-	/**
-	 * Constructs a new job output vertex with the given name.
-	 * 
-	 * @param name
-	 *        the name of the new job output vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	protected AbstractJobOutputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-	
-	protected AbstractJobOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-		jobGraph.addVertex(this);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index cc7a5d0..cc2cbd8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -16,563 +16,286 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph;
 
-import java.io.IOException;
 import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.commons.lang3.Validate;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
+import org.apache.flink.core.io.InputSplitSource;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.util.EnumUtils;
-import org.apache.flink.util.StringUtils;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 
 /**
  * An abstract base class for a job vertex.
  */
-public abstract class AbstractJobVertex implements IOReadableWritable {
+public class AbstractJobVertex implements java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
 
 	private static final String DEFAULT_NAME = "(unnamed vertex)";
 	
-	/**
-	 * List of produced data sets, one per writer
-	 */
-	private final ArrayList<IntermediateDataSet> results = new ArrayList<IntermediateDataSet>();
+	
+	// --------------------------------------------------------------------------------------------
+	// Members that define the structure / topology of the graph
+	// --------------------------------------------------------------------------------------------
 
-	/**
-	 * List of edges with incoming data. One per Reader.
-	 */
-	private final ArrayList<JobEdge> inputs = new ArrayList<JobEdge>();
+	/** The ID of the vertex. */
+	private final JobVertexID id;
 
-	/**
-	 * The name of the vertex or task, respectively.
-	 */
+	/** The name of the vertex */
 	private final String name;
 
-	/**
-	 * The ID of the vertex.
-	 */
-	private final JobVertexID id;
+	/** List of produced data sets, one per writer */
+	private final ArrayList<IntermediateDataSet> results = new ArrayList<IntermediateDataSet>();
 
-	/**
-	 * The graph this vertex belongs to.
-	 */
-	private final JobGraph jobGraph;
+	/** List of edges with incoming data. One per Reader. */
+	private final ArrayList<JobEdge> inputs = new ArrayList<JobEdge>();
 
-	/**
-	 * Number of subtasks to split this task into at runtime.
-	 */
-	private int numberOfSubtasks = -1;
+	/** Number of subtasks to split this task into at runtime.*/
+	private int parallelism = -1;
 
-	/**
-	 * Other task to share a (set of) of instances with at runtime.
-	 */
-	private AbstractJobVertex vertexToShareInstancesWith;
+	/** Custom configuration passed to the assigned task at runtime. */
+	private Configuration configuration;
 
-	/**
-	 * Custom configuration passed to the assigned task at runtime.
-	 */
-	private Configuration configuration = new Configuration();
+	/** The class of the invokable. */
+	private String invokableClassName;
 
-	/**
-	 * The class of the invokable.
-	 */
-	protected Class<? extends AbstractInvokable> invokableClass;
+	/** Optionally, a source of input splits */
+	private InputSplitSource<?> inputSplitSource;
+	
+	/** Optionally, a sharing group that allows subtasks from different job vertices to run concurrently in one slot */
+	private SlotSharingGroup slotSharingGroup;
 
 	// --------------------------------------------------------------------------------------------
-	
+
 	/**
 	 * Constructs a new job vertex and assigns it with the given name.
 	 * 
-	 * @param name
-	 *        the name of the new job vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
+	 * @param name The name of the new job vertex.
 	 */
-	protected AbstractJobVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
+	public AbstractJobVertex(String name) {
+		this(name, null);
 	}
 	
 	/**
 	 * Constructs a new job vertex and assigns it with the given name.
 	 * 
-	 * @param name
-	 *        the name of the new job vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
+	 * @param name The name of the new job vertex.
+	 * @param id The id of the job vertex.
 	 */
-	protected AbstractJobVertex(String name, JobVertexID id, JobGraph jobGraph) {
+	public AbstractJobVertex(String name, JobVertexID id) {
 		this.name = name == null ? DEFAULT_NAME : name;
 		this.id = id == null ? new JobVertexID() : id;
-		this.jobGraph = jobGraph;
 	}
 	
 	// --------------------------------------------------------------------------------------------
-
+	
 	/**
-	 * Connects the job vertex to the specified job vertex.
+	 * Returns the ID of this job vertex.
 	 * 
-	 * @param vertex
-	 *        the vertex this vertex should connect to
-	 * @throws JobGraphDefinitionException
-	 *         thrown if the given vertex cannot be connected to <code>vertex</code> in the requested manner
+	 * @return The ID of this job vertex
 	 */
-	public void connectTo(final AbstractJobVertex vertex) throws JobGraphDefinitionException {
-		this.connectTo(vertex, null, -1, -1, DistributionPattern.BIPARTITE);
+	public JobVertexID getID() {
+		return this.id;
 	}
-
+	
 	/**
-	 * Connects the job vertex to the specified job vertex.
+	 * Returns the name of the vertex.
 	 * 
-	 * @param vertex
-	 *        the vertex this vertex should connect to
-	 * @param indexOfOutputGate
-	 *        index of the producing task's output gate to be used, <code>-1</code> will determine the next free index
-	 *        number
-	 * @param indexOfInputGate
-	 *        index of the consuming task's input gate to be used, <code>-1</code> will determine the next free index
-	 *        number
-	 * @throws JobGraphDefinitionException
-	 *         thrown if the given vertex cannot be connected to <code>vertex</code> in the requested manner
+	 * @return The name of the vertex.
 	 */
-	public void connectTo(final AbstractJobVertex vertex, final int indexOfOutputGate, final int indexOfInputGate)
-			throws JobGraphDefinitionException {
-		this.connectTo(vertex, null, indexOfOutputGate, indexOfInputGate, DistributionPattern.BIPARTITE);
+	public String getName() {
+		return this.name;
 	}
 
 	/**
-	 * Connects the job vertex to the specified job vertex.
+	 * Returns the number of produced intermediate data sets.
 	 * 
-	 * @param vertex
-	 *        the vertex this vertex should connect to
-	 * @param channelType
-	 *        the channel type the two vertices should be connected by at runtime
-	 * @throws JobGraphDefinitionException
-	 *         thrown if the given vertex cannot be connected to <code>vertex</code> in the requested manner
+	 * @return The number of produced intermediate data sets.
 	 */
-	public void connectTo(final AbstractJobVertex vertex, final ChannelType channelType) throws JobGraphDefinitionException {
-		this.connectTo(vertex, channelType, -1, -1, DistributionPattern.BIPARTITE);
+	public int getNumberOfProducedIntermediateDataSets() {
+		return this.results.size();
 	}
 
 	/**
-	 * Connects the job vertex to the specified job vertex.
+	 * Returns the number of inputs.
 	 * 
-	 * @param vertex
-	 *        the vertex this vertex should connect to
-	 * @param channelType
-	 *        the channel type the two vertices should be connected by at runtime
-	 * @param distributionPattern
-	 *        the distribution pattern between the two job vertices
-	 * @throws JobGraphDefinitionException
-	 *         thrown if the given vertex cannot be connected to <code>vertex</code> in the requested manner
+	 * @return The number of inputs.
 	 */
-	public void connectTo(final AbstractJobVertex vertex, final ChannelType channelType,
-			final DistributionPattern distributionPattern)
-			throws JobGraphDefinitionException {
-		this.connectTo(vertex, channelType, -1, -1, distributionPattern);
+	public int getNumberOfInputs() {
+		return this.inputs.size();
 	}
 
 	/**
-	 * Connects the job vertex to the specified job vertex.
+	 * Returns the vertex's configuration object which can be used to pass custom settings to the task at runtime.
 	 * 
-	 * @param vertex
-	 *        the vertex this vertex should connect to
-	 * @param channelType
-	 *        the channel type the two vertices should be connected by at runtime
-	 * @param indexOfOutputGate
-	 *        index of the producing task's output gate to be used, <code>-1</code> will determine the next free index
-	 *        number
-	 * @param indexOfInputGate
-	 *        index of the consuming task's input gate to be used, <code>-1</code> will determine the next free index
-	 *        number
-	 * @param distributionPattern
-	 * 		  the distribution pattern between the two job vertices
-	 * @throws JobGraphDefinitionException
-	 *         thrown if the given vertex cannot be connected to <code>vertex</code> in the requested manner
+	 * @return the vertex's configuration object
 	 */
-	public void connectTo(final AbstractJobVertex vertex, final ChannelType channelType, int indexOfOutputGate, int indexOfInputGate,
-			DistributionPattern distributionPattern)
-			throws JobGraphDefinitionException {
-
-		if (vertex == null) {
-			throw new JobGraphDefinitionException("Target vertex is null!");
-		}
-
-		if (indexOfOutputGate == -1) {
-			indexOfOutputGate = getFirstFreeOutputGateIndex();
-		}
-
-		// Make sure the array is big enough
-		for (int i = this.forwardEdges.size(); i <= indexOfOutputGate; i++) {
-			this.forwardEdges.add(null);
-		}
-
-		if (this.forwardEdges.get(indexOfOutputGate) != null) {
-			throw new JobGraphDefinitionException("Source vertex " + this.name + " already has an edge at index "
-				+ indexOfOutputGate);
-		}
-
-		if (indexOfInputGate == -1) {
-			indexOfInputGate = vertex.getFirstFreeInputGateIndex();
-		} else {
-			if (vertex.getBackwardConnection(indexOfInputGate) != null) {
-				throw new JobGraphDefinitionException("Target vertex " + vertex.getName()
-					+ " already has an edge at index " + indexOfInputGate);
-			}
+	public Configuration getConfiguration() {
+		if (this.configuration == null) {
+			this.configuration = new Configuration();
 		}
-
-		// Add new edge
-		this.forwardEdges.set(indexOfOutputGate, new JobEdge(vertex, channelType, indexOfInputGate,
-			distributionPattern));
-		vertex.connectBacklink(this, channelType, indexOfOutputGate, indexOfInputGate,
-			distributionPattern);
+		return this.configuration;
 	}
-
-	/**
-	 * Returns the index of this vertex's first free output gate.
-	 * 
-	 * @return the index of the first free output gate
-	 */
-	protected int getFirstFreeOutputGateIndex() {
-
-		for (int i = 0; i < this.forwardEdges.size(); i++) {
-
-			if (this.forwardEdges.get(i) == null) {
-				return i;
-			}
-		}
-
-		return this.forwardEdges.size();
+	
+	public void setInvokableClass(Class<? extends AbstractInvokable> invokable) {
+		Validate.notNull(invokable);
+		this.invokableClassName = invokable.getName();
 	}
-
+	
 	/**
-	 * Returns the index of this vertex's first free input gate.
+	 * Returns the name of the invokable class which represents the task of this vertex.
 	 * 
-	 * @return the index of the first free input gate
+	 * @return The name of the invokable class, <code>null</code> if not set.
 	 */
-	protected int getFirstFreeInputGateIndex() {
-
-		for (int i = 0; i < this.backwardEdges.size(); i++) {
-
-			if (this.backwardEdges.get(i) == null) {
-				return i;
-			}
-		}
-
-		return this.backwardEdges.size();
+	public String getInvokableClassName() {
+		return this.invokableClassName;
 	}
-
+	
 	/**
-	 * Creates a backward link from a connected job vertex.
+	 * Returns the invokable class which represents the task of this vertex
 	 * 
-	 * @param vertex
-	 *        the job vertex to connect to
-	 * @param channelType
-	 *        the channel type the two vertices should be connected by at runtime
-	 * @param indexOfOutputGate
-	 *        index of the producing task's output gate to be used
-	 * @param indexOfInputGate
-	 *        index of the consuming task's input gate to be used
-	 * @param distributionPattern
-	 * 		  the distribution pattern between the two job vertices
+	 * @param cl The classloader used to resolve user-defined classes
+	 * @return The invokable class, <code>null</code> if it is not set
 	 */
-	private void connectBacklink(final AbstractJobVertex vertex, final ChannelType channelType,
-			final int indexOfOutputGate, final int indexOfInputGate,
-			DistributionPattern distributionPattern) {
-
-		// Make sure the array is big enough
-		for (int i = this.backwardEdges.size(); i <= indexOfInputGate; i++) {
-			this.backwardEdges.add(null);
+	public Class<? extends AbstractInvokable> getInvokableClass(ClassLoader cl) {
+		if (cl == null) {
+			throw new NullPointerException("The classloader must not be null.");
+		}
+		if (invokableClassName == null) {
+			return null;
+		}
+		
+		try {
+			return Class.forName(invokableClassName, true, cl).asSubclass(AbstractInvokable.class);
+		}
+		catch (ClassNotFoundException e) {
+			throw new RuntimeException("The user-code class could not be resolved.", e);
+		}
+		catch (ClassCastException e) {
+			throw new RuntimeException("The user-code class is no subclass of " + AbstractInvokable.class.getName(), e);
 		}
-
-		this.backwardEdges.set(indexOfInputGate, new JobEdge(vertex, channelType, indexOfOutputGate,
-			distributionPattern));
-	}
-
-	/**
-	 * Sets the name of the vertex.
-	 * 
-	 * @param name 
-	 *        The name of the vertex.
-	 */
-	public void setName(String name) {
-		this.name = name;
 	}
 	
 	/**
-	 * Returns the name of the vertex.
+	 * Gets the degree of parallelism of the task.
 	 * 
-	 * @return the name of the vertex or <code>null</code> if no name is set.
+	 * @return The degree of parallelism of the task.
 	 */
-	public String getName() {
-		return this.name;
+	public int getParallelism() {
+		return parallelism;
 	}
 
 	/**
-	 * Returns the number of forward connections.
+	 * Sets the degree of parallelism for the task.
 	 * 
-	 * @return the number of forward connections
+	 * @param parallelism The degree of parallelism for the task.
 	 */
-	public int getNumberOfForwardConnections() {
-		return this.forwardEdges.size();
+	public void setParallelism(int parallelism) {
+		if (parallelism < 1) {
+			throw new IllegalArgumentException("The degree of parallelism must be at least one.");
+		}
+		this.parallelism = parallelism;
 	}
-
-	/**
-	 * Returns the number of backward connections.
-	 * 
-	 * @return the number of backward connections
-	 */
-	public int getNumberOfBackwardConnections() {
-		return this.backwardEdges.size();
+	
+	public InputSplitSource<?> getInputSplitSource() {
+		return inputSplitSource;
 	}
 
-	/**
-	 * Returns the forward edge with index <code>index</code>.
-	 * 
-	 * @param index
-	 *        the index of the edge
-	 * @return the forward edge or <code>null</code> if no edge exists at the specified index.
-	 */
-	public JobEdge getForwardConnection(final int index) {
-
-		if (index < this.forwardEdges.size()) {
-			return this.forwardEdges.get(index);
-		}
-
-		return null;
+	public void setInputSplitSource(InputSplitSource<?> inputSplitSource) {
+		this.inputSplitSource = inputSplitSource;
 	}
-
-	/**
-	 * Returns the backward edge with index <code>index</code>.
-	 * 
-	 * @param index
-	 *        the index of the edge
-	 * @return the backward edge or <code>null</code> if no edge exists at the specified index
-	 */
-	public JobEdge getBackwardConnection(final int index) {
-
-		if (index < this.backwardEdges.size()) {
-			return this.backwardEdges.get(index);
-		}
-
-		return null;
+	
+	public List<IntermediateDataSet> getProducedDataSets() {
+		return this.results;
 	}
-
-	/**
-	 * Returns the ID of this job vertex.
-	 * 
-	 * @return the ID of this job vertex
-	 */
-	public JobVertexID getID() {
-		return this.id;
+	
+	public List<JobEdge> getInputs() {
+		return this.inputs;
 	}
-
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void read(final DataInputView in) throws IOException {
-
-		if (jobGraph == null) {
-			throw new IOException("jobGraph is null, cannot deserialize");
-		}
-
-		// Read number of subtasks
-		this.numberOfSubtasks = in.readInt();
-
-		// Read vertex to share instances with
-		if (in.readBoolean()) {
-			final JobVertexID id = new JobVertexID();
-			id.read(in);
-			final AbstractJobVertex vertexToShareInstancesWith = this.jobGraph.findVertexByID(id);
-			if (vertexToShareInstancesWith == null) {
-				throw new IOException("Cannot find vertex with id " + id + " share instances with");
-			}
-
-			this.vertexToShareInstancesWith = vertexToShareInstancesWith;
-		}
-
-		// Find the class loader for the job
-		final ClassLoader cl = LibraryCacheManager.getClassLoader(this.getJobGraph().getJobID());
-		if (cl == null) {
-			throw new IOException("Cannot find class loader for vertex " + getID());
-		}
-
-		// Re-instantiate the configuration object with the correct class loader and read the configuration
-		this.configuration = new Configuration(cl);
-		this.configuration.read(in);
-
-		// Read number of forward edges
-		final int numForwardEdges = in.readInt();
-
-		// Now reconnect to other vertices via the reconstruction map
-		final JobVertexID tmpID = new JobVertexID();
-		for (int i = 0; i < numForwardEdges; i++) {
-			if (in.readBoolean()) {
-				tmpID.read(in);
-				final AbstractJobVertex jv = jobGraph.findVertexByID(tmpID);
-				if (jv == null) {
-					throw new IOException("Cannot find vertex with id " + tmpID);
-				}
-
-				final ChannelType channelType = EnumUtils.readEnum(in, ChannelType.class);
-				final DistributionPattern distributionPattern = EnumUtils.readEnum(in, DistributionPattern.class);
-				final int indexOfInputGate = in.readInt();
-
-				try {
-					this.connectTo(jv, channelType, i, indexOfInputGate, distributionPattern);
-				} catch (JobGraphDefinitionException e) {
-					throw new IOException(StringUtils.stringifyException(e));
-				}
-			} else {
-				this.forwardEdges.add(null);
-			}
-		}
-
-		// Read the invokable class
-		final boolean isNotNull = in.readBoolean();
-		if (!isNotNull) {
-			return;
+	
+	public void setSlotSharingGroup(SlotSharingGroup grp) {
+		if (this.slotSharingGroup != null) {
+			this.slotSharingGroup.removeVertexFromGroup(id);
 		}
-
-		// Read the name of the expected class
-		final String className = StringRecord.readString(in);
-
-		try {
-			this.invokableClass = (Class<? extends AbstractInvokable>) Class.forName(className, true, cl);
-		} catch (ClassNotFoundException cnfe) {
-			throw new IOException("Class " + className + " not found in one of the supplied jar files: "
-				+ StringUtils.stringifyException(cnfe));
+		
+		this.slotSharingGroup = grp;
+		if (grp != null) {
+			grp.addVertexToGroup(id);
 		}
 	}
-
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-
-		// Number of subtasks
-		out.writeInt(this.numberOfSubtasks);
-
-		// Vertex to share instance with
-		if (this.vertexToShareInstancesWith != null) {
-			out.writeBoolean(true);
-			this.vertexToShareInstancesWith.getID().write(out);
-		} else {
-			out.writeBoolean(false);
-		}
-
-		// Write the configuration
-		this.configuration.write(out);
-
-		// We ignore the backward edges and connect them when we reconstruct the graph on the remote side, only write
-		// number of forward edges
-		out.writeInt(this.forwardEdges.size());
-
-		// Now output the IDs of the vertices this vertex is connected to
-		for (int i = 0; i < this.forwardEdges.size(); i++) {
-			final JobEdge edge = this.forwardEdges.get(i);
-			if (edge == null) {
-				out.writeBoolean(false);
-			} else {
-				out.writeBoolean(true);
-				edge.getConnectedVertex().getID().write(out);
-				EnumUtils.writeEnum(out, edge.getChannelType());
-				EnumUtils.writeEnum(out, edge.getDistributionPattern());
-				out.writeInt(edge.getIndexOfInputGate());
-			}
-		}
-
-		// Write the invokable class
-		if (this.invokableClass == null) {
-			out.writeBoolean(false);
-			return;
-		}
-
-		out.writeBoolean(true);
-
-		// Write out the name of the class
-		StringRecord.writeString(out, this.invokableClass.getName());
+	
+	public SlotSharingGroup getSlotSharingGroup() {
+		return slotSharingGroup;
 	}
+	
+	// --------------------------------------------------------------------------------------------
 
-	/**
-	 * Returns the job graph this job vertex belongs to.
-	 * 
-	 * @return the job graph this job vertex belongs to or <code>null</code> if no job graph has been set yet
-	 */
-	public JobGraph getJobGraph() {
-		return this.jobGraph;
+	public IntermediateDataSet createAndAddResultDataSet() {
+		return createAndAddResultDataSet(new IntermediateDataSetID());
 	}
-
-	/**
-	 * Sets the number of subtasks the task this vertex represents should be split into at runtime.
-	 * 
-	 * @param numberOfSubtasks
-	 *        the number of subtasks this vertex represents should be split into at runtime
-	 */
-	public void setNumberOfSubtasks(final int numberOfSubtasks) {
-		this.numberOfSubtasks = numberOfSubtasks;
+	
+	public IntermediateDataSet createAndAddResultDataSet(IntermediateDataSetID id) {
+		IntermediateDataSet result = new IntermediateDataSet(id, this);
+		this.results.add(result);
+		return result;
 	}
-
-	/**
-	 * Returns the number of subtasks the task this vertex represents should be split into at runtime.
-	 * 
-	 * @return the number of subtasks this vertex represents should be split into at runtime, <code>-1</code> if
-	 *         unspecified
-	 */
-	public int getNumberOfSubtasks() {
-		return this.numberOfSubtasks;
+	
+	public void connectDataSetAsInput(IntermediateDataSet dataSet, DistributionPattern distPattern) {
+		JobEdge edge = new JobEdge(dataSet, this, distPattern);
+		this.inputs.add(edge);
+		dataSet.addConsumer(edge);
 	}
-
-	/**
-	 * Sets the vertex this vertex should share its instances with at runtime.
-	 * 
-	 * @param vertex
-	 *        the vertex this vertex should share its instances with at runtime
-	 */
-	public void setVertexToShareInstancesWith(final AbstractJobVertex vertex) {
-		this.vertexToShareInstancesWith = vertex;
+	
+	public void connectNewDataSetAsInput(AbstractJobVertex input, DistributionPattern distPattern) {
+		IntermediateDataSet dataSet = input.createAndAddResultDataSet();
+		JobEdge edge = new JobEdge(dataSet, this, distPattern);
+		this.inputs.add(edge);
+		dataSet.addConsumer(edge);
 	}
-
-	/**
-	 * Returns the vertex this vertex should share its instance with at runtime.
-	 * 
-	 * @return the vertex this vertex should share its instance with at runtime, <code>null</code> if undefined
-	 */
-	public AbstractJobVertex getVertexToShareInstancesWith() {
-		return this.vertexToShareInstancesWith;
+	
+	public void connectIdInput(IntermediateDataSetID dataSetId, DistributionPattern distPattern) {
+		JobEdge edge = new JobEdge(dataSetId, this, distPattern);
+		this.inputs.add(edge);
 	}
-
-	/**
-	 * Returns the vertex's configuration object which can be used to pass custom settings to the task at runtime.
-	 * 
-	 * @return the vertex's configuration object
-	 */
-	public Configuration getConfiguration() {
-		return this.configuration;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public boolean isInputVertex() {
+		return this.inputs.isEmpty();
 	}
-
-	public void setInvokableClass(Class<? extends AbstractInvokable> invokable) {
-		Validate.notNull(invokable);
-		this.invokableClass = invokable;
+	
+	public boolean isOutputVertex() {
+		return this.results.isEmpty();
+	}
+	
+	public boolean hasNoConnectedInputs() {
+		for (JobEdge edge : inputs) {
+			if (!edge.isIdReference()) {
+				return false;
+			}
+		}
+		
+		return true;
 	}
 	
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * Returns the invokable class which represents the task of this vertex
+	 * A hook that can be overwritten by sub classes to implement logic that is called by the 
+	 * master when the job starts.
 	 * 
-	 * @return the invokable class, <code>null</code> if it is not set
+	 * @param loader The class loader for user defined code.
+	 * @throws Exception The method may throw exceptions which cause the job to fail immediately.
 	 */
-	public Class<? extends AbstractInvokable> getInvokableClass() {
-		return this.invokableClass;
-	}
+	public void initializeOnMaster(ClassLoader loader) throws Exception {}
 	
+	// --------------------------------------------------------------------------------------------
+
 	@Override
 	public String toString() {
-		return this.name + " (" + this.invokableClass + ')';
+		return this.name + " (" + this.invokableClassName + ')';
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/DistributionPattern.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/DistributionPattern.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/DistributionPattern.java
index 32e3233..a447d36 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/DistributionPattern.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/DistributionPattern.java
@@ -16,15 +16,12 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph;
 
 /**
  * A distribution pattern determines which subtasks of a producing Nephele task a wired to which
  * subtasks of a consuming subtask.
- * 
  */
-
 public enum DistributionPattern {
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java
deleted file mode 100644
index f79264a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * 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.jobgraph;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.operators.util.UserCodeWrapper;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-
-
-public class InputFormatInputVertex extends AbstractJobInputVertex {
-
-	private InputFormat<?, ?> inputFormat;
-	
-	public InputFormatInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-	
-	/**
-	 * Creates a new job file input vertex with the specified name.
-	 * 
-	 * @param name
-	 *        The name of the new job file input vertex.
-	 * @param jobGraph
-	 *        The job graph this vertex belongs to.
-	 */
-	public InputFormatInputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        The job graph this vertex belongs to.
-	 */
-	public InputFormatInputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-	
-	public void setInputFormat(InputFormat<?, ?> format) {
-		this.inputFormat = format;
-	}
-	
-	public void initializeInputFormatFromTaskConfig(ClassLoader cl) {
-		TaskConfig cfg = new TaskConfig(getConfiguration());
-		
-		UserCodeWrapper<InputFormat<?, ?>> wrapper = cfg.<InputFormat<?, ?>>getStubWrapper(cl);
-		
-		if (wrapper != null) {
-			this.inputFormat = wrapper.getUserCodeObject(InputFormat.class, cl);
-			this.inputFormat.configure(cfg.getStubParameters());
-		}
-	}
-
-	/**
-	 * Gets the input split type class
-	 *
-	 * @return Input split type class
-	 */
-	@Override
-	public Class<? extends InputSplit> getInputSplitType() {
-		if (inputFormat == null){
-			return InputSplit.class;
-		}
-
-		return inputFormat.getInputSplitType();
-	}
-
-	/**
-	 * Gets the input splits from the input format.
-	 *
-	 * @param minNumSplits Number of minimal input splits
-	 * @return Array of input splits
-	 * @throws IOException
-	 */
-	@Override
-	public InputSplit[] getInputSplits(int minNumSplits) throws IOException {
-		if (inputFormat == null){
-			return null;
-		}
-
-		return inputFormat.createInputSplits(minNumSplits);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java
new file mode 100644
index 0000000..29e235d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatVertex.java
@@ -0,0 +1,58 @@
+/**
+ * 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.jobgraph;
+
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.operators.util.UserCodeWrapper;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+
+public class InputFormatVertex extends AbstractJobVertex {
+
+	private static final long serialVersionUID = 1L;
+	
+	/** Caches the output format associated to this output vertex. */
+	private transient InputFormat<?, ?> inputFormat;
+	
+	
+	public InputFormatVertex(String name) {
+		super(name);
+	}
+	
+	public InputFormatVertex(String name, JobVertexID id) {
+		super(name, id);
+	}
+	
+	
+	@Override
+	public void initializeOnMaster(ClassLoader loader) throws Exception {
+		if (inputFormat == null) {
+			TaskConfig cfg = new TaskConfig(getConfiguration());
+			UserCodeWrapper<InputFormat<?, ?>> wrapper = cfg.<InputFormat<?, ?>>getStubWrapper(loader);
+			
+			if (wrapper == null) {
+				throw new Exception("No input format present in InputFormatVertex's task configuration.");
+			}
+			
+			inputFormat = wrapper.getUserCodeObject(InputFormat.class, loader);
+			inputFormat.configure(cfg.getStubParameters());
+		}
+		
+		setInputSplitSource(inputFormat);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java
new file mode 100644
index 0000000..25a901b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSet.java
@@ -0,0 +1,83 @@
+/**
+ * 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.jobgraph;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * An intermediate data set is the data set produced by an operator - either a
+ * source or any intermediate operation.
+ * 
+ * Intermediate data sets may be read by other operators, materialized, or
+ * discarded.
+ */
+public class IntermediateDataSet implements java.io.Serializable {
+	
+	private static final long serialVersionUID = 1L;
+
+	
+	private final IntermediateDataSetID id; 		// the identifier
+	
+	private final AbstractJobVertex producer;		// the operation that produced this data set
+	
+	private final List<JobEdge> consumers = new ArrayList<JobEdge>();
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public IntermediateDataSet(AbstractJobVertex producer) {
+		this(new IntermediateDataSetID(), producer);
+	}
+	
+	public IntermediateDataSet(IntermediateDataSetID id, AbstractJobVertex producer) {
+		if (id == null || producer == null) {
+			throw new NullPointerException();
+		}
+		
+		this.id = id;
+		this.producer = producer;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	
+	public IntermediateDataSetID getId() {
+		return id;
+	}
+
+	public AbstractJobVertex getProducer() {
+		return producer;
+	}
+	
+	public List<JobEdge> getConsumers() {
+		return this.consumers;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void addConsumer(JobEdge edge) {
+		this.consumers.add(edge);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "Intermediate Data Set (" + id + ")";
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java
new file mode 100644
index 0000000..5c6a596
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/IntermediateDataSetID.java
@@ -0,0 +1,53 @@
+/**
+ * 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.jobgraph;
+
+import java.util.UUID;
+
+import org.apache.flink.runtime.AbstractID;
+
+public class IntermediateDataSetID extends AbstractID {
+
+	private static final long serialVersionUID = 1L;
+
+	/**
+	 * Creates an new random intermediate data set ID.
+	 */
+	public IntermediateDataSetID() {
+		super();
+	}
+	
+	/**
+	 * Creates a new intermediate data set ID with the bytes of the given ID.
+	 * 
+	 * @param from The ID to create this ID from.
+	 */
+	public IntermediateDataSetID(AbstractID from) {
+		super(from);
+	}
+	
+	/**
+	 * Creates a new intermediate data set ID with the bytes of the given UUID.
+	 * 
+	 * @param from The UUID to create this ID from.
+	 */
+	public IntermediateDataSetID(UUID from) {
+		super(from.getLeastSignificantBits(), from.getMostSignificantBits());
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
index 33b6576..b542f4f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
@@ -18,87 +18,124 @@
 
 package org.apache.flink.runtime.jobgraph;
 
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-
 /**
- * Objects of this class represent edges in the user's job graph.
- * The edges can be annotated by a specific channel and compression level.
+ * This class represent edges (communication channels) in a job graph.
+ * The edges always go from an intermediate result partition to a job vertex.
+ * An edge is parameterized with its {@link DistributionPattern}.
  */
-public class JobEdge {
-
-	/**
-	 * The channel type to be used for the resulting channel.
-	 */
-	private final ChannelType channelType;
+public class JobEdge implements java.io.Serializable {
 
-	/**
-	 * The vertex connected to this edge.
-	 */
-	private final AbstractJobVertex connectedVertex;
-
-	/**
-	 * The index of the consuming task's input gate.
-	 */
-	private final int indexOfInputGate;
+	private static final long serialVersionUID = 1L;
 	
-	/**
-	 * The distribution pattern that should be used for this job edge.
-	 */
+	
+	/** The vertex connected to this edge. */
+	private final AbstractJobVertex target;
+
+	/** The distribution pattern that should be used for this job edge. */
 	private final DistributionPattern distributionPattern;
+	
+	/** The data set at the source of the edge, may be null if the edge is not yet connected*/
+	private IntermediateDataSet source;
+	
+	/** The id of the source intermediate data set */
+	private IntermediateDataSetID sourceId;
 
 	/**
-	 * Constructs a new job edge.
+	 * Constructs a new job edge, that connects an intermediate result to a consumer task.
 	 * 
-	 * @param connectedVertex
-	 *        the vertex this edge should connect to
-	 * @param channelType
-	 *        the channel type this edge should be translated to at runtime
-	 * @param compressionLevel
-	 *        the compression level the corresponding channel should have at runtime
-	 * @param indexOfInputGate
-	 *        index of the consuming task's input gate that this edge connects to
+	 * @param source The data set that is at the source of this edge.
+	 * @param target The operation that is at the target of this edge.
+	 * @param distributionPattern The pattern that defines how the connection behaves in parallel.
 	 */
-	public JobEdge(final AbstractJobVertex connectedVertex, final ChannelType channelType,
-			final int indexOfInputGate, final DistributionPattern distributionPattern) {
-		this.connectedVertex = connectedVertex;
-		this.channelType = channelType;
-		this.indexOfInputGate = indexOfInputGate;
+	public JobEdge(IntermediateDataSet source, AbstractJobVertex target, DistributionPattern distributionPattern) {
+		if (source == null || target == null || distributionPattern == null) {
+			throw new NullPointerException();
+		}
+		this.target = target;
 		this.distributionPattern = distributionPattern;
+		this.source = source;
+		this.sourceId = source.getId();
 	}
-
+	
 	/**
-	 * Returns the channel type assigned to this edge.
+	 * Constructs a new job edge that refers to an intermediate result via the Id, rather than directly through
+	 * the intermediate data set structure.
 	 * 
-	 * @return the channel type assigned to this edge
+	 * @param sourceId The id of the data set that is at the source of this edge.
+	 * @param target The operation that is at the target of this edge.
+	 * @param distributionPattern The pattern that defines how the connection behaves in parallel.
 	 */
-	public ChannelType getChannelType() {
-		return this.channelType;
+	public JobEdge(IntermediateDataSetID sourceId, AbstractJobVertex target, DistributionPattern distributionPattern) {
+		if (sourceId == null || target == null || distributionPattern == null) {
+			throw new NullPointerException();
+		}
+		this.target = target;
+		this.distributionPattern = distributionPattern;
+		this.sourceId = sourceId;
 	}
 
+
 	/**
-	 * Returns the vertex this edge is connected to.
+	 * Returns the data set at the source of the edge. May be null, if the edge refers to the source via an ID
+	 * and has not been connected.
 	 * 
-	 * @return the vertex this edge is connected to
+	 * @return The data set at the source of the edge
 	 */
-	public AbstractJobVertex getConnectedVertex() {
-		return this.connectedVertex;
+	public IntermediateDataSet getSource() {
+		return source;
 	}
 
 	/**
-	 * Returns the index of the consuming task's input gate that this edge connects to.
+	 * Returns the vertex connected to this edge.
 	 * 
-	 * @return the index of the consuming task's input gate that this edge connects to
+	 * @return The vertex connected to this edge.
 	 */
-	public int getIndexOfInputGate() {
-		return this.indexOfInputGate;
+	public AbstractJobVertex getTarget() {
+		return target;
 	}
 	
 	/**
 	 * Returns the distribution pattern used for this edge.
 	 * 
-	 * @return
+	 * @return The distribution pattern used for this edge.
 	 */
 	public DistributionPattern getDistributionPattern(){
 		return this.distributionPattern;
 	}
+	
+	/**
+	 * Gets the ID of the consumed data set.
+	 * 
+	 * @return The ID of the consumed data set.
+	 */
+	public IntermediateDataSetID getSourceId() {
+		return sourceId;
+	}
+	
+	public boolean isIdReference() {
+		return this.source == null;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void connecDataSet(IntermediateDataSet dataSet) {
+		if (dataSet == null) {
+			throw new NullPointerException();
+		}
+		if (this.source != null) {
+			throw new IllegalStateException("The edge is already connected.");
+		}
+		if (!dataSet.getId().equals(sourceId)) {
+			throw new IllegalArgumentException("The data set to connect does not match the sourceId.");
+		}
+		
+		this.source = dataSet;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return String.format("%s --> %s []", sourceId, target, distributionPattern.name());
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 39dc382..f8b5ab9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -16,90 +16,135 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph;
 
 import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
+import org.apache.flink.api.common.InvalidProgramException;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStream;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStream;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.util.ClassUtils;
+import org.apache.flink.types.StringValue;
 
 /**
- * A job graph represents an entire job in Nephele. A job graph must consists at least of one job vertex
- * and must be acyclic.
+ * A job graph represents an entire Flink runtime job.
  */
 public class JobGraph implements IOReadableWritable {
 
-	/**
-	 * List of task vertices included in this job graph.
-	 */
-	private Map<JobVertexID, AbstractJobVertex> taskVertices = new LinkedHashMap<JobVertexID, AbstractJobVertex>();
+	/** Size of the buffer to be allocated for transferring attached files. */
+	private static final int BUFFERSIZE = 8192;
+	
+	
+	// --------------------------------------------------------------------------------------------
+	// Members that define the structure / topology of the graph
+	// --------------------------------------------------------------------------------------------
+	
+	/** List of JAR files required to run this job. */
+	private final ArrayList<Path> userJars = new ArrayList<Path>();
+	
+	/** List of task vertices included in this job graph. */
+	private final Map<JobVertexID, AbstractJobVertex> taskVertices = new LinkedHashMap<JobVertexID, AbstractJobVertex>();
+
+	/** The job configuration attached to this job. */
+	private final Configuration jobConfiguration = new Configuration();
+	
+	/** ID of this job. */
+	private final JobID jobID;
 
+	/** Name of this job. */
+	private String jobName;
+	
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * ID of this job.
+	 * Constructs a new job graph with no name and a random job ID.
 	 */
-	private JobID jobID;
+	public JobGraph() {
+		this((String) null);
+	}
 
 	/**
-	 * Name of this job.
+	 * Constructs a new job graph with the given name and a random job ID.
+	 * 
+	 * @param jobName The name of the job
 	 */
-	private String jobName;
-
+	public JobGraph(String jobName) {
+		this(null, jobName);
+	}
+	
 	/**
-	 * The job configuration attached to this job.
+	 * Constructs a new job graph with the given name and a random job ID.
+	 * 
+	 * @param jobId The id of the job
+	 * @param jobName The name of the job
 	 */
-	private Configuration jobConfiguration = new Configuration();
-
+	public JobGraph(JobID jobId, String jobName) {
+		this.jobID = jobId == null ? new JobID() : jobId;;
+		this.jobName = jobName == null ? "(unnamed job)" : jobName;
+	}
+	
 	/**
-	 * List of JAR files required to run this job.
+	 * Constructs a new job graph with no name and a random job ID.
+	 * 
+	 * @param vertices The vertices to add to the graph.
 	 */
-	private final ArrayList<Path> userJars = new ArrayList<Path>();
+	public JobGraph(AbstractJobVertex... vertices) {
+		this(null, vertices);
+	}
 
 	/**
-	 * Size of the buffer to be allocated for transferring attached files.
+	 * Constructs a new job graph with the given name and a random job ID.
+	 * 
+	 * @param jobName The name of the job.
+	 * @param vertices The vertices to add to the graph.
 	 */
-	private static final int BUFFERSIZE = 8192;
-
-	// --------------------------------------------------------------------------------------------
+	public JobGraph(String jobName, AbstractJobVertex... vertices) {
+		this(null, jobName, vertices);
+	}
 	
 	/**
-	 * Constructs a new job graph with a random job ID.
+	 * Constructs a new job graph with the given name and a random job ID.
+	 * 
+	 * @param jobId The id of the job.
+	 * @param jobName The name of the job.
+	 * @param vertices The vertices to add to the graph.
 	 */
-	public JobGraph() {
-		this.jobID = new JobID();
+	public JobGraph(JobID jobId, String jobName, AbstractJobVertex... vertices) {
+		this(jobId, jobName);
+		
+		for (AbstractJobVertex vertex : vertices) {
+			addVertex(vertex);
+		}
 	}
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * Constructs a new job graph with the given name and a random job ID.
+	 * Returns the ID of the job.
 	 * 
-	 * @param jobName
-	 *        the name for this job graph
+	 * @return the ID of the job
 	 */
-	public JobGraph(String jobName) {
-		this();
-		this.jobName = jobName;
+	public JobID getJobID() {
+		return this.jobID;
 	}
-
+	
 	/**
 	 * Returns the name assigned to the job graph.
 	 * 
@@ -121,7 +166,7 @@ public class JobGraph implements IOReadableWritable {
 	/**
 	 * Adds a new task vertex to the job graph if it is not already included.
 	 * 
-	 * @param taskVertex
+	 * @param vertex
 	 *        the new task vertex to be added
 	 */
 	public void addVertex(AbstractJobVertex vertex) {
@@ -130,77 +175,37 @@ public class JobGraph implements IOReadableWritable {
 		
 		// if we had a prior association, restore and throw an exception
 		if (previous != null) {
-			taskVertices.put(id, vertex);
+			taskVertices.put(id, previous);
 			throw new IllegalArgumentException("The JobGraph already contains a vertex with that id.");
 		}
 	}
 
 	/**
-	 * Returns the number of task vertices registered with the job graph.
-	 * 
-	 * @return the number of task vertices registered with the job graph
-	 */
-	public int getNumberOfTaskVertices() {
-		return this.taskVertices.size();
-	}
-
-	/**
 	 * Returns an Iterable to iterate all vertices registered with the job graph.
 	 * 
 	 * @return an Iterable to iterate all vertices registered with the job graph
 	 */
-	public Iterable<AbstractJobVertex> getTaskVertices() {
+	public Iterable<AbstractJobVertex> getVertices() {
 		return this.taskVertices.values();
 	}
-
-	/**
-	 * Returns the number of all job vertices registered with this job graph.
-	 * 
-	 * @return the number of all job vertices registered with this job graph
-	 */
-	public int getNumberOfVertices() {
-		return this.taskVertices.size();
-	}
-
-	/**
-	 * Auxiliary method to collect all vertices which are reachable from the input vertices.
-	 *
-	 * @param jv
-	 *        the currently considered job vertex
-	 * @param collector
-	 *        a temporary list to store the vertices that have already been visisted
-	 */
-	private void collectVertices(final AbstractJobVertex jv, final HashSet<JobVertexID> visited, final
-			List<AbstractJobVertex> collector) {
-		visited.add(jv.getID());
-		collector.add(jv);
-
-		for(int i =0; i < jv.getNumberOfForwardConnections(); i++){
-			AbstractJobVertex vertex = jv.getForwardConnection(i).getConnectedVertex();
-
-			if(!visited.contains(vertex.getID())){
-				collectVertices(vertex, visited, collector);
-			}
-		}
-	}
-
+	
 	/**
 	 * Returns an array of all job vertices that are registered with the job graph. The order in which the vertices
 	 * appear in the list is not defined.
 	 * 
 	 * @return an array of all job vertices that are registered with the job graph
 	 */
-	public AbstractJobVertex[] getAllJobVertices() {
+	public AbstractJobVertex[] getVerticesAsArray() {
 		return this.taskVertices.values().toArray(new AbstractJobVertex[this.taskVertices.size()]);
 	}
 
 	/**
-	 * Returns the ID of the job.
+	 * Returns the number of all vertices.
 	 * 
-	 * @return the ID of the job
+	 * @return The number of all vertices.
 	 */
-	public JobID getJobID() {
-		return this.jobID;
+	public int getNumberOfVertices() {
+		return this.taskVertices.size();
 	}
 
 	/**
@@ -213,295 +218,164 @@ public class JobGraph implements IOReadableWritable {
 	public AbstractJobVertex findVertexByID(JobVertexID id) {
 		return this.taskVertices.get(id);
 	}
+	
+	// --------------------------------------------------------------------------------------------
 
-	/**
-	 * Checks if the job graph is weakly connected.
-	 * 
-	 * @return <code>true</code> if the job graph is weakly connected, otherwise <code>false</code>
-	 */
-	public boolean isWeaklyConnected() {
-
-		final AbstractJobVertex[] reachable = getAllReachableJobVertices();
-		final AbstractJobVertex[] all = getAllJobVertices();
-
-		// Check if number if reachable vertices matches number of registered vertices
-		if (reachable.length != all.length) {
-			return false;
-		}
-
-		return true;
-	}
-
-	/**
-	 * Checks if the job graph is acyclic.
-	 * 
-	 * @return <code>true</code> if the job graph is acyclic, <code>false</code> otherwise
-	 */
-	public boolean isAcyclic() {
-
-		final AbstractJobVertex[] reachable = getAllReachableJobVertices();
-
-		final HashSet<JobVertexID> temporarilyMarked = new HashSet<JobVertexID>();
-		final HashSet<JobVertexID> permanentlyMarked = new HashSet<JobVertexID>();
-
-		for(int i = 0; i < reachable.length; i++){
-			if(detectCycle(reachable[i], temporarilyMarked, permanentlyMarked)){
-				return false;
-			}
+	public List<AbstractJobVertex> getVerticesSortedTopologicallyFromSources() throws InvalidProgramException {
+		// early out on empty lists
+		if (this.taskVertices.isEmpty()) {
+			return Collections.emptyList();
 		}
-
-		return true;
-	}
-
-	/**
-	 * Auxiliary method for cycle detection. Performs a depth-first traversal with vertex markings to detect a cycle.
-	 * If a node with a temporary marking is found, then there is a cycle. Once all children of a vertex have been
-	 * traversed the parent node cannot be part of another cycle and is thus permanently marked.
-	 *
-	 * @param jv current job vertex to check
-	 * @param temporarilyMarked set of temporarily marked nodes
-	 * @param permanentlyMarked set of permanently marked nodes
-	 * @return <code>true</code> if there is a cycle, <code>false</code> otherwise
-	 */
-	private boolean detectCycle(final AbstractJobVertex jv, final HashSet<JobVertexID> temporarilyMarked,
-								final HashSet<JobVertexID> permanentlyMarked){
-		JobVertexID vertexID = jv.getID();
-
-		if(permanentlyMarked.contains(vertexID)){
-			return false;
-		}else if(temporarilyMarked.contains(vertexID)){
-			return true;
-		}else{
-			temporarilyMarked.add(vertexID);
-
-			for(int i = 0; i < jv.getNumberOfForwardConnections(); i++){
-				if(detectCycle(jv.getForwardConnection(i).getConnectedVertex(), temporarilyMarked, permanentlyMarked)){
-					return true;
+		
+		ArrayList<AbstractJobVertex> sorted = new ArrayList<AbstractJobVertex>(this.taskVertices.size());
+		LinkedHashSet<AbstractJobVertex> remaining = new LinkedHashSet<AbstractJobVertex>(this.taskVertices.values());
+		
+		// start by finding the vertices with no input edges
+		// and the ones with disconnected inputs (that refer to some standalone data set)
+		{
+			Iterator<AbstractJobVertex> iter = remaining.iterator();
+			while (iter.hasNext()) {
+				AbstractJobVertex vertex = iter.next();
+				
+				if (vertex.hasNoConnectedInputs()) {
+					sorted.add(vertex);
+					iter.remove();
 				}
 			}
-
-			permanentlyMarked.add(vertexID);
-			return false;
-		}
-	}
-
-	/**
-	 * Checks for all registered job vertices if their in-/out-degree is correct.
-	 * 
-	 * @return <code>null</code> if the in-/out-degree of all vertices is correct or the first job vertex whose
-	 *         in-/out-degree is incorrect.
-	 */
-	public AbstractJobVertex areVertexDegreesCorrect() {
-
-		// Check input vertices
-		final Iterator<AbstractJobInputVertex> iter = getInputVertices();
-		while (iter.hasNext()) {
-
-			final AbstractJobVertex jv = iter.next();
-
-			if (jv.getNumberOfForwardConnections() < 1 || jv.getNumberOfBackwardConnections() > 0) {
-				return jv;
-			}
 		}
-
-		// Check task vertices
-		final Iterator<JobTaskVertex> iter2 = getTaskVertices();
-		while (iter2.hasNext()) {
-
-			final AbstractJobVertex jv = iter2.next();
-
-			if (jv.getNumberOfForwardConnections() < 1 || jv.getNumberOfBackwardConnections() < 1) {
-				return jv;
+		
+		int startNodePos = 0;
+		
+		// traverse from the nodes that were added until we found all elements
+		while (!remaining.isEmpty()) {
+			
+			// first check if we have more candidates to start traversing from. if not, then the
+			// graph is cyclic, which is not permitted
+			if (startNodePos >= sorted.size()) {
+				throw new InvalidProgramException("The job graph is cyclic.");
 			}
+			
+			AbstractJobVertex current = sorted.get(startNodePos++);
+			addNodesThatHaveNoNewPredecessors(current, sorted, remaining);
 		}
-
-		// Check output vertices
-		final Iterator<AbstractJobOutputVertex> iter3 = getOutputVertices();
-		while (iter3.hasNext()) {
-
-			final AbstractJobVertex jv = iter3.next();
-
-			if (jv.getNumberOfForwardConnections() > 0 || jv.getNumberOfBackwardConnections() < 1) {
-				return jv;
+		
+		return sorted;
+	}
+	
+	private void addNodesThatHaveNoNewPredecessors(AbstractJobVertex start, ArrayList<AbstractJobVertex> target, LinkedHashSet<AbstractJobVertex> remaining) {
+		
+		// forward traverse over all produced data sets and all their consumers
+		for (IntermediateDataSet dataSet : start.getProducedDataSets()) {
+			for (JobEdge edge : dataSet.getConsumers()) {
+				
+				// a vertex can be added, if it has no predecessors that are still in the 'remaining' set
+				AbstractJobVertex v = edge.getTarget();
+				boolean hasNewPredecessors = false;
+				
+				for (JobEdge e : v.getInputs()) {
+					// skip the edge through which we came
+					if (e == edge) {
+						continue;
+					}
+					
+					IntermediateDataSet source = e.getSource();
+					if (remaining.contains(source.getProducer())) {
+						hasNewPredecessors = true;
+						break;
+					}
+				}
+				
+				if (!hasNewPredecessors) {
+					target.add(v);
+					remaining.remove(v);
+					addNodesThatHaveNoNewPredecessors(v, target, remaining);
+				}
 			}
 		}
-
-		return null;
 	}
-
-
+	
+	// --------------------------------------------------------------------------------------------
+	//  Serialization / Deserialization
+	// --------------------------------------------------------------------------------------------
+	
 	@Override
-	public void read(final DataInputView in) throws IOException {
-
-		// Read job id
+	public void read(DataInputView in) throws IOException {
+		// write the simple fields
 		this.jobID.read(in);
-
-		// Read the job name
-		this.jobName = StringRecord.readString(in);
-
-		// Read required jar files
-		readRequiredJarFiles(in);
-
-		// First read total number of vertices;
+		this.jobName = StringValue.readString(in);
+		this.jobConfiguration.read(in);
+		
 		final int numVertices = in.readInt();
-
-		// First, recreate each vertex and add it to reconstructionMap
+		
+		@SuppressWarnings("resource")
+		ObjectInputStream ois = new ObjectInputStream(new DataInputViewStream(in));
 		for (int i = 0; i < numVertices; i++) {
-			final String className = StringRecord.readString(in);
-			final JobVertexID id = new JobVertexID();
-			id.read(in);
-			final String vertexName = StringRecord.readString(in);
-
-			Class<? extends IOReadableWritable> c;
-			try {
-				c = ClassUtils.getRecordByName(className);
-			} catch (ClassNotFoundException cnfe) {
-				throw new IOException(cnfe.toString());
-			}
-
-			// Find constructor
-			Constructor<? extends IOReadableWritable> cst;
 			try {
-				cst = c.getConstructor(String.class, JobVertexID.class, JobGraph.class);
-			} catch (SecurityException e1) {
-				throw new IOException(e1.toString());
-			} catch (NoSuchMethodException e1) {
-				throw new IOException(e1.toString());
+				AbstractJobVertex vertex = (AbstractJobVertex) ois.readObject();
+				taskVertices.put(vertex.getID(), vertex);
 			}
-
-			try {
-				cst.newInstance(vertexName, id, this);
-			} catch (IllegalArgumentException e) {
-				throw new IOException(e.toString());
-			} catch (InstantiationException e) {
-				throw new IOException(e.toString());
-			} catch (IllegalAccessException e) {
-				throw new IOException(e.toString());
-			} catch (InvocationTargetException e) {
-				throw new IOException(e.toString());
-			}
-		}
-
-		final JobVertexID tmpID = new JobVertexID();
-		for (int i = 0; i < numVertices; i++) {
-
-			AbstractJobVertex jv;
-
-			tmpID.read(in);
-			if (inputVertices.containsKey(tmpID)) {
-				jv = inputVertices.get(tmpID);
-			} else {
-				if (outputVertices.containsKey(tmpID)) {
-					jv = outputVertices.get(tmpID);
-				} else {
-					if (taskVertices.containsKey(tmpID)) {
-						jv = taskVertices.get(tmpID);
-					} else {
-						throw new IOException("Cannot find vertex with ID " + tmpID + " in any vertex map.");
-					}
-				}
+			catch (ClassNotFoundException e) {
+				throw new IOException(e);
 			}
-
-			// Read the vertex data
-			jv.read(in);
-		}
-
-		// Find the class loader for the job
-		final ClassLoader cl = LibraryCacheManager.getClassLoader(this.jobID);
-		if (cl == null) {
-			throw new IOException("Cannot find class loader for job graph " + this.jobID);
 		}
+		ois.close();
 
-		// Re-instantiate the job configuration object and read the configuration
-		this.jobConfiguration = new Configuration(cl);
-		this.jobConfiguration.read(in);
+		// Read required jar files
+		readRequiredJarFiles(in);
 	}
 
 
 	@Override
-	public void write(final DataOutputView out) throws IOException {
-
-		// Write job ID
+	public void write(DataOutputView out) throws IOException {
+		
+		// write the simple fields
 		this.jobID.write(out);
-
-		// Write out job name
-		StringRecord.writeString(out, this.jobName);
-
-		final AbstractJobVertex[] allVertices = this.getAllJobVertices();
-
-		// Write out all required jar files
-		writeRequiredJarFiles(out, allVertices);
-
-		// Write total number of vertices
-		out.writeInt(allVertices.length);
-
-		// First write out class name and id for every vertex
-		for (int i = 0; i < allVertices.length; i++) {
-
-			final String className = allVertices[i].getClass().getName();
-			StringRecord.writeString(out, className);
-			allVertices[i].getID().write(out);
-			StringRecord.writeString(out, allVertices[i].getName());
-		}
-
-		// Now write out vertices themselves
-		for (int i = 0; i < allVertices.length; i++) {
-			allVertices[i].getID().write(out);
-			allVertices[i].write(out);
-		}
-
-		// Write out configuration objects
+		StringValue.writeString(this.jobName, out);
 		this.jobConfiguration.write(out);
+		
+		// write the task vertices using java serialization (to resolve references in the object graph)
+		out.writeInt(taskVertices.size());
+		
+		ObjectOutputStream oos = new ObjectOutputStream(new DataOutputViewStream(out));
+		for (AbstractJobVertex vertex : this.taskVertices.values()) {
+			oos.writeObject(vertex);
+		}
+		oos.close();
+		
+		// Write out all required jar files
+		writeRequiredJarFiles(out);
 	}
 
+	// --------------------------------------------------------------------------------------------
+	//  Handling of attached JAR files
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * Writes the JAR files of all vertices in array <code>jobVertices</code> to the specified output stream.
+	 * Adds the path of a JAR file required to run the job on a task manager.
 	 * 
-	 * @param out
-	 *        the output stream to write the JAR files to
-	 * @param jobVertices
-	 *        array of job vertices whose required JAR file are to be written to the output stream
-	 * @throws IOException
-	 *         thrown if an error occurs while writing to the stream
+	 * @param jar
+	 *        path of the JAR file required to run the job on a task manager
 	 */
-	private void writeRequiredJarFiles(final DataOutputView out, final AbstractJobVertex[] jobVertices) throws
-			IOException {
-
-		// Now check if all the collected jar files really exist
-		final FileSystem fs = FileSystem.getLocalFileSystem();
-
-		for (int i = 0; i < this.userJars.size(); i++) {
-			if (!fs.exists(this.userJars.get(i))) {
-				throw new IOException("Cannot find jar file " + this.userJars.get(i));
-			}
+	public void addJar(Path jar) {
+		if (jar == null) {
+			throw new IllegalArgumentException();
 		}
 
-		// How many jar files follow?
-		out.writeInt(this.userJars.size());
-
-		for (int i = 0; i < this.userJars.size(); i++) {
-
-			final Path jar = this.userJars.get(i);
-
-			// Write out the actual path
-			jar.write(out);
-
-			// Write out the length of the file
-			final FileStatus file = fs.getFileStatus(jar);
-			out.writeLong(file.getLen());
-
-			// Now write the jar file
-			final FSDataInputStream inStream = fs.open(this.userJars.get(i));
-			final byte[] buf = new byte[BUFFERSIZE];
-			int read = inStream.read(buf, 0, buf.length);
-			while (read > 0) {
-				out.write(buf, 0, read);
-				read = inStream.read(buf, 0, buf.length);
-			}
+		if (!userJars.contains(jar)) {
+			userJars.add(jar);
 		}
 	}
 
 	/**
+	 * Returns a (possibly empty) array of paths to JAR files which are required to run the job on a task manager.
+	 * 
+	 * @return a (possibly empty) array of paths to JAR files which are required to run the job on a task manager
+	 */
+	public Path[] getJars() {
+		return userJars.toArray(new Path[userJars.size()]);
+	}
+	
+	/**
 	 * Reads required JAR files from an input stream and adds them to the
 	 * library cache manager.
 	 * 
@@ -535,98 +409,48 @@ public class JobGraph implements IOReadableWritable {
 		// Register this job with the library cache manager
 		LibraryCacheManager.register(this.jobID, this.userJars.toArray(new Path[0]));
 	}
-
-	/**
-	 * Adds the path of a JAR file required to run the job on a task manager.
-	 * 
-	 * @param jar
-	 *        path of the JAR file required to run the job on a task manager
-	 */
-	public void addJar(final Path jar) {
-
-		if (jar == null) {
-			return;
-		}
-
-		if (!userJars.contains(jar)) {
-			userJars.add(jar);
-		}
-	}
-
-	/**
-	 * Returns a (possibly empty) array of paths to JAR files which are required to run the job on a task manager.
-	 * 
-	 * @return a (possibly empty) array of paths to JAR files which are required to run the job on a task manager
-	 */
-	public Path[] getJars() {
-
-		return userJars.toArray(new Path[userJars.size()]);
-	}
-
+	
 	/**
-	 * Checks if any vertex of this job graph has an outgoing edge which is set to <code>null</code>. If this is the
-	 * case the respective vertex is returned.
+	 * Writes the JAR files of all vertices in array <code>jobVertices</code> to the specified output stream.
 	 * 
-	 * @return the vertex which has an outgoing edge set to <code>null</code> or <code>null</code> if no such vertex
-	 *         exists
+	 * @param out
+	 *        the output stream to write the JAR files to
+	 * @throws IOException
+	 *         thrown if an error occurs while writing to the stream
 	 */
-	public AbstractJobVertex findVertexWithNullEdges() {
-
-		final AbstractJobVertex[] allVertices = getAllJobVertices();
+	private void writeRequiredJarFiles(DataOutputView out) throws IOException {
 
-		for (int i = 0; i < allVertices.length; i++) {
-
-			for (int j = 0; j < allVertices[i].getNumberOfForwardConnections(); j++) {
-				if (allVertices[i].getForwardConnection(j) == null) {
-					return allVertices[i];
-				}
-			}
+		// Now check if all the collected jar files really exist
+		final FileSystem fs = FileSystem.getLocalFileSystem();
 
-			for (int j = 0; j < allVertices[i].getNumberOfBackwardConnections(); j++) {
-				if (allVertices[i].getBackwardConnection(j) == null) {
-					return allVertices[i];
-				}
+		for (int i = 0; i < this.userJars.size(); i++) {
+			if (!fs.exists(this.userJars.get(i))) {
+				throw new IOException("Cannot find jar file " + this.userJars.get(i));
 			}
 		}
 
-		return null;
-	}
-
-	/**
-	 * Checks if the instance dependency chain created with the <code>setVertexToShareInstancesWith</code> method is
-	 * acyclic.
-	 * 
-	 * @return <code>true</code> if the dependency chain is acyclic, <code>false</code> otherwise
-	 */
-	public boolean isInstanceDependencyChainAcyclic() {
-
-		final AbstractJobVertex[] allVertices = this.getAllJobVertices();
-		final Set<AbstractJobVertex> alreadyVisited = new HashSet<AbstractJobVertex>();
-
-		for (AbstractJobVertex vertex : allVertices) {
+		// How many jar files follow?
+		out.writeInt(this.userJars.size());
 
-			if (alreadyVisited.contains(vertex)) {
-				continue;
-			}
+		for (int i = 0; i < this.userJars.size(); i++) {
 
-			AbstractJobVertex vertexToShareInstancesWith = vertex.getVertexToShareInstancesWith();
-			if (vertexToShareInstancesWith != null) {
+			final Path jar = this.userJars.get(i);
 
-				final Set<AbstractJobVertex> cycleMap = new HashSet<AbstractJobVertex>();
+			// Write out the actual path
+			jar.write(out);
 
-				while (vertexToShareInstancesWith != null) {
+			// Write out the length of the file
+			final FileStatus file = fs.getFileStatus(jar);
+			out.writeLong(file.getLen());
 
-					if (cycleMap.contains(vertexToShareInstancesWith)) {
-						return false;
-					} else {
-						alreadyVisited.add(vertexToShareInstancesWith);
-						cycleMap.add(vertexToShareInstancesWith);
-						vertexToShareInstancesWith = vertexToShareInstancesWith.getVertexToShareInstancesWith();
-					}
-				}
+			// Now write the jar file
+			final FSDataInputStream inStream = fs.open(this.userJars.get(i));
+			final byte[] buf = new byte[BUFFERSIZE];
+			int read = inStream.read(buf, 0, buf.length);
+			while (read > 0) {
+				out.write(buf, 0, read);
+				read = inStream.read(buf, 0, buf.length);
 			}
 		}
-
-		return true;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphDefinitionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphDefinitionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphDefinitionException.java
deleted file mode 100644
index b647c4e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraphDefinitionException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/**
- * 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.jobgraph;
-
-public class JobGraphDefinitionException extends Exception {
-
-	/**
-	 * 
-	 */
-	private static final long serialVersionUID = -5791426627063063469L;
-
-	public JobGraphDefinitionException(String msg) {
-		super(msg);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
index 0adbb11..60b2edf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
@@ -16,46 +16,28 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph;
 
 /**
- * Defines the possible status of a job once it has been
- * accepted by the job manager.
- * <p>
- * This class is thread-safe.
- * 
+ * Possible states of a job once it has been accepted by the job manager.
  */
 public enum JobStatus {
 
-	/**
-	 * All tasks of the job are in the execution state CREATED.
-	 */
+	/** Job is newly created, no task has started to run. */
 	CREATED,
 
-	/**
-	 * All tasks of the job have been accepted by the scheduler, resources have been requested.
-	 */
-	SCHEDULED,
-
-	/**
-	 * At least one task of the job is running, none has definitely failed.
-	 */
+	/** Some tasks are scheduled or running, some may be pending, some may be finished. */
 	RUNNING,
 
-	/**
-	 * At least one task of the job has definitively failed and cannot
-	 * be recovered anymore. As a result, the job has been terminated.
-	 */
+	/** The job has failed to to non-recoverable task failure */
 	FAILED,
 
-	/**
-	 * All tasks of the job are canceled as a result of a user request. The job has been terminated.
-	 */
+	/** Job is being cancelled */
+	CANCELLING,
+	
+	/** Job has been cancelled */
 	CANCELED,
 
-	/**
-	 * All of the job's tasks have successfully finished.
-	 */
+	/** All of the job's tasks have successfully finished. */
 	FINISHED
 };

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobTaskVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobTaskVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobTaskVertex.java
deleted file mode 100644
index b59766b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobTaskVertex.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * 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.jobgraph;
-
-/**
- * A JobTaskVertex is the vertex type for regular tasks (with both input and output) in Nephele.
- * Tasks running inside a JobTaskVertex must specify at least one record reader and one record writer.
- * 
- */
-public class JobTaskVertex extends AbstractJobVertex {
-
-	/**
-	 * Creates a new job task vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name for the new job task vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobTaskVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-	
-	public JobTaskVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-		jobGraph.addVertex(this);
-	}
-
-	/**
-	 * Creates a new job task vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobTaskVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
index cb63b06..d6d3e37 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
@@ -19,10 +19,16 @@
 package org.apache.flink.runtime.jobgraph;
 
 import org.apache.flink.runtime.AbstractID;
+import org.apache.flink.runtime.managementgraph.ManagementVertexID;
 
 /**
  * A class for statistically unique job vertex IDs.
  */
 public class JobVertexID extends AbstractID {
+	
 	private static final long serialVersionUID = 1L;
+	
+	public ManagementVertexID toManagementVertexId(int subtaskIndex) {
+		return new ManagementVertexID(subtaskIndex, getLowerPart() + getUpperPart());
+	}
 }


[32/63] [abbrv] git commit: Fix buffer leak in TaskManager / test tasks

Posted by se...@apache.org.
Fix buffer leak in TaskManager / test tasks


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/1fdd7e60
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/1fdd7e60
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/1fdd7e60

Branch: refs/heads/master
Commit: 1fdd7e6060351f4684ca665206c48c4410208225
Parents: ae57c7c
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 03:15:33 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../flink/runtime/io/network/ChannelManager.java       |  5 +++++
 .../flink/runtime/io/network/api/RecordWriter.java     | 11 +++++++++++
 .../io/network/bufferprovider/GlobalBufferPool.java    |  1 -
 .../flink/runtime/jobmanager/JobManagerITCase.java     | 13 +++++++++----
 4 files changed, 25 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1fdd7e60/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
index 602f88b..2d2afb2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
@@ -44,6 +44,7 @@ import org.apache.flink.runtime.io.network.gates.OutputGate;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.protocols.ChannelLookupProtocol;
 import org.apache.flink.runtime.taskmanager.Task;
+import org.apache.flink.util.ExceptionUtils;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -596,6 +597,10 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 		} catch (CancelTaskException e) {
 			releaseEnvelope(envelope);
 			throw e;
+		} catch (Throwable t) {
+			releaseEnvelope(envelope);
+			ExceptionUtils.rethrow(t, "Error while requesting receiver list.");
+			return null; // silence the compiler
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1fdd7e60/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/RecordWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/RecordWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/RecordWriter.java
index da3bfb3..82c78af 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/RecordWriter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/RecordWriter.java
@@ -138,4 +138,15 @@ public class RecordWriter<T extends IOReadableWritable> extends BufferWriter {
 			}
 		}
 	}
+	
+	public void clearBuffers() {
+		if (this.serializers != null) {
+			for (RecordSerializer<?> s: this.serializers) {
+				Buffer b = s.getCurrentBuffer();
+				if (b != null) {
+					b.recycleBuffer();
+				}
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1fdd7e60/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/bufferprovider/GlobalBufferPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/bufferprovider/GlobalBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/bufferprovider/GlobalBufferPool.java
index 844ca6e..db7499a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/bufferprovider/GlobalBufferPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/bufferprovider/GlobalBufferPool.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.network.bufferprovider;
 
 import java.util.Queue;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/1fdd7e60/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index f661ea0..a0224de 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -877,10 +877,15 @@ public class JobManagerITCase {
 
 		@Override
 		public void invoke() throws Exception {
-			writer.initializeSerializers();
-			writer.emit(new IntegerRecord(42));
-			writer.emit(new IntegerRecord(1337));
-			writer.flush();
+			try {
+				writer.initializeSerializers();
+				writer.emit(new IntegerRecord(42));
+				writer.emit(new IntegerRecord(1337));
+				writer.flush();
+			}
+			finally {
+				writer.clearBuffers();
+			}
 		}
 	}
 	


[36/63] [abbrv] Finalize ExecutionGraph state machine and calls

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java
new file mode 100644
index 0000000..d97abdb
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionStateProgressTest.java
@@ -0,0 +1,79 @@
+/**
+ * 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 static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+
+import java.util.Arrays;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.protocols.TaskOperationProtocol;
+import org.junit.Test;
+
+public class ExecutionStateProgressTest {
+
+	@Test
+	public void testAccumulatedStateFinished() {
+		try {
+			final JobID jid = new JobID();
+			final JobVertexID vid = new JobVertexID();
+			
+			AbstractJobVertex ajv = new AbstractJobVertex("TestVertex", vid);
+			ajv.setParallelism(3);
+			ajv.setInvokableClass(mock(AbstractInvokable.class).getClass());
+			
+			ExecutionGraph graph = new ExecutionGraph(jid, "test job", new Configuration());
+			graph.attachJobGraph(Arrays.asList(ajv));
+			
+			setGraphStatus(graph, JobStatus.RUNNING);
+			
+			ExecutionJobVertex ejv = graph.getJobVertex(vid);
+			
+			LibraryCacheManager.register(jid, new String[0]);
+			
+			// mock resources and mock taskmanager
+			TaskOperationProtocol taskManager = getSimpleAcknowledgingTaskmanager();
+			for (ExecutionVertex ee : ejv.getTaskVertices()) {
+				AllocatedSlot slot = getInstance(taskManager).allocateSlot(jid);
+				ee.deployToSlot(slot);
+			}
+			
+			// finish all
+			for (ExecutionVertex ee : ejv.getTaskVertices()) {
+				ee.executionFinished();
+			}
+			
+			assertTrue(ejv.isInFinalState());
+			assertEquals(JobStatus.FINISHED, graph.getState());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
index 299f7ba..ce1ab30 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
@@ -26,10 +26,10 @@ import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.times;
 
 import java.io.IOException;
-import java.util.NoSuchElementException;
 
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobID;
@@ -37,9 +37,9 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
-import org.apache.flink.util.LogUtils;
-import org.apache.log4j.Level;
+
 import org.junit.Test;
+
 import org.mockito.Matchers;
 
 public class ExecutionVertexCancelTest {
@@ -54,13 +54,19 @@ public class ExecutionVertexCancelTest {
 			final JobVertexID jid = new JobVertexID();
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 			
-			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 			
 			vertex.cancel();
 			
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -74,14 +80,20 @@ public class ExecutionVertexCancelTest {
 			final JobVertexID jid = new JobVertexID();
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 			
-			setVertexState(vertex, ExecutionState2.SCHEDULED);
-			assertEquals(ExecutionState2.SCHEDULED, vertex.getExecutionState());
+			setVertexState(vertex, ExecutionState.SCHEDULED);
+			assertEquals(ExecutionState.SCHEDULED, vertex.getExecutionState());
 			
 			vertex.cancel();
 			
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -97,51 +109,63 @@ public class ExecutionVertexCancelTest {
 			
 			final ExecutionJobVertex ejv = getJobVertexExecutingTriggered(jid, actions);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID execId = vertex.getCurrentExecutionAttempt().getAttemptId();
 			
-			setVertexState(vertex, ExecutionState2.SCHEDULED);
-			assertEquals(ExecutionState2.SCHEDULED, vertex.getExecutionState());
+			setVertexState(vertex, ExecutionState.SCHEDULED);
+			assertEquals(ExecutionState.SCHEDULED, vertex.getExecutionState());
 			
 			// task manager mock
 			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
-			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, true), new TaskOperationResult(jid, 0, false));
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(execId, true));
+			when(taskManager.cancelTask(execId)).thenReturn(new TaskOperationResult(execId, true), new TaskOperationResult(execId, false));
 			
 			Instance instance = getInstance(taskManager);
 			AllocatedSlot slot = instance.allocateSlot(new JobID());
 			
+			LibraryCacheManager.register(ejv.getJobId(), new String[0]);
 			vertex.deployToSlot(slot);
 			
-			assertEquals(ExecutionState2.DEPLOYING, vertex.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState());
 			
 			vertex.cancel();
 			 
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			
 			verify(taskManager, times(0)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
-			verify(taskManager, times(0)).cancelTask(jid, 0);
+			verify(taskManager, times(0)).cancelTask(execId);
 
 			// first action happens (deploy)
 			actions.triggerNextAction();
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			verify(taskManager, times(1)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
 			
 			// the deploy call found itself in canceling after it returned and needs to send a cancel call
 			// the call did not yet execute, so it is still in canceling
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			
 			// second action happens (cancel call from cancel function)
 			actions.triggerNextAction();
 			
+			// TaskManager reports back (canceling done)
+			vertex.getCurrentExecutionAttempt().cancelingComplete();
+			
 			// should properly set state to cancelled
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 			
 			// trigger the correction canceling call
 			actions.triggerNextAction();
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 			
-			verify(taskManager, times(2)).cancelTask(jid, 0);
+			verify(taskManager, times(2)).cancelTask(execId);
 			
 			assertTrue(slot.isReleased());
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -157,31 +181,33 @@ public class ExecutionVertexCancelTest {
 			
 			final ExecutionJobVertex ejv = getJobVertexExecutingTriggered(jid, actions);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID execId = vertex.getCurrentExecutionAttempt().getAttemptId();
 			
-			setVertexState(vertex, ExecutionState2.SCHEDULED);
-			assertEquals(ExecutionState2.SCHEDULED, vertex.getExecutionState());
+			setVertexState(vertex, ExecutionState.SCHEDULED);
+			assertEquals(ExecutionState.SCHEDULED, vertex.getExecutionState());
 			
 			// task manager mock
 			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(execId, true));
 			
 			// first return NOT SUCCESS (task not found, cancel call overtook deploy call), then success (cancel call after deploy call)
-			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, false), new TaskOperationResult(jid, 0, true));
+			when(taskManager.cancelTask(execId)).thenReturn(new TaskOperationResult(execId, false), new TaskOperationResult(execId, true));
 			
 			Instance instance = getInstance(taskManager);
 			AllocatedSlot slot = instance.allocateSlot(new JobID());
 			
+			LibraryCacheManager.register(ejv.getJobId(), new String[0]);
 			vertex.deployToSlot(slot);
 			
-			assertEquals(ExecutionState2.DEPLOYING, vertex.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState());
 			
 			vertex.cancel();
 			 
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			
 			verify(taskManager, times(0)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
-			verify(taskManager, times(0)).cancelTask(jid, 0);
+			verify(taskManager, times(0)).cancelTask(execId);
 
 			// first action happens (deploy)
 			Runnable deployAction = actions.popNextAction();
@@ -191,7 +217,7 @@ public class ExecutionVertexCancelTest {
 			cancelAction.run();
 			
 			// did not find the task, not properly cancelled, stay in canceling
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			
 			// deploy action next
 			deployAction.run();
@@ -200,15 +226,23 @@ public class ExecutionVertexCancelTest {
 			
 			// the deploy call found itself in canceling after it returned and needs to send a cancel call
 			// the call did not yet execute, so it is still in canceling
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			
 			// trigger the correcting cancel call, should properly set state to cancelled
 			actions.triggerNextAction();
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			vertex.getCurrentExecutionAttempt().cancelingComplete();
 			
-			verify(taskManager, times(2)).cancelTask(jid, 0);
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
+			
+			verify(taskManager, times(2)).cancelTask(execId);
 			
 			assertTrue(slot.isReleased());
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -222,26 +256,34 @@ public class ExecutionVertexCancelTest {
 			final JobVertexID jid = new JobVertexID();
 			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID execId = vertex.getCurrentExecutionAttempt().getAttemptId();
 
 			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, true));
+			when(taskManager.cancelTask(execId)).thenReturn(new TaskOperationResult(execId, true));
 			
 			Instance instance = getInstance(taskManager);
 			AllocatedSlot slot = instance.allocateSlot(new JobID());
 
-			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexState(vertex, ExecutionState.RUNNING);
 			setVertexResource(vertex, slot);
 			
-			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
 			
 			vertex.cancel();
+			vertex.getCurrentExecutionAttempt().cancelingComplete(); // responce by task manager once actially canceled
 			
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 			
-			verify(taskManager).cancelTask(jid, 0);
+			verify(taskManager).cancelTask(execId);
 			
 			assertTrue(slot.isReleased());
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -253,40 +295,45 @@ public class ExecutionVertexCancelTest {
 	public void testRepeatedCancelFromRunning() {
 		try {
 			final JobVertexID jid = new JobVertexID();
-			final ActionQueue actions = new ActionQueue();
-			final ExecutionJobVertex ejv = getJobVertexExecutingTriggered(jid, actions);
+			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID execId = vertex.getCurrentExecutionAttempt().getAttemptId();
 
 			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, true));
+			when(taskManager.cancelTask(execId)).thenReturn(new TaskOperationResult(execId, true));
 			
 			Instance instance = getInstance(taskManager);
 			AllocatedSlot slot = instance.allocateSlot(new JobID());
 
-			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexState(vertex, ExecutionState.RUNNING);
 			setVertexResource(vertex, slot);
 			
-			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
 			
 			vertex.cancel();
 			
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			
 			vertex.cancel();
 			
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
 			
-			actions.triggerNextAction();
+			// callback by TaskManager after canceling completes
+			vertex.getCurrentExecutionAttempt().cancelingComplete();
 			
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 			
-			try {
-				actions.triggerNextAction();
-				fail("Too many calls sent.");
-			} catch (NoSuchElementException e) {}
+			// check that we did not overdo our cancel calls
+			verify(taskManager, times(1)).cancelTask(execId);
 			
 			assertTrue(slot.isReleased());
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -302,22 +349,28 @@ public class ExecutionVertexCancelTest {
 			final JobVertexID jid = new JobVertexID();
 			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID execId = vertex.getCurrentExecutionAttempt().getAttemptId();
 
 			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, false));
+			when(taskManager.cancelTask(execId)).thenReturn(new TaskOperationResult(execId, false));
 			
 			Instance instance = getInstance(taskManager);
 			AllocatedSlot slot = instance.allocateSlot(new JobID());
 
-			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexState(vertex, ExecutionState.RUNNING);
 			setVertexResource(vertex, slot);
 			
-			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
 			
 			vertex.cancel();
 			
-			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -327,31 +380,36 @@ public class ExecutionVertexCancelTest {
 	
 	@Test
 	public void testCancelCallFails() {
-		// this may happen when the task finished or failed while the call was in progress
-		LogUtils.initializeDefaultConsoleLogger(Level.OFF);
 		
 		try {
 			final JobVertexID jid = new JobVertexID();
 			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID execId = vertex.getCurrentExecutionAttempt().getAttemptId();
 
 			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.cancelTask(jid, 0)).thenThrow(new IOException("RPC call failed"));
+			when(taskManager.cancelTask(execId)).thenThrow(new IOException("RPC call failed"));
 			
 			Instance instance = getInstance(taskManager);
 			AllocatedSlot slot = instance.allocateSlot(new JobID());
 
-			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexState(vertex, ExecutionState.RUNNING);
 			setVertexResource(vertex, slot);
 			
-			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
 			
 			vertex.cancel();
 			
-			assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
 			
 			assertTrue(slot.isReleased());
+			
+			assertNotNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CANCELING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -369,19 +427,19 @@ public class ExecutionVertexCancelTest {
 			final JobVertexID jid = new JobVertexID();
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
-			setVertexState(vertex, ExecutionState2.CANCELED);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			setVertexState(vertex, ExecutionState.CANCELED);
 			
-			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 			
 			// 1)
 			// scheduling after being created should be tolerated (no exception) because
 			// it can occur as the result of races
 			{
 				DefaultScheduler scheduler = mock(DefaultScheduler.class);
-				vertex.scheduleForExecution(scheduler);
+				vertex.scheduleForExecution(scheduler, false);
 				
-				assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+				assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 			}
 			
 			// 2)
@@ -396,7 +454,7 @@ public class ExecutionVertexCancelTest {
 				fail("Method should throw an exception");
 			}
 			catch (IllegalStateException e) {
-				assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+				assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
 			}
 		}
 		catch (Exception e) {
@@ -407,17 +465,18 @@ public class ExecutionVertexCancelTest {
 	
 	@Test
 	public void testActionsWhileCancelling() {
+		
 		try {
 			final JobVertexID jid = new JobVertexID();
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
 			
 			// scheduling while canceling is an illegal state transition
 			try {
-				ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
-				setVertexState(vertex, ExecutionState2.CANCELING);
+				ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+				setVertexState(vertex, ExecutionState.CANCELING);
 				
 				DefaultScheduler scheduler = mock(DefaultScheduler.class);
-				vertex.scheduleForExecution(scheduler);
+				vertex.scheduleForExecution(scheduler, false);
 				fail("Method should throw an exception");
 			}
 			catch (IllegalStateException e) {}
@@ -425,8 +484,8 @@ public class ExecutionVertexCancelTest {
 			
 			// deploying while in canceling state is illegal (should immediately go to canceled)
 			try {
-				ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
-				setVertexState(vertex, ExecutionState2.CANCELING);
+				ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+				setVertexState(vertex, ExecutionState.CANCELING);
 				
 				TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
 				Instance instance = getInstance(taskManager);
@@ -440,19 +499,19 @@ public class ExecutionVertexCancelTest {
 			
 			// fail while canceling
 			{
-				ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+				ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 				
 				TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
 				Instance instance = getInstance(taskManager);
 				AllocatedSlot slot = instance.allocateSlot(new JobID());
 				
 				setVertexResource(vertex, slot);
-				setVertexState(vertex, ExecutionState2.CANCELING);
+				setVertexState(vertex, ExecutionState.CANCELING);
 				
 				Exception failureCause = new Exception("test exception");
 				
 				vertex.fail(failureCause);
-				assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+				assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
 				assertEquals(failureCause, vertex.getFailureCause());
 				
 				assertTrue(slot.isReleased());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
index 5c61993..a1bb1cb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
@@ -19,24 +19,32 @@
 package org.apache.flink.runtime.executiongraph;
 
 import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*;
+
 import static org.junit.Assert.*;
+
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
+import static org.mockito.Matchers.any;
 
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
+
 import org.junit.Test;
+
 import org.mockito.Matchers;
 
 public class ExecutionVertexDeploymentTest {
 
+	
 	@Test
 	public void testDeployCall() {
 		try {
@@ -50,11 +58,11 @@ public class ExecutionVertexDeploymentTest {
 			
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 			
-			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 			vertex.deployToSlot(slot);
-			assertEquals(ExecutionState2.DEPLOYING, vertex.getExecutionState());
+			assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState());
 			
 			// no repeated scheduling
 			try {
@@ -62,6 +70,11 @@ public class ExecutionVertexDeploymentTest {
 				fail("Scheduled from wrong state");
 			}
 			catch (IllegalStateException e) {}
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -75,19 +88,22 @@ public class ExecutionVertexDeploymentTest {
 			final JobVertexID jid = new JobVertexID();
 			
 			// mock taskmanager to simply accept the call
-			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
-			
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
 			final Instance instance = getInstance(taskManager);
 			final AllocatedSlot slot = instance.allocateSlot(new JobID());
 			
 			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(vertex.getCurrentExecutionAttempt().getAttemptId(), true));
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 			
-			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			LibraryCacheManager.register(vertex.getJobId(), new String[0]);
 			vertex.deployToSlot(slot);
-			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
 			
 			// no repeated scheduling
 			try {
@@ -97,6 +113,12 @@ public class ExecutionVertexDeploymentTest {
 			catch (IllegalStateException e) {}
 			
 			verify(taskManager).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+			
+			assertNull(vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.RUNNING) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -111,16 +133,19 @@ public class ExecutionVertexDeploymentTest {
 			
 			// mock taskmanager to simply accept the call
 			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
+			
 			
 			final Instance instance = getInstance(taskManager);
 			final AllocatedSlot slot = instance.allocateSlot(new JobID());
 			
 			final ExecutionJobVertex ejv = getJobVertexExecutingAsynchronously(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(vertex.getCurrentExecutionAttempt().getAttemptId(), true));
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 			
-			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			LibraryCacheManager.register(vertex.getJobId(), new String[0]);
 			vertex.deployToSlot(slot);
 			
 			// no repeated scheduling
@@ -132,12 +157,12 @@ public class ExecutionVertexDeploymentTest {
 			
 			// wait until the state transition must be done
 			for (int i = 0; i < 100; i++) {
-				if (vertex.getExecutionState() != ExecutionState2.RUNNING) {
+				if (vertex.getExecutionState() != ExecutionState.RUNNING) {
 					Thread.sleep(10);
 				}
 			}
 			
-			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
 			
 			// no repeated scheduling
 			try {
@@ -147,6 +172,10 @@ public class ExecutionVertexDeploymentTest {
 			catch (IllegalStateException e) {}
 			
 			verify(taskManager).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.RUNNING) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -156,25 +185,34 @@ public class ExecutionVertexDeploymentTest {
 	
 	@Test
 	public void testDeployFailedSynchronous() {
+		final String ERROR_MESSAGE = "test_failure_error_message";
+		
 		try {
 			final JobVertexID jid = new JobVertexID();
 			
 			// mock taskmanager to simply accept the call
-			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, false, "failed"));
-			
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
 			final Instance instance = getInstance(taskManager);
 			final AllocatedSlot slot = instance.allocateSlot(new JobID());
 			
 			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(vertex.getCurrentExecutionAttempt().getAttemptId(), false, ERROR_MESSAGE));
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
 			
-			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			LibraryCacheManager.register(vertex.getJobId(), new String[0]);
 			vertex.deployToSlot(slot);
 			
-			assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
 			assertNotNull(vertex.getFailureCause());
+			assertTrue(vertex.getFailureCause().getMessage().contains(ERROR_MESSAGE));
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -184,32 +222,141 @@ public class ExecutionVertexDeploymentTest {
 	
 	@Test
 	public void testDeployFailedAsynchronously() {
+		final String ERROR_MESSAGE = "test_failure_error_message";
+		
 		try {
 			final JobVertexID jid = new JobVertexID();
 			
 			// mock taskmanager to simply accept the call
-			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
-			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, false, "failed"));
-			
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
 			final Instance instance = getInstance(taskManager);
 			final AllocatedSlot slot = instance.allocateSlot(new JobID());
 			
 			final ExecutionJobVertex ejv = getJobVertexExecutingAsynchronously(jid);
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 			
-			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(vertex.getCurrentExecutionAttempt().getAttemptId(), false, ERROR_MESSAGE));
 			
-			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
+			
+			LibraryCacheManager.register(vertex.getJobId(), new String[0]);
 			vertex.deployToSlot(slot);
 			
 			// wait until the state transition must be done
 			for (int i = 0; i < 100; i++) {
-				if (vertex.getExecutionState() != ExecutionState2.FAILED) {
+				if (vertex.getExecutionState() != ExecutionState.FAILED) {
 					Thread.sleep(10);
 				}
 			}
 			
-			assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
 			assertNotNull(vertex.getFailureCause());
+			assertTrue(vertex.getFailureCause().getMessage().contains(ERROR_MESSAGE));
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testFailExternallyDuringDeploy() {
+		
+		try {
+			final JobVertexID jid = new JobVertexID();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
+			
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
+			vertex.deployToSlot(slot);
+			assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState());
+			
+			Exception testError = new Exception("test error");
+			vertex.fail(testError);
+			
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
+			assertEquals(testError, vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testFailCallOvertakesDeploymentAnswer() {
+		
+		try {
+			final ActionQueue queue = new ActionQueue();
+			final JobVertexID jid = new JobVertexID();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexExecutingTriggered(jid, queue);
+			
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID eid = vertex.getCurrentExecutionAttempt().getAttemptId();
+			
+			// the deployment call succeeds regularly
+			when(taskManager.submitTask(any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(eid, true));
+			
+			// first cancel call does not find a task, second one finds it
+			when(taskManager.cancelTask(any(ExecutionAttemptID.class))).thenReturn(
+					new TaskOperationResult(eid, false), new TaskOperationResult(eid, true));
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
+			
+			LibraryCacheManager.register(vertex.getJobId(), new String[0]);
+			vertex.deployToSlot(slot);
+			assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState());
+			
+			Exception testError = new Exception("test error");
+			vertex.fail(testError);
+			
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
+			
+			// now the deploy call returns
+			Runnable deploy = queue.popNextAction();
+			Runnable cancel1 = queue.popNextAction();
+			
+			// cancel call overtakes
+			cancel1.run();
+			deploy.run();
+			
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
+			
+			// should have sent another cancel call
+			queue.triggerNextAction();
+			
+			assertEquals(testError, vertex.getFailureCause());
+			
+			assertTrue(vertex.getStateTimestamp(ExecutionState.CREATED) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.DEPLOYING) > 0);
+			assertTrue(vertex.getStateTimestamp(ExecutionState.FAILED) > 0);
+			
+			// should have received two cancel calls
+			verify(taskManager, times(2)).cancelTask(eid);
+			verify(taskManager, times(1)).submitTask(any(TaskDeploymentDescriptor.class));
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
new file mode 100644
index 0000000..43d6547
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
@@ -0,0 +1,141 @@
+/**
+ * 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 static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance;
+import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getJobVertexNotExecuting;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
+import org.apache.flink.runtime.protocols.TaskOperationProtocol;
+
+import org.junit.Test;
+
+import org.mockito.Matchers;
+
+public class ExecutionVertexSchedulingTest {
+	
+	@Test
+	public void testSlotReleasedWhenScheduledImmediately() {
+		
+		try {
+			// a slot than cannot be deployed to
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			slot.cancel();
+			assertFalse(slot.isReleased());
+			
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(new JobVertexID());
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			
+			DefaultScheduler scheduler = mock(DefaultScheduler.class);
+			when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot);
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
+			// try to deploy to the slot
+			vertex.scheduleForExecution(scheduler, false);
+			
+			// will have failed
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
+			assertTrue(slot.isReleased());
+			
+			verify(taskManager, times(0)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSlotReleasedWhenScheduledQueued() {
+
+		try {
+			// a slot than cannot be deployed to
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			slot.cancel();
+			assertFalse(slot.isReleased());
+			
+			final SlotAllocationFuture future = new SlotAllocationFuture();
+			
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(new JobVertexID());
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			
+			DefaultScheduler scheduler = mock(DefaultScheduler.class);
+			when(scheduler.scheduleQueued(Matchers.any(ScheduledUnit.class))).thenReturn(future);
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
+			// try to deploy to the slot
+			vertex.scheduleForExecution(scheduler, true);
+			
+			// future has not yet a slot
+			assertEquals(ExecutionState.SCHEDULED, vertex.getExecutionState());
+			
+			future.setSlot(slot);
+			
+			// will have failed
+			assertEquals(ExecutionState.FAILED, vertex.getExecutionState());
+			assertTrue(slot.isReleased());
+			
+			verify(taskManager, times(0)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testScheduleToDeploy() {
+		try {
+			// a slot than cannot be deployed to
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(new JobVertexID());
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			
+			DefaultScheduler scheduler = mock(DefaultScheduler.class);
+			when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot);
+			
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
+
+			// try to deploy to the slot
+			vertex.scheduleForExecution(scheduler, false);
+			assertEquals(ExecutionState.DEPLOYING, vertex.getExecutionState());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
index 0ea00d3..5a73253 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
@@ -66,10 +66,10 @@ public class PointwisePatternTest {
 		
 		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
 		
-		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+		for (ExecutionVertex ev : target.getTaskVertices()) {
 			assertEquals(1, ev.getNumberOfInputs());
 			
-			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			ExecutionEdge[] inEdges = ev.getInputEdges(0);
 			assertEquals(1, inEdges.length);
 			
 			assertEquals(ev.getParallelSubtaskIndex(), inEdges[0].getSource().getPartition());
@@ -101,10 +101,10 @@ public class PointwisePatternTest {
 		
 		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
 		
-		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+		for (ExecutionVertex ev : target.getTaskVertices()) {
 			assertEquals(1, ev.getNumberOfInputs());
 			
-			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			ExecutionEdge[] inEdges = ev.getInputEdges(0);
 			assertEquals(2, inEdges.length);
 			
 			assertEquals(ev.getParallelSubtaskIndex() * 2, inEdges[0].getSource().getPartition());
@@ -137,10 +137,10 @@ public class PointwisePatternTest {
 		
 		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
 		
-		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+		for (ExecutionVertex ev : target.getTaskVertices()) {
 			assertEquals(1, ev.getNumberOfInputs());
 			
-			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			ExecutionEdge[] inEdges = ev.getInputEdges(0);
 			assertEquals(3, inEdges.length);
 			
 			assertEquals(ev.getParallelSubtaskIndex() * 3, inEdges[0].getSource().getPartition());
@@ -174,10 +174,10 @@ public class PointwisePatternTest {
 		
 		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
 		
-		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+		for (ExecutionVertex ev : target.getTaskVertices()) {
 			assertEquals(1, ev.getNumberOfInputs());
 			
-			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			ExecutionEdge[] inEdges = ev.getInputEdges(0);
 			assertEquals(1, inEdges.length);
 			
 			assertEquals(ev.getParallelSubtaskIndex() / 2, inEdges[0].getSource().getPartition());
@@ -209,10 +209,10 @@ public class PointwisePatternTest {
 		
 		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
 		
-		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+		for (ExecutionVertex ev : target.getTaskVertices()) {
 			assertEquals(1, ev.getNumberOfInputs());
 			
-			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			ExecutionEdge[] inEdges = ev.getInputEdges(0);
 			assertEquals(1, inEdges.length);
 			
 			assertEquals(ev.getParallelSubtaskIndex() / 7, inEdges[0].getSource().getPartition());
@@ -266,10 +266,10 @@ public class PointwisePatternTest {
 		
 		int[] timesUsed = new int[lowDop];
 		
-		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+		for (ExecutionVertex ev : target.getTaskVertices()) {
 			assertEquals(1, ev.getNumberOfInputs());
 			
-			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			ExecutionEdge[] inEdges = ev.getInputEdges(0);
 			assertEquals(1, inEdges.length);
 			
 			
@@ -312,13 +312,13 @@ public class PointwisePatternTest {
 		
 		int[] timesUsed = new int[highDop];
 		
-		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+		for (ExecutionVertex ev : target.getTaskVertices()) {
 			assertEquals(1, ev.getNumberOfInputs());
 			
-			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			ExecutionEdge[] inEdges = ev.getInputEdges(0);
 			assertTrue(inEdges.length >= factor && inEdges.length <= factor + delta);
 			
-			for (ExecutionEdge2 ee : inEdges) {
+			for (ExecutionEdge ee : inEdges) {
 				timesUsed[ee.getSource().getPartition()]++;
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/fs/LineReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/fs/LineReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/fs/LineReaderTest.java
index f4dd731..03ea8df 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/fs/LineReaderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/fs/LineReaderTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.fs;
 
 import static org.junit.Assert.assertEquals;
@@ -29,7 +28,6 @@ import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.fs.local.LocalFileSystem;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java
index c10a5ef..67c0805 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.*;
 
 import java.net.InetAddress;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.junit.Test;
 import org.mockito.Matchers;
@@ -69,8 +69,8 @@ public class AllocatedSlotTest {
 	@Test
 	public void testSetExecutionVertex() {
 		try {
-			ExecutionVertex2 ev = mock(ExecutionVertex2.class);
-			ExecutionVertex2 ev_2 = mock(ExecutionVertex2.class);
+			Execution ev = mock(Execution.class);
+			Execution ev_2 = mock(Execution.class);
 			
 			// assign to alive slot
 			{
@@ -111,7 +111,7 @@ public class AllocatedSlotTest {
 	@Test
 	public void testReleaseCancelsVertex() {
 		try {
-			ExecutionVertex2 ev = mock(ExecutionVertex2.class);
+			Execution ev = mock(Execution.class);
 			
 			AllocatedSlot slot = getSlot();
 			assertTrue(slot.setExecutedVertex(ev));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
index 2583771..9a7622e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
@@ -37,7 +37,6 @@ import org.apache.flink.runtime.taskmanager.TaskManager;
 
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class LocalInstanceManagerTest {
@@ -152,7 +151,9 @@ public class LocalInstanceManagerTest {
 	private static final class MockRPC implements JobManagerProtocol {
 
 		@Override
-		public void updateTaskExecutionState(TaskExecutionState taskExecutionState) {}
+		public boolean updateTaskExecutionState(TaskExecutionState taskExecutionState) {
+			return false;
+		}
 
 		@Override
 		public boolean sendHeartbeat(InstanceID taskManagerId) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
index 8c36200..5837b81 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import java.io.EOFException;
@@ -27,9 +26,9 @@ import java.util.Random;
 
 import org.junit.Assert;
 
-import org.junit.BeforeClass;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -41,11 +40,8 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-
 /**
  * Integration test case for the I/O manager.
- *
- *
  */
 public class IOManagerITCase {
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index b8aac10..c7e3463 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -53,7 +53,7 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Test Job", vertex);
 			
-			JobManager jm = startJobManager();
+			JobManager jm = startJobManager(3);
 			try {
 				
 				// we need to register the job at the library cache manager (with no libraries)
@@ -104,11 +104,12 @@ public class JobManagerITCase {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	private static final JobManager startJobManager() throws Exception {
+	private static final JobManager startJobManager(int numSlots) throws Exception {
 		Configuration cfg = new Configuration();
 		cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
 		cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, getAvailablePort());
 		cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10);
+		cfg.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots);
 		
 		GlobalConfiguration.includeConfiguration(cfg);
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
deleted file mode 100644
index 835af95..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerTest.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * 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.jobmanager;
-
-public class JobManagerTest {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
index 10fca8a..b092312 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.jobmanager.scheduler;
 
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct;
-import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyVertex;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyTask;
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex;
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance;
 
@@ -120,17 +120,17 @@ public class SchedulerIsolatedTasksTest {
 			assertEquals(5, scheduler.getNumberOfAvailableSlots());
 			
 			// schedule something into all slots
-			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
-			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
-			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
-			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
-			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
 			
 			// the slots should all be different
 			assertTrue(areAllDistinct(s1, s2, s3, s4, s5));
 			
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+				scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
 				fail("Scheduler accepted scheduling request without available resource.");
 			}
 			catch (NoResourceAvailableException e) {
@@ -143,8 +143,8 @@ public class SchedulerIsolatedTasksTest {
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			
 			// now we can schedule some more slots
-			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
-			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
+			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
 			
 			assertTrue(areAllDistinct(s1, s2, s3, s4, s5, s6, s7));
 			
@@ -242,7 +242,7 @@ public class SchedulerIsolatedTasksTest {
 			disposeThread.start();
 			
 			for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) {
-				SlotAllocationFuture future = scheduler.scheduleQueued(new ScheduledUnit(getDummyVertex()));
+				SlotAllocationFuture future = scheduler.scheduleQueued(new ScheduledUnit(getDummyTask()));
 				future.setFutureAction(action);
 				allAllocatedSlots.add(future);
 			}
@@ -281,11 +281,11 @@ public class SchedulerIsolatedTasksTest {
 			scheduler.newInstanceAvailable(i3);
 			
 			List<AllocatedSlot> slots = new ArrayList<AllocatedSlot>();
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
-			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask())));
 			
 			i2.markDead();
 			
@@ -306,7 +306,7 @@ public class SchedulerIsolatedTasksTest {
 			
 			// cannot get another slot, since all instances are dead
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+				scheduler.scheduleImmediately(new ScheduledUnit(getDummyTask()));
 				fail("Scheduler served a slot from a dead instance");
 			}
 			catch (NoResourceAvailableException e) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
index 78a397e..d2e7598 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
@@ -28,7 +28,8 @@ import java.util.Collection;
 import java.util.HashSet;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.Execution;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
@@ -67,17 +68,32 @@ public class SchedulerTestUtils {
 	}
 	
 	
-	public static ExecutionVertex2 getDummyVertex() {
-		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
+	public static Execution getDummyTask() {
+		ExecutionVertex vertex = mock(ExecutionVertex.class);
+		when(vertex.getJobId()).thenReturn(new JobID());
+		when(vertex.toString()).thenReturn("TEST-VERTEX");
+		
+		Execution execution = mock(Execution.class);
+		when(execution.getVertex()).thenReturn(vertex);
+		
+		return execution;
+	}
+	
+	public static Execution getTestVertex(Iterable<Instance> preferredLocations) {
+		ExecutionVertex vertex = mock(ExecutionVertex.class);
 		
+		when(vertex.getPreferredLocations()).thenReturn(preferredLocations);
 		when(vertex.getJobId()).thenReturn(new JobID());
 		when(vertex.toString()).thenReturn("TEST-VERTEX");
 		
-		return vertex;
+		Execution execution = mock(Execution.class);
+		when(execution.getVertex()).thenReturn(vertex);
+		
+		return execution;
 	}
 	
-	public static ExecutionVertex2 getTestVertex(JobVertexID jid, int taskIndex, int numTasks) {
-		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
+	public static Execution getTestVertex(JobVertexID jid, int taskIndex, int numTasks) {
+		ExecutionVertex vertex = mock(ExecutionVertex.class);
 		
 		when(vertex.getPreferredLocations()).thenReturn(null);
 		when(vertex.getJobId()).thenReturn(new JobID());
@@ -86,11 +102,14 @@ public class SchedulerTestUtils {
 		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks);
 		when(vertex.toString()).thenReturn("TEST-VERTEX");
 		
-		return vertex;
+		Execution execution = mock(Execution.class);
+		when(execution.getVertex()).thenReturn(vertex);
+		
+		return execution;
 	}
 	
-	public static ExecutionVertex2 getTestVertexWithLocation(JobVertexID jid, int taskIndex, int numTasks, Instance... locations) {
-		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
+	public static Execution getTestVertexWithLocation(JobVertexID jid, int taskIndex, int numTasks, Instance... locations) {
+		ExecutionVertex vertex = mock(ExecutionVertex.class);
 		
 		when(vertex.getPreferredLocations()).thenReturn(Arrays.asList(locations));
 		when(vertex.getJobId()).thenReturn(new JobID());
@@ -99,17 +118,10 @@ public class SchedulerTestUtils {
 		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks);
 		when(vertex.toString()).thenReturn("TEST-VERTEX");
 		
-		return vertex;
-	}
-	
-	public static ExecutionVertex2 getTestVertex(Iterable<Instance> preferredLocations) {
-		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
-		
-		when(vertex.getPreferredLocations()).thenReturn(preferredLocations);
-		when(vertex.getJobId()).thenReturn(new JobID());
-		when(vertex.toString()).thenReturn("TEST-VERTEX");
+		Execution execution = mock(Execution.class);
+		when(execution.getVertex()).thenReturn(vertex);
 		
-		return vertex;
+		return execution;
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
index 7ae3d40..7a47510 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.hash;
 
 import static org.junit.Assert.assertEquals;
@@ -53,10 +52,10 @@ import org.apache.flink.types.Key;
 import org.apache.flink.types.NullKeyFieldException;
 import org.apache.flink.types.Record;
 import org.apache.flink.util.MutableObjectIterator;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class HashTableITCase {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
index 9a18742..666ecd5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
@@ -48,12 +48,11 @@ import org.apache.flink.types.IntValue;
 import org.apache.flink.types.Record;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
+
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
-
 public class CombiningUnilateralSortMergerITCase {
 	
 	private static final Logger LOG = LoggerFactory.getLogger(CombiningUnilateralSortMergerITCase.class);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
index 91dc5b3..6e5ab3e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
@@ -20,7 +20,9 @@ package org.apache.flink.runtime.taskmanager;
 
 import static org.junit.Assert.*;
 
-import org.apache.flink.runtime.execution.ExecutionState2;
+import java.io.IOException;
+
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
@@ -33,11 +35,11 @@ public class TaskExecutionStateTest {
 		try {
 			final JobID jid = new JobID();
 			final ExecutionAttemptID executionId = new ExecutionAttemptID();
-			final ExecutionState2 state = ExecutionState2.RUNNING;
-			final String description = "some test description";
+			final ExecutionState state = ExecutionState.RUNNING;
+			final Throwable error = new RuntimeException("some test error message");
 			
-			TaskExecutionState s1 = new TaskExecutionState(jid, executionId, state, description);
-			TaskExecutionState s2 = new TaskExecutionState(jid, executionId, state, description);
+			TaskExecutionState s1 = new TaskExecutionState(jid, executionId, state, error);
+			TaskExecutionState s2 = new TaskExecutionState(jid, executionId, state, error);
 			
 			assertEquals(s1.hashCode(), s2.hashCode());
 			assertEquals(s1, s2);
@@ -53,16 +55,23 @@ public class TaskExecutionStateTest {
 		try {
 			final JobID jid = new JobID();
 			final ExecutionAttemptID executionId = new ExecutionAttemptID();
-			final ExecutionState2 state = ExecutionState2.DEPLOYING;
-			final String description = "foo bar";
+			final ExecutionState state = ExecutionState.DEPLOYING;
+			final Throwable error = new IOException("fubar");
+			
+			TaskExecutionState original1 = new TaskExecutionState(jid, executionId, state, error);
+			TaskExecutionState original2 = new TaskExecutionState(jid, executionId, state);
+			
+			TaskExecutionState writableCopy1 = CommonTestUtils.createCopyWritable(original1);
+			TaskExecutionState writableCopy2 = CommonTestUtils.createCopyWritable(original2);
 			
-			TaskExecutionState original = new TaskExecutionState(jid, executionId, state, description);
+			TaskExecutionState javaSerCopy1 = CommonTestUtils.createCopySerializable(original1);
+			TaskExecutionState javaSerCopy2 = CommonTestUtils.createCopySerializable(original2);
 			
-			TaskExecutionState writableCopy = CommonTestUtils.createCopyWritable(original);
-			TaskExecutionState javaSerCopy = CommonTestUtils.createCopySerializable(original);
+			assertEquals(original1, writableCopy1);
+			assertEquals(original1, javaSerCopy1);
 			
-			assertEquals(original, writableCopy);
-			assertEquals(original, javaSerCopy);
+			assertEquals(original2, writableCopy2);
+			assertEquals(original2, javaSerCopy2);
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index 3eb4a61..13261d7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -33,7 +33,7 @@ import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.deployment.ChannelDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.instance.HardwareDescription;
@@ -42,6 +42,7 @@ import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
 import org.apache.flink.runtime.io.network.api.RecordReader;
 import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.io.network.bufferprovider.GlobalBufferPool;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -78,7 +79,6 @@ public class TaskManagerTest {
 			TaskOperationResult result = tm.submitTask(tdd);
 			assertTrue(result.isSuccess());
 			assertEquals(eid, result.getExecutionId());
-			assertEquals(vid, result.getVertexId());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -126,8 +126,6 @@ public class TaskManagerTest {
 			assertTrue(result2.isSuccess());
 			assertEquals(eid1, result1.getExecutionId());
 			assertEquals(eid2, result2.getExecutionId());
-			assertEquals(vid1, result1.getVertexId());
-			assertEquals(vid2, result2.getVertexId());
 			
 			Map<ExecutionAttemptID, Task> tasks = tm.getAllRunningTasks();
 			assertEquals(2, tasks.size());
@@ -137,24 +135,24 @@ public class TaskManagerTest {
 			assertNotNull(t1);
 			assertNotNull(t2);
 			
-			assertEquals(ExecutionState2.RUNNING, t1.getExecutionState());
-			assertEquals(ExecutionState2.RUNNING, t2.getExecutionState());
+			assertEquals(ExecutionState.RUNNING, t1.getExecutionState());
+			assertEquals(ExecutionState.RUNNING, t2.getExecutionState());
 			
 			// cancel one task
-			assertTrue(tm.cancelTask(vid1, 1, eid1).isSuccess());
+			assertTrue(tm.cancelTask(eid1).isSuccess());
 			t1.getEnvironment().getExecutingThread().join();
-			assertEquals(ExecutionState2.CANCELED, t1.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, t1.getExecutionState());
 			
 			tasks = tm.getAllRunningTasks();
 			assertEquals(1, tasks.size());
 			
 			// try to cancel a non existing task
-			assertFalse(tm.cancelTask(vid1, 1, eid1).isSuccess());
+			assertFalse(tm.cancelTask(eid1).isSuccess());
 			
 			// cancel the second task
-			assertTrue(tm.cancelTask(vid2, 2, eid2).isSuccess());
+			assertTrue(tm.cancelTask(eid2).isSuccess());
 			t2.getEnvironment().getExecutingThread().join();
-			assertEquals(ExecutionState2.CANCELED, t2.getExecutionState());
+			assertEquals(ExecutionState.CANCELED, t2.getExecutionState());
 			
 			tasks = tm.getAllRunningTasks();
 			assertEquals(0, tasks.size());
@@ -162,6 +160,8 @@ public class TaskManagerTest {
 			// the class loaders should be de-registered
 			assertNull(LibraryCacheManager.getClassLoader(jid1));
 			assertNull(LibraryCacheManager.getClassLoader(jid2));
+			
+			assertNetworkResourcesReleased(tm);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -205,6 +205,7 @@ public class TaskManagerTest {
 			
 			Map<ExecutionAttemptID, Task> tasks = tm.getAllRunningTasks();
 			assertEquals(0, tasks.size());
+			assertNetworkResourcesReleased(tm);
 			
 			// the class loaders should be de-registered
 			assertNull(LibraryCacheManager.getClassLoader(jid));
@@ -262,8 +263,6 @@ public class TaskManagerTest {
 			assertTrue(result2.isSuccess());
 			assertEquals(eid1, result1.getExecutionId());
 			assertEquals(eid2, result2.getExecutionId());
-			assertEquals(vid1, result1.getVertexId());
-			assertEquals(vid2, result2.getVertexId());
 			
 			Map<ExecutionAttemptID, Task> tasks = tm.getAllRunningTasks();
 			
@@ -278,14 +277,17 @@ public class TaskManagerTest {
 				t2.getEnvironment().getExecutingThread().join();
 			}
 			
-			assertEquals(ExecutionState2.FINISHED, t1.getExecutionState());
-			assertEquals(ExecutionState2.FINISHED, t2.getExecutionState());
+			assertEquals(ExecutionState.FINISHED, t1.getExecutionState());
+			assertEquals(ExecutionState.FINISHED, t2.getExecutionState());
 			
 			tasks = tm.getAllRunningTasks();
 			assertEquals(0, tasks.size());
 			
 			// the class loaders should be de-registered
 			assertNull(LibraryCacheManager.getClassLoader(jid));
+			
+			// make sure that the global buffer pool has all buffers back
+			assertNetworkResourcesReleased(tm);
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -293,9 +295,97 @@ public class TaskManagerTest {
 		}
 	}
 	
+	@Test
+	public void testCancellingDependentAndStateUpdateFails() {
+		
+		// this tests creates two tasks. the sender sends data, and fails to send the
+		// state update back to the job manager
+		// the second one blocks to be canceled
+		
+		try {
+			JobID jid = new JobID();
+			
+			JobVertexID vid1 = new JobVertexID();
+			JobVertexID vid2 = new JobVertexID();
+			
+			ExecutionAttemptID eid1 = new ExecutionAttemptID();
+			ExecutionAttemptID eid2 = new ExecutionAttemptID();
+			
+			ChannelID senderId = new ChannelID();
+			ChannelID receiverId = new ChannelID();
+			
+			JobManager jobManager = getJobManagerMockBase();
+			when(jobManager.updateTaskExecutionState(any(TaskExecutionState.class))).thenReturn(false);
+			when(jobManager.lookupConnectionInfo(Matchers.any(InstanceConnectionInfo.class), Matchers.eq(jid), Matchers.eq(senderId)))
+				.thenReturn(ConnectionInfoLookupResponse.createReceiverFoundAndReady(receiverId));
+			
+			TaskManager tm = createTaskManager(jobManager);
+			
+			ChannelDeploymentDescriptor cdd = new ChannelDeploymentDescriptor(senderId, receiverId);
+			
+			TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1,
+					new Configuration(), new Configuration(), Sender.class.getName(),
+					Collections.singletonList(new GateDeploymentDescriptor(Collections.singletonList(cdd))), 
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7,
+					new Configuration(), new Configuration(), ReceiverBlocking.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					Collections.singletonList(new GateDeploymentDescriptor(Collections.singletonList(cdd))),
+					new String[0], 0);
+			
+			// register the job twice (for two tasks) at the lib cache
+			LibraryCacheManager.register(jid, new String[0]);
+			LibraryCacheManager.register(jid, new String[0]);
+			assertNotNull(LibraryCacheManager.getClassLoader(jid));
+			
+			// deploy sender before receiver, so the target is online when the sender requests the connection info
+			TaskOperationResult result2 = tm.submitTask(tdd2);
+			TaskOperationResult result1 = tm.submitTask(tdd1);
+			
+			assertTrue(result1.isSuccess());
+			assertTrue(result2.isSuccess());
+			
+			Map<ExecutionAttemptID, Task> tasks = tm.getAllRunningTasks();
+			
+			Task t1 = tasks.get(eid1);
+			Task t2 = tasks.get(eid2);
+			
+			// cancel task 2. task one should either fail, or be done
+			tm.cancelTask(eid2);
+			
+			// wait until the task second task is canceled
+			if (t2 != null) {
+				t2.getEnvironment().getExecutingThread().join();
+			}
+			
+			if (t1 != null) {
+				if (t1.getExecutionState() == ExecutionState.RUNNING) {
+					tm.cancelTask(eid1);
+				}
+				t1.getEnvironment().getExecutingThread().join();
+			}
+			
+			// the task that failed to send the finished state 
+			assertEquals(0, tasks.size());
+			assertNetworkResourcesReleased(tm);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	
+	private static void assertNetworkResourcesReleased(TaskManager tm) {
+		GlobalBufferPool gbp = tm.getChannelManager().getGlobalBufferPool();
+		assertEquals(gbp.numBuffers(), gbp.numAvailableBuffers());
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	
-	public static JobManager getJobManagerMockBase() {
+	public static JobManager getJobManagerMockBase() throws Exception {
 		JobManager jm = mock(JobManager.class);
 		
 		final InstanceID iid = new InstanceID();
@@ -305,6 +395,8 @@ public class TaskManagerTest {
 		
 		when(jm.sendHeartbeat(iid)).thenReturn(true);
 		
+		when(jm.updateTaskExecutionState(any(TaskExecutionState.class))).thenReturn(true);
+		
 		return jm;
 	}
 	
@@ -382,4 +474,19 @@ public class TaskManagerTest {
 			}
 		}
 	}
+	
+	public static final class ReceiverBlocking extends AbstractInvokable {
+
+		@Override
+		public void registerInputOutput() {
+			new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			synchronized(this) {
+				wait();
+			}
+		}
+	}
 }


[27/63] [abbrv] git commit: [FLINK-1094] Reworked, improved, and testes split assigners

Posted by se...@apache.org.
[FLINK-1094] Reworked, improved, and testes split assigners


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/c32569ae
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/c32569ae
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/c32569ae

Branch: refs/heads/master
Commit: c32569aed12ffa968e2c2289c2d56db262c0eba4
Parents: 028fcf5
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jul 18 04:22:00 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../org/apache/flink/core/fs/BlockLocation.java |  10 +-
 .../flink/core/fs/FileChannelWrapper.java       | 239 ------------
 .../apache/flink/core/fs/FileInputSplit.java    | 148 +++-----
 .../apache/flink/core/io/GenericInputSplit.java |  39 +-
 .../org/apache/flink/core/io/InputSplit.java    |   5 +-
 .../flink/core/io/InputSplitAssigner.java       |  35 ++
 .../flink/core/io/LocatableInputSplit.java      |  96 +++--
 .../DefaultInputSplitAssigner.java              |  96 ++---
 .../splitassigner/InputSplitAssigner.java       |  59 ---
 .../splitassigner/InputSplitManager.java        | 266 -------------
 .../splitassigner/InputSplitTracker.java        | 166 --------
 .../splitassigner/InputSplitWrapper.java        |  20 +-
 .../LocatableInputSplitAssigner.java            | 225 +++++++----
 .../splitassigner/LocatableInputSplitList.java  | 211 -----------
 .../file/FileInputSplitAssigner.java            | 119 ------
 .../splitassigner/file/FileInputSplitList.java  | 209 ----------
 .../instance/DefaultInstanceManagerTest.java    |   4 -
 .../instance/LocalInstanceManagerTest.java      |   9 +-
 .../splitassigner/DefaultSplitAssignerTest.java | 121 ++++++
 .../LocatableSplitAssignerTest.java             | 379 +++++++++++++++++++
 20 files changed, 883 insertions(+), 1573 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java b/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java
index b557c36..14056ab 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/BlockLocation.java
@@ -16,22 +16,19 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.fs;
 
 import java.io.IOException;
 
 /**
- * A BlockLocation lists hosts, offset and length
- * of block.
- * 
+ * A BlockLocation lists hosts, offset and length of block.
  */
 public interface BlockLocation extends Comparable<BlockLocation> {
 
 	/**
 	 * Get the list of hosts (hostname) hosting this block.
 	 * 
-	 * @return a list of hosts (hostname) hosting this block
+	 * @return A list of hosts (hostname) hosting this block.
 	 * @throws IOException
 	 *         thrown if the list of hosts could not be retrieved
 	 */
@@ -40,7 +37,7 @@ public interface BlockLocation extends Comparable<BlockLocation> {
 	/**
 	 * Get the start offset of the file associated with this block.
 	 * 
-	 * @return the start offset of the file associated with this block
+	 * @return The start offset of the file associated with this block.
 	 */
 	long getOffset();
 
@@ -50,5 +47,4 @@ public interface BlockLocation extends Comparable<BlockLocation> {
 	 * @return the length of the block
 	 */
 	long getLength();
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-core/src/main/java/org/apache/flink/core/fs/FileChannelWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileChannelWrapper.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileChannelWrapper.java
deleted file mode 100644
index bf87642..0000000
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileChannelWrapper.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/**
- * 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.core.fs;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.MappedByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileLock;
-import java.nio.channels.ReadableByteChannel;
-import java.nio.channels.WritableByteChannel;
-
-public final class FileChannelWrapper extends FileChannel {
-
-	private final FileSystem fs;
-
-	private final Path checkpointFile;
-
-	private final byte[] buf;
-
-	private final short replication;
-
-	private FSDataOutputStream outputStream = null;
-
-	private FSDataInputStream inputStream = null;
-
-	private long nextExpectedWritePosition = 0L;
-
-	private long nextExpectedReadPosition = 0L;
-
-	public FileChannelWrapper(final FileSystem fs, final Path checkpointFile, final int bufferSize,
-			final short replication) {
-
-		this.fs = fs;
-		this.checkpointFile = checkpointFile;
-		this.buf = new byte[bufferSize];
-		this.replication = replication;
-	}
-
-
-	@Override
-	public void force(final boolean metaData) throws IOException {
-
-		throw new UnsupportedOperationException("Method force is not implemented");
-	}
-
-
-	@Override
-	public FileLock lock(final long position, final long size, final boolean shared) throws IOException {
-
-		throw new UnsupportedOperationException("Method lock is not implemented");
-	}
-
-
-	@Override
-	public MappedByteBuffer map(final MapMode mode, final long position, final long size) throws IOException {
-
-		throw new UnsupportedOperationException("Method map is not implemented");
-	}
-
-
-	@Override
-	public long position() throws IOException {
-
-		throw new UnsupportedOperationException("Method position is not implemented");
-	}
-
-
-	@Override
-	public FileChannel position(final long newPosition) throws IOException {
-
-		throw new UnsupportedOperationException("Method position is not implemented");
-	}
-
-
-	@Override
-	public int read(final ByteBuffer dst) throws IOException {
-
-		throw new UnsupportedOperationException("Method read is not implemented");
-	}
-
-
-	@Override
-	public int read(final ByteBuffer dst, final long position) throws IOException {
-
-		final int length = Math.min(this.buf.length, dst.remaining());
-
-		final FSDataInputStream inputStream = getInputStream();
-		if (position != this.nextExpectedReadPosition) {
-			System.out.println("Next expected position is " + this.nextExpectedReadPosition + ", seeking to "
-				+ position);
-			inputStream.seek(position);
-			this.nextExpectedReadPosition = position;
-		}
-
-		final int bytesRead = inputStream.read(this.buf, 0, length);
-		if (bytesRead == -1) {
-			return -1;
-		}
-
-		dst.put(this.buf, 0, length);
-
-		this.nextExpectedReadPosition += bytesRead;
-
-		return bytesRead;
-	}
-
-
-	@Override
-	public long read(final ByteBuffer[] dsts, final int offset, final int length) throws IOException {
-
-		throw new UnsupportedOperationException("Method read is not implemented");
-	}
-
-
-	@Override
-	public long size() throws IOException {
-
-		throw new UnsupportedOperationException("Method size is not implemented");
-	}
-
-
-	@Override
-	public long transferFrom(final ReadableByteChannel src, final long position, final long count) throws IOException {
-
-		throw new UnsupportedOperationException("Method transferFrom is not implemented");
-	}
-
-
-	@Override
-	public long transferTo(final long position, final long count, final WritableByteChannel target) throws IOException {
-
-		throw new UnsupportedOperationException("Method transferTo is not implemented");
-	}
-
-
-	@Override
-	public FileChannel truncate(final long size) throws IOException {
-
-		throw new UnsupportedOperationException("Method truncate is not implemented");
-	}
-
-
-	@Override
-	public FileLock tryLock(final long position, final long size, final boolean shared) throws IOException {
-
-		throw new UnsupportedOperationException("Method tryLock is not implemented");
-	}
-
-
-	@Override
-	public int write(final ByteBuffer src) throws IOException {
-
-		return write(src, this.nextExpectedWritePosition);
-	}
-
-
-	@Override
-	public int write(final ByteBuffer src, final long position) throws IOException {
-
-		if (position != this.nextExpectedWritePosition) {
-			throw new IOException("Next expected write position is " + this.nextExpectedWritePosition);
-		}
-
-		final FSDataOutputStream outputStream = getOutputStream();
-
-		int totalBytesWritten = 0;
-
-		while (src.hasRemaining()) {
-
-			final int length = Math.min(this.buf.length, src.remaining());
-			src.get(this.buf, 0, length);
-			outputStream.write(this.buf, 0, length);
-			totalBytesWritten += length;
-		}
-
-		this.nextExpectedWritePosition += totalBytesWritten;
-
-		return totalBytesWritten;
-	}
-
-
-	@Override
-	public long write(final ByteBuffer[] srcs, final int offset, final int length) throws IOException {
-
-		throw new UnsupportedOperationException("Method write is not implemented");
-	}
-
-	private FSDataOutputStream getOutputStream() throws IOException {
-
-		if (this.outputStream == null) {
-			this.outputStream = this.fs.create(this.checkpointFile, false, this.buf.length, this.replication,
-				this.fs.getDefaultBlockSize());
-		}
-
-		return this.outputStream;
-	}
-
-	private FSDataInputStream getInputStream() throws IOException {
-
-		if (this.inputStream == null) {
-			this.inputStream = this.fs.open(this.checkpointFile, this.buf.length);
-		}
-
-		return this.inputStream;
-	}
-
-
-	@Override
-	protected void implCloseChannel() throws IOException {
-
-		if (this.outputStream != null) {
-			this.outputStream.close();
-			this.outputStream = null;
-		}
-
-		if (this.inputStream != null) {
-			this.inputStream.close();
-			this.inputStream = null;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
index 40058c0..c4a69ba 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
@@ -16,28 +16,21 @@
  * limitations under the License.
  */
 
-
-/**
- * This file is based on source code from the Hadoop Project (http://hadoop.apache.org/), licensed by the Apache
- * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for
- * additional information regarding copyright ownership. 
- */
-
 package org.apache.flink.core.fs;
 
 import java.io.IOException;
 
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.io.StringRecord;
+import org.apache.flink.core.io.LocatableInputSplit;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
 /**
  * A file input split provides information on a particular part of a file, possibly
- * hosted on a distributed file system and replicated among several hosts.
- * 
+ * hosted on a distributed file system and replicated among several hosts. 
  */
-public class FileInputSplit implements InputSplit {
+public class FileInputSplit extends LocatableInputSplit {
+
+	private static final long serialVersionUID = 1L;
 
 	/**
 	 * The path of the file this file split refers to.
@@ -54,16 +47,8 @@ public class FileInputSplit implements InputSplit {
 	 */
 	private long length;
 
-	/**
-	 * List of hosts (hostnames) containing the block, possibly <code>null</code>.
-	 */
-	private String[] hosts;
-
-	/**
-	 * The logical number of the split.
-	 */
-	private int partitionNumber;
-
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Constructs a split with host information.
 	 * 
@@ -78,20 +63,21 @@ public class FileInputSplit implements InputSplit {
 	 * @param hosts
 	 *        the list of hosts containing the block, possibly <code>null</code>
 	 */
-	public FileInputSplit(final int num, final Path file, final long start, final long length, final String[] hosts) {
-		this.partitionNumber = num;
+	public FileInputSplit(int num, Path file, long start, long length, String[] hosts) {
+		super(num, hosts);
+		
 		this.file = file;
 		this.start = start;
 		this.length = length;
-		this.hosts = hosts;
 	}
 
 	/**
-	 * Constructor used to reconstruct the object at the receiver of an RPC call.
+	 * Default constructor for deserialization.
 	 */
-	public FileInputSplit() {
-	}
+	public FileInputSplit() {}
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Returns the path of the file containing this split's data.
 	 * 
@@ -118,35 +104,17 @@ public class FileInputSplit implements InputSplit {
 	public long getLength() {
 		return length;
 	}
-
-	/**
-	 * Gets the names of the hosts that this file split resides on.
-	 * 
-	 * @return The names of the hosts that this file split resides on.
-	 */
-	public String[] getHostNames() {
-		if (this.hosts == null) {
-			return new String[] {};
-		} else {
-			return this.hosts;
-		}
-	}
+	
+	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public int getSplitNumber() {
-		return this.partitionNumber;
-	}
-
-	@Override
-	public String toString() {
-		return "[" + this.partitionNumber + "] " + file + ":" + start + "+" + length;
-	}
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-		// write partition number
-		out.writeInt(this.partitionNumber);
+	public void write(DataOutputView out) throws IOException {
+		super.write(out);
 
+		// write start and length
+		out.writeLong(this.start);
+		out.writeLong(this.length);
+		
 		// write file
 		if (this.file != null) {
 			out.writeBoolean(true);
@@ -154,48 +122,56 @@ public class FileInputSplit implements InputSplit {
 		} else {
 			out.writeBoolean(false);
 		}
-
-		// write start and length
-		out.writeLong(this.start);
-		out.writeLong(this.length);
-
-		// write hosts
-		if (this.hosts == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			out.writeInt(this.hosts.length);
-			for (int i = 0; i < this.hosts.length; i++) {
-				StringRecord.writeString(out, this.hosts[i]);
-			}
-		}
 	}
 
-
 	@Override
-	public void read(final DataInputView in) throws IOException {
-		// read partition number
-		this.partitionNumber = in.readInt();
-
+	public void read(DataInputView in) throws IOException {
+		super.read(in);
+		
+		this.start = in.readLong();
+		this.length = in.readLong();
+		
 		// read file path
 		boolean isNotNull = in.readBoolean();
 		if (isNotNull) {
 			this.file = new Path();
 			this.file.read(in);
 		}
-
-		this.start = in.readLong();
-		this.length = in.readLong();
-
-		isNotNull = in.readBoolean();
-		if (isNotNull) {
-			final int numHosts = in.readInt();
-			this.hosts = new String[numHosts];
-			for (int i = 0; i < numHosts; i++) {
-				this.hosts[i] = StringRecord.readString(in);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public int hashCode() {
+		return getSplitNumber() ^ (file == null ? 0 : file.hashCode());
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		}
+		else if (obj != null && super.equals(obj) && obj instanceof FileInputSplit) {
+			FileInputSplit other = (FileInputSplit) obj;
+			
+			if (this.file != null) {
+				if (!this.file.equals(other.file)) {
+					return false;
+				}
 			}
-		} else {
-			this.hosts = null;
+			else if (other.file != null) {
+				return false;
+			}
+			
+			return this.start == other.start && this.length == other.length;
+		}
+		else {
+			return false;
 		}
 	}
+	
+	@Override
+	public String toString() {
+		return '[' + getSplitNumber() + "] " + file + ":" + start + "+" + length;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java b/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
index ceecfa5..850ba1c 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.io;
 
 import java.io.IOException;
@@ -27,17 +26,19 @@ import org.apache.flink.core.memory.DataOutputView;
 /**
  * A generic input split that has only a partition number.
  */
-public class GenericInputSplit implements InputSplit {
+public class GenericInputSplit implements InputSplit, java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
 
 	/**
 	 * The number of this split.
 	 */
-	protected int partitionNumber;
+	private int partitionNumber;
 
 	/**
 	 * The total number of partitions
 	 */
-	protected int totalNumberOfPartitions;
+	private int totalNumberOfPartitions;
 	
 	// --------------------------------------------------------------------------------------------
 
@@ -60,6 +61,17 @@ public class GenericInputSplit implements InputSplit {
 	// --------------------------------------------------------------------------------------------
 
 	@Override
+	public int getSplitNumber() {
+		return this.partitionNumber;
+	}
+	
+	public int getTotalNumberOfSplits() {
+		return this.totalNumberOfPartitions;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
 	public void write(DataOutputView out) throws IOException {
 		out.writeInt(this.partitionNumber);
 		out.writeInt(this.totalNumberOfPartitions);
@@ -70,16 +82,25 @@ public class GenericInputSplit implements InputSplit {
 		this.partitionNumber = in.readInt();
 		this.totalNumberOfPartitions = in.readInt();
 	}
+	
+	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public int getSplitNumber() {
-		return this.partitionNumber;
+	public int hashCode() {
+		return this.partitionNumber ^ this.totalNumberOfPartitions;
 	}
 	
-	public int getTotalNumberOfSplits() {
-		return this.totalNumberOfPartitions;
+	@Override
+	public boolean equals(Object obj) {
+		if (obj != null && obj instanceof GenericInputSplit) {
+			GenericInputSplit other = (GenericInputSplit) obj;
+			return this.partitionNumber == other.partitionNumber &&
+					this.totalNumberOfPartitions == other.totalNumberOfPartitions;
+		} else {
+			return false;
+		}
 	}
-
+	
 	public String toString() {
 		return "GenericSplit (" + this.partitionNumber + "/" + this.totalNumberOfPartitions + ")";
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-core/src/main/java/org/apache/flink/core/io/InputSplit.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/InputSplit.java b/flink-core/src/main/java/org/apache/flink/core/io/InputSplit.java
index 5f09d10..5ce0378 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/InputSplit.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/InputSplit.java
@@ -16,14 +16,13 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.io;
 
-
 /**
  * This interface must be implemented by all kind of input splits that can be assigned to input formats.
  */
-public interface InputSplit extends IOReadableWritable {
+public interface InputSplit extends IOReadableWritable, java.io.Serializable {
+	
 	/**
 	 * Returns the number of this input split.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-core/src/main/java/org/apache/flink/core/io/InputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/InputSplitAssigner.java b/flink-core/src/main/java/org/apache/flink/core/io/InputSplitAssigner.java
new file mode 100644
index 0000000..f01c9f3
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/io/InputSplitAssigner.java
@@ -0,0 +1,35 @@
+/**
+ * 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.core.io;
+
+/**
+ * An input split assigner distributes the {@link InputSplit}s among the instances on which a
+ * data source exists.
+ */
+public interface InputSplitAssigner {
+
+	/**
+	 * Returns the next input split that shall be consumed. The consumer's host is passed as a parameter
+	 * to allow localized assignments.
+	 * 
+	 * @param host The address of the host to assign the split to.
+	 * @return the next input split to be consumed, or <code>null</code> if no more splits remain.
+	 */
+	InputSplit getNextInputSplit(String host);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-core/src/main/java/org/apache/flink/core/io/LocatableInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/LocatableInputSplit.java b/flink-core/src/main/java/org/apache/flink/core/io/LocatableInputSplit.java
index 9a4e366..cc36d99 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/LocatableInputSplit.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/LocatableInputSplit.java
@@ -16,20 +16,23 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.io;
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
-
 /**
  * A locatable input split is an input split referring to input data which is located on one or more hosts.
  */
-public class LocatableInputSplit implements InputSplit {
+public class LocatableInputSplit implements InputSplit, java.io.Serializable {
+	
+	private static final long serialVersionUID = 1L;
 
+	private static final String[] EMPTY_ARR = new String[0];
+	
 	/**
 	 * The number of the split.
 	 */
@@ -40,6 +43,8 @@ public class LocatableInputSplit implements InputSplit {
 	 */
 	private String[] hostnames;
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Creates a new locatable input split.
 	 * 
@@ -48,73 +53,86 @@ public class LocatableInputSplit implements InputSplit {
 	 * @param hostnames
 	 *        the names of the hosts storing the data this input split refers to
 	 */
-	public LocatableInputSplit(final int splitNumber, final String[] hostnames) {
-
-		this.hostnames = hostnames;
+	public LocatableInputSplit(int splitNumber, String[] hostnames) {
+		this.splitNumber = splitNumber;
+		this.hostnames = hostnames == null ? EMPTY_ARR : hostnames;
+	}
+	
+	public LocatableInputSplit(int splitNumber, String hostname) {
+		this.splitNumber = splitNumber;
+		this.hostnames = hostname == null ? EMPTY_ARR : new String[] { hostname };
 	}
 
 	/**
 	 * Default constructor for serialization/deserialization.
 	 */
-	public LocatableInputSplit() {
-	}
+	public LocatableInputSplit() {}
 
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public int getSplitNumber() {
+		return this.splitNumber;
+	}
+	
 	/**
 	 * Returns the names of the hosts storing the data this input split refers to
 	 * 
 	 * @return the names of the hosts storing the data this input split refers to
 	 */
 	public String[] getHostnames() {
-
-		if (this.hostnames == null) {
-			return new String[] {};
-		}
-
 		return this.hostnames;
 	}
 
+	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public void write(final DataOutputView out) throws IOException {
-
-		// Write the split number
+	public void write(DataOutputView out) throws IOException {
 		out.writeInt(this.splitNumber);
-
-		// Write hostnames
-		if (this.hostnames == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			out.writeInt(this.hostnames.length);
-			for (int i = 0; i < this.hostnames.length; i++) {
-				StringRecord.writeString(out, this.hostnames[i]);
-			}
+		out.writeInt(this.hostnames.length);
+		for (int i = 0; i < this.hostnames.length; i++) {
+			StringRecord.writeString(out, this.hostnames[i]);
 		}
 	}
 
-
 	@Override
-	public void read(final DataInputView in) throws IOException {
-
-		// Read the split number
+	public void read(DataInputView in) throws IOException {
 		this.splitNumber = in.readInt();
 
-		// Read hostnames
-		if (in.readBoolean()) {
-			final int numHosts = in.readInt();
+		final int numHosts = in.readInt();
+		if (numHosts == 0) {
+			this.hostnames = EMPTY_ARR;
+		} else {
 			this.hostnames = new String[numHosts];
 			for (int i = 0; i < numHosts; i++) {
 				this.hostnames[i] = StringRecord.readString(in);
 			}
-		} else {
-			this.hostnames = null;
 		}
 	}
-
-
+	
+	// --------------------------------------------------------------------------------------------
+	
 	@Override
-	public int getSplitNumber() {
-
+	public int hashCode() {
 		return this.splitNumber;
 	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj == this) {
+			return true;
+		}
+		else if (obj != null && obj instanceof LocatableInputSplit) {
+			LocatableInputSplit other = (LocatableInputSplit) obj;
+			return other.splitNumber == this.splitNumber && Arrays.deepEquals(other.hostnames, this.hostnames);
+		}
+		else {
+			return false;
+		}
+	}
+	
+	@Override
+	public String toString() {
+		return "Locatable Split (" + splitNumber + ") at " + Arrays.toString(this.hostnames);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java
index 916772a..44dae64 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java
@@ -16,88 +16,60 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobmanager.splitassigner;
 
-import java.util.Arrays;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.ConcurrentMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.core.io.InputSplitAssigner;
 
 /**
  * This is the default implementation of the {@link InputSplitAssigner} interface. The default input split assigner
- * simply returns all input splits of an input vertex in the order they were originally computed. The default input
- * split assigner is always used when a more specific {@link InputSplitAssigned} could not be found.
- * <p>
- * This class is thread-safe.
- * 
+ * simply returns all input splits of an input vertex in the order they were originally computed.
  */
 public class DefaultInputSplitAssigner implements InputSplitAssigner {
 
-	/**
-	 * The logging object used to report information and errors.
-	 */
+	/** The logging object used to report information and errors. */
 	private static final Logger LOG = LoggerFactory.getLogger(DefaultInputSplitAssigner.class);
 
-	/**
-	 * The split map stores a list of all input splits that still must be consumed by a specific input vertex.
-	 */
-	private final ConcurrentMap<ExecutionGroupVertex, Queue<InputSplit>> splitMap = new ConcurrentHashMap<ExecutionGroupVertex, Queue<InputSplit>>();
-
-
-	@Override
-	public void registerGroupVertex(final ExecutionGroupVertex groupVertex) {
-
-		final InputSplit[] inputSplits = groupVertex.getInputSplits();
+	/** The list of all splits */
+	private final List<InputSplit> splits = new ArrayList<InputSplit>();
 
-		if (inputSplits == null) {
-			return;
-		}
-
-		if (inputSplits.length == 0) {
-			return;
-		}
-
-		final Queue<InputSplit> queue = new ConcurrentLinkedQueue<InputSplit>();
-		if (this.splitMap.putIfAbsent(groupVertex, queue) != null) {
-			LOG.error("Group vertex " + groupVertex.getName() + " already has a split queue");
-		}
 
-		queue.addAll(Arrays.asList(inputSplits));
+	public DefaultInputSplitAssigner(InputSplit[] splits) {
+		Collections.addAll(this.splits, splits);
 	}
-
-
-	@Override
-	public void unregisterGroupVertex(final ExecutionGroupVertex groupVertex) {
-
-		this.splitMap.remove(groupVertex);
+	
+	public DefaultInputSplitAssigner(Collection<? extends InputSplit> splits) {
+		this.splits.addAll(splits);
 	}
-
-
+	
+	
 	@Override
-	public InputSplit getNextInputSplit(final ExecutionVertex vertex) {
-
-		final Queue<InputSplit> queue = this.splitMap.get(vertex.getGroupVertex());
-		if (queue == null) {
-			final JobID jobID = vertex.getExecutionGraph().getJobID();
-			LOG.error("Cannot find split queue for vertex " + vertex.getGroupVertex() + " (job " + jobID + ")");
-			return null;
+	public InputSplit getNextInputSplit(String host) {
+		InputSplit next = null;
+		
+		// keep the synchronized part short
+		synchronized (this.splits) {
+			if (this.splits.size() > 0) {
+				next = this.splits.remove(this.splits.size() - 1);
+			}
 		}
-
-		InputSplit nextSplit = queue.poll();
-
-		if (LOG.isDebugEnabled() && nextSplit != null) {
-			LOG.debug("Assigning split " + nextSplit.getSplitNumber() + " to " + vertex);
+		
+		if (LOG.isDebugEnabled()) {
+			if (next == null) {
+				LOG.debug("Assigning split " + next + " to " + host);
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("No more input splits available");
+				}
+			}
 		}
-
-		return nextSplit;
+		return next;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitAssigner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitAssigner.java
deleted file mode 100644
index 70e7ef9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitAssigner.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-
-/**
- * The input split assigner interface must be implemented by every component which is supposed to dynamically offer
- * input splits to the input vertices of a job at runtime.
- * 
- */
-public interface InputSplitAssigner {
-
-	/**
-	 * Registers an input vertex with the input split assigner.
-	 * 
-	 * @param groupVertex
-	 *        the input vertex to register
-	 */
-	void registerGroupVertex(ExecutionGroupVertex groupVertex);
-
-	/**
-	 * Unregisters an input vertex from the input split assigner. All resources allocated to the input vertex are freed
-	 * as part of this operation.
-	 * 
-	 * @param groupVertex
-	 *        the input vertex to unregister
-	 */
-	void unregisterGroupVertex(ExecutionGroupVertex groupVertex);
-
-	/**
-	 * Returns the next input split that shall be consumed by the given input vertex.
-	 * 
-	 * @param vertex
-	 *        the vertex for which the next input split to be consumed shall be determined
-	 * @return the next input split to be consumed or <code>null</code> if no more splits shall be consumed by the given
-	 *         vertex
-	 */
-	InputSplit getNextInputSplit(ExecutionVertex vertex);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitManager.java
deleted file mode 100644
index d9de4ee..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitManager.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertexIterator;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobmanager.splitassigner.file.FileInputSplitAssigner;
-import org.apache.flink.util.StringUtils;
-
-/**
- * The input split manager is responsible for serving input splits to {@link AbstractInputTask} objects at runtime.
- * Before passed on to the {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler}, an {@link ExecutionGraph} is registered with the input split
- * manager and all included input vertices of the graph register their generated input splits with the manager. Each
- * type of input split can be assigned to a specific {@link InputSplitAssigner} which is loaded by the input split
- * manager at runtime.
- * <p>
- * This class is thread-safe.
- */
-public final class InputSplitManager {
-
-	/**
-	 * The logging object which is used to report information and errors.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(InputSplitManager.class);
-
-	/**
-	 * The prefix of the configuration key which is used to retrieve the class names of the individual
-	 * {@link InputSplitAssigner} classes
-	 */
-	private static final String INPUT_SPLIT_CONFIG_KEY_PREFIX = "inputsplit.assigner.";
-
-	/**
-	 * A cache which stores the mapping of group vertices to assigner objects for fast retrieval during the job
-	 * execution.
-	 */
-	private final Map<ExecutionGroupVertex, InputSplitAssigner> assignerCache = new ConcurrentHashMap<ExecutionGroupVertex, InputSplitAssigner>();
-
-	/**
-	 * A map holding an instance of each available {@link InputSplitAssigner}, accessible via the class name of the
-	 * corresponding split type.
-	 */
-	private final Map<Class<? extends InputSplit>, InputSplitAssigner> loadedAssigners = new HashMap<Class<? extends InputSplit>, InputSplitAssigner>();
-
-	/**
-	 * The input split tracker makes sure that a vertex retrieves the same sequence of input splits after being
-	 * restarted.
-	 */
-	private final InputSplitTracker inputSplitTracker = new InputSplitTracker();
-
-	/**
-	 * The default input split assigner which is always used if a more specific assigner cannot be found.
-	 */
-	private final InputSplitAssigner defaultAssigner = new DefaultInputSplitAssigner();
-
-	/**
-	 * Registers a new job represented by its {@link ExecutionGraph} with the input split manager.
-	 * 
-	 * @param executionGraph
-	 *        the job to be registered
-	 */
-	public void registerJob(final ExecutionGraph executionGraph) {
-
-		final Iterator<ExecutionGroupVertex> it = new ExecutionGroupVertexIterator(executionGraph, true, -1);
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			final InputSplit[] inputSplits = groupVertex.getInputSplits();
-
-			if (inputSplits == null) {
-				continue;
-			}
-
-			if (inputSplits.length == 0) {
-				continue;
-			}
-
-			final InputSplitAssigner assigner = getAssignerByType(groupVertex.getInputSplitType(), true);
-			// Add entry to cache for fast retrieval during the job execution
-			this.assignerCache.put(groupVertex, assigner);
-
-			assigner.registerGroupVertex(groupVertex);
-		}
-
-		// Register job with the input split tracker
-		this.inputSplitTracker.registerJob(executionGraph);
-	}
-
-	/**
-	 * Unregisters the given job represented by its {@link ExecutionGraph} with the input split manager.
-	 * 
-	 * @param executionGraph
-	 *        the job to be unregistered
-	 */
-	public void unregisterJob(final ExecutionGraph executionGraph) {
-
-		final Iterator<ExecutionGroupVertex> it = new ExecutionGroupVertexIterator(executionGraph, true, -1);
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			final InputSplit[] inputSplits = groupVertex.getInputSplits();
-
-			if (inputSplits == null) {
-				continue;
-			}
-
-			if (inputSplits.length == 0) {
-				continue;
-			}
-
-			final InputSplitAssigner assigner = this.assignerCache.remove(groupVertex);
-			if (assigner == null) {
-				LOG.error("Group vertex " + groupVertex.getName()
-					+ " is unregistered, but cannot be found in assigner cache");
-				continue;
-			}
-
-			assigner.unregisterGroupVertex(groupVertex);
-		}
-
-		// Unregister job from input split tracker
-		this.inputSplitTracker.unregisterJob(executionGraph);
-	}
-
-	/**
-	 * Returns the next input split the input split manager (or the responsible {@link InputSplitAssigner} to be more
-	 * precise) has chosen for the given vertex to consume.
-	 * 
-	 * @param vertex
-	 *        the vertex for which the next input split is to be determined
-	 * @param sequenceNumber
-	 *        the sequence number of the vertex's request
-	 * @return the next input split to consume or <code>null</code> if the vertex shall consume no more input splits
-	 */
-	public InputSplit getNextInputSplit(final ExecutionVertex vertex, final int sequenceNumber) {
-
-		InputSplit nextInputSplit = this.inputSplitTracker.getInputSplitFromLog(vertex, sequenceNumber);
-		if (nextInputSplit != null) {
-			LOG.info("Input split " + nextInputSplit.getSplitNumber() + " for vertex " + vertex + " replayed from log");
-			return nextInputSplit;
-		}
-
-		final ExecutionGroupVertex groupVertex = vertex.getGroupVertex();
-		final InputSplitAssigner inputSplitAssigner = this.assignerCache.get(groupVertex);
-		if (inputSplitAssigner == null) {
-			final JobID jobID = groupVertex.getExecutionStage().getExecutionGraph().getJobID();
-			LOG.error("Cannot find input assigner for group vertex " + groupVertex.getName() + " (job " + jobID + ")");
-			return null;
-		}
-
-		nextInputSplit = inputSplitAssigner.getNextInputSplit(vertex);
-		if (nextInputSplit != null) {
-			this.inputSplitTracker.addInputSplitToLog(vertex, sequenceNumber, nextInputSplit);
-			LOG.info(vertex + " receives input split " + nextInputSplit.getSplitNumber());
-		}
-
-		return nextInputSplit;
-	}
-
-	/**
-	 * Returns the {@link InputSplitAssigner} which is defined for the given type of input split.
-	 * 
-	 * @param inputSplitType
-	 *        the type of input split to find the corresponding {@link InputSplitAssigner} for
-	 * @param allowLoading
-	 *        <code>true</code> to indicate that the input split assigner is allowed to load additional classes if
-	 *        necessary, <code>false</code> otherwise
-	 * @return the {@link InputSplitAssigner} responsible for the given type of input split
-	 */
-	private InputSplitAssigner getAssignerByType(final Class<? extends InputSplit> inputSplitType,
-			final boolean allowLoading) {
-
-		synchronized (this.loadedAssigners) {
-
-			InputSplitAssigner assigner = this.loadedAssigners.get(inputSplitType);
-			if (assigner == null && allowLoading) {
-				assigner = loadInputSplitAssigner(inputSplitType);
-				if (assigner != null) {
-					this.loadedAssigners.put(inputSplitType, assigner);
-				}
-			}
-
-			if (assigner != null) {
-				return assigner;
-			}
-		}
-
-		LOG.warn("Unable to find specific input split provider for type " + inputSplitType.getName()
-			+ ", using default assigner");
-
-		return this.defaultAssigner;
-	}
-
-	/**
-	 * Attempts to find the responsible type of {@link InputSplitAssigner} for the given type of input split from the
-	 * configuration and instantiate an object for it.
-	 * 
-	 * @param inputSplitType
-	 *        the type of input split to load the {@link InputSplitAssigner} for
-	 * @return the newly loaded {@link InputSplitAssigner} object or <code>null</code> if no such object could be
-	 *         located or loaded
-	 */
-	private InputSplitAssigner loadInputSplitAssigner(final Class<? extends InputSplit> inputSplitType) {
-
-		final String className = inputSplitType.getName();
-		
-		final String assignerKey = INPUT_SPLIT_CONFIG_KEY_PREFIX + className;
-		LOG.info("Trying to load input split assigner for type " + className);
-
-		String assignerClassName = GlobalConfiguration.getString(assignerKey, null);
-
-		// Provide hard-wired default configuration for FileInputSplit objects to make configuration more robust
-		if (assignerClassName == null) {
-			if (FileInputSplit.class == inputSplitType) {
-				return new FileInputSplitAssigner();
-			}
-			else if (GenericInputSplit.class == inputSplitType) {
-				return new DefaultInputSplitAssigner();
-			}
-			else {
-				return null;
-			}
-		}
-
-		try {
-			final Class<? extends InputSplitAssigner> assignerClass =
-					Class.forName(assignerClassName).asSubclass(InputSplitAssigner.class);
-			return assignerClass.newInstance();
-		}
-		catch (Exception e) {
-			LOG.error(StringUtils.stringifyException(e));
-		}
-
-		return null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitTracker.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitTracker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitTracker.java
deleted file mode 100644
index 013fbec..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitTracker.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionGraphIterator;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertexIterator;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-
-/**
- * The input split tracker maintains a log of all the input splits that are handed out to the individual input vertices.
- * In case of an input vertex must be restarted the input split tracker makes sure that the vertex receives the same
- * sequence of input splits as in its original run up to the point that it crashed.
- * <p>
- * This class is thread-safe.
- * 
- */
-final class InputSplitTracker {
-
-	/**
-	 * The logging object which is used to report information and errors.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(InputSplitTracker.class);
-
-	/**
-	 * The central split map which stores the logs of the individual input vertices.
-	 */
-	private final ConcurrentMap<ExecutionVertexID, List<InputSplit>> splitMap = new ConcurrentHashMap<ExecutionVertexID, List<InputSplit>>();
-
-	/**
-	 * Constructor with package visibility only.
-	 */
-	InputSplitTracker() {
-	}
-
-	/**
-	 * Registers a new job with the input split tracker.
-	 * 
-	 * @param eg
-	 *        the execution graph of the job to be registered
-	 */
-	void registerJob(final ExecutionGraph eg) {
-
-		final Iterator<ExecutionGroupVertex> it = new ExecutionGroupVertexIterator(eg, true, -1);
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			final InputSplit[] inputSplits = groupVertex.getInputSplits();
-
-			if (inputSplits == null) {
-				continue;
-			}
-
-			if (inputSplits.length == 0) {
-				continue;
-			}
-
-			for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); ++i) {
-				final ExecutionVertex vertex = groupVertex.getGroupMember(i);
-				if (this.splitMap.put(vertex.getID(), new ArrayList<InputSplit>()) != null) {
-					LOG.error("InputSplitTracker must keep track of two vertices with ID " + vertex.getID());
-				}
-			}
-		}
-	}
-
-	/**
-	 * Unregisters a job from the input split tracker.
-	 * 
-	 * @param eg
-	 *        the execution graph of the job to be unregistered
-	 */
-	void unregisterJob(final ExecutionGraph eg) {
-
-		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(eg, true);
-		while (it.hasNext()) {
-			this.splitMap.remove(it.next().getID());
-		}
-	}
-
-	/**
-	 * Returns the input split with the given sequence number from the specified vertex's log or <code>null</code> if no
-	 * such input split exists.
-	 * 
-	 * @param vertex
-	 *        the vertex for which the input split shall be returned from the log
-	 * @param sequenceNumber
-	 *        the sequence number identifying the log entry
-	 * @return the input split that was stored under the given sequence number of the vertex's log or <code>null</code>
-	 *         if no such input split exists
-	 */
-	InputSplit getInputSplitFromLog(final ExecutionVertex vertex, final int sequenceNumber) {
-
-		final List<InputSplit> inputSplitLog = this.splitMap.get(vertex.getID());
-		if (inputSplitLog == null) {
-			LOG.error("Cannot find input split log for vertex " + vertex + " (" + vertex.getID() + ")");
-			return null;
-		}
-
-		synchronized (inputSplitLog) {
-
-			if (sequenceNumber < inputSplitLog.size()) {
-				return inputSplitLog.get(sequenceNumber);
-			}
-		}
-
-		return null;
-	}
-
-	/**
-	 * Adds the given input split to the vertex's log and stores it under the specified sequence number.
-	 * 
-	 * @param vertex
-	 *        the vertex for which the input split shall be stored
-	 * @param sequenceNumber
-	 *        the sequence number identifying the log entry under which the input split shall be stored
-	 * @param inputSplit
-	 *        the input split to be stored
-	 */
-	void addInputSplitToLog(final ExecutionVertex vertex, final int sequenceNumber, final InputSplit inputSplit) {
-
-		final List<InputSplit> inputSplitLog = this.splitMap.get(vertex.getID());
-		if (inputSplitLog == null) {
-			LOG.error("Cannot find input split log for vertex " + vertex + " (" + vertex.getID() + ")");
-			return;
-		}
-
-		synchronized (inputSplitLog) {
-			if (inputSplitLog.size() != sequenceNumber) {
-				LOG.error("Expected input split with sequence number " + inputSplitLog.size() + " for vertex " + vertex
-					+ " (" + vertex.getID() + ") but received " + sequenceNumber + ", skipping...");
-				return;
-			}
-
-			inputSplitLog.add(inputSplit);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java
index 6713714..958fa4c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java
@@ -28,12 +28,10 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.util.StringUtils;
 
 /**
  * An input split wrapper object wraps an input split for RPC calls. In particular, the input split wrapper ensures that
  * the right class loader is used to instantiate the wrapped input split object.
- * 
  */
 public final class InputSplitWrapper implements IOReadableWritable {
 
@@ -45,7 +43,7 @@ public final class InputSplitWrapper implements IOReadableWritable {
 	/**
 	 * The wrapped input split.
 	 */
-	private InputSplit inputSplit = null;
+	private InputSplit inputSplit;
 
 	/**
 	 * Constructs a new input split wrapper.
@@ -55,10 +53,9 @@ public final class InputSplitWrapper implements IOReadableWritable {
 	 * @param inputSplit
 	 *        the input split to be wrapped
 	 */
-	public InputSplitWrapper(final JobID jobID, final InputSplit inputSplit) {
-
+	public InputSplitWrapper(JobID jobID, InputSplit inputSplit) {
 		if (jobID == null) {
-			throw new IllegalArgumentException("Argument jobID must not be null");
+			throw new NullPointerException();
 		}
 
 		this.jobID = jobID;
@@ -74,7 +71,7 @@ public final class InputSplitWrapper implements IOReadableWritable {
 
 
 	@Override
-	public void write(final DataOutputView out) throws IOException {
+	public void write(DataOutputView out) throws IOException {
 
 		// Write the job ID
 		this.jobID.write(out);
@@ -96,7 +93,7 @@ public final class InputSplitWrapper implements IOReadableWritable {
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public void read(final DataInputView in) throws IOException {
+	public void read(DataInputView in) throws IOException {
 
 		// Read the job ID
 		this.jobID.read(in);
@@ -117,15 +114,15 @@ public final class InputSplitWrapper implements IOReadableWritable {
 			try {
 				splitClass = (Class<? extends InputSplit>) Class.forName(className, true, cl);
 			} catch (ClassNotFoundException e) {
-				throw new IOException(StringUtils.stringifyException(e));
+				throw new IOException(e);
 			}
 
 			try {
 				this.inputSplit = splitClass.newInstance();
 			} catch (InstantiationException e) {
-				throw new IOException(StringUtils.stringifyException(e));
+				throw new IOException(e);
 			} catch (IllegalAccessException e) {
-				throw new IOException(StringUtils.stringifyException(e));
+				throw new IOException(e);
 			}
 
 			// Read the input split itself
@@ -142,7 +139,6 @@ public final class InputSplitWrapper implements IOReadableWritable {
 	 * @return the wrapped input split, possibly <code>null</code>
 	 */
 	public InputSplit getInputSplit() {
-
 		return this.inputSplit;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java
index 6a45866..08b5b65 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java
@@ -16,105 +16,182 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobmanager.splitassigner;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.core.io.LocatableInputSplit;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.instance.Instance;
 
 /**
- * The locatable input split assigner is a specific implementation of the {@link InputSplitAssigner} interface for
- * {@link LocatableInputSplit} objects. The locatable input split assigner offers to take the storage location of the
- * individual locatable input splits into account. It attempts to always assign the splits to vertices in a way that the
- * data locality is preserved as well as possible.
- * <p>
- * This class is thread-safe.
- * 
+ * The locatable input split assigner assigns to each host splits that are local, before assigning
+ * splits that are not local. 
  */
 public final class LocatableInputSplitAssigner implements InputSplitAssigner {
 
-	/**
-	 * The logging object which is used to report information and errors.
-	 */
 	private static final Logger LOG = LoggerFactory.getLogger(LocatableInputSplitAssigner.class);
 
-	private final ConcurrentMap<ExecutionGroupVertex, LocatableInputSplitList> vertexMap = new ConcurrentHashMap<ExecutionGroupVertex, LocatableInputSplitList>();
-
 
-	@Override
-	public void registerGroupVertex(final ExecutionGroupVertex groupVertex) {
+	private final Set<LocatableInputSplit> unassigned = new HashSet<LocatableInputSplit>();
+	
+	private final ConcurrentHashMap<String, List<LocatableInputSplit>> localPerHost = new ConcurrentHashMap<String, List<LocatableInputSplit>>();
+	
+	private int localAssignments;		// lock protected by the unassigned set lock
+	
+	private int remoteAssignments;		// lock protected by the unassigned set lock
 
-		if (!LocatableInputSplit.class.isAssignableFrom(groupVertex.getInputSplitType())) {
-			LOG.error(groupVertex.getName() + " produces input splits of type " + groupVertex.getInputSplitType()
-				+ " and cannot be handled by this split assigner");
-			return;
-		}
+	// --------------------------------------------------------------------------------------------
+	
+	public LocatableInputSplitAssigner(Collection<LocatableInputSplit> splits) {
+		this.unassigned.addAll(splits);
+	}
+	
+	public LocatableInputSplitAssigner(LocatableInputSplit[] splits) {
+		Collections.addAll(this.unassigned, splits);
+	}
+	
+	// --------------------------------------------------------------------------------------------
 
-		// Ignore vertices that do not produce splits
-		final InputSplit[] inputSplits = groupVertex.getInputSplits();
-		if (inputSplits == null) {
-			return;
+	@Override
+	public LocatableInputSplit getNextInputSplit(String host) {
+		// for a null host, we return an arbitrary split
+		if (host == null) {
+			
+			synchronized (this.unassigned) {
+				Iterator<LocatableInputSplit> iter = this.unassigned.iterator();
+				if (iter.hasNext()) {
+					LocatableInputSplit next = iter.next();
+					iter.remove();
+					
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Assigning arbitrary split to null host.");
+					}
+					
+					remoteAssignments++;
+					return next;
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("No more input splits remaining.");
+					}
+					return null;
+				}
+			}
 		}
-
-		if (inputSplits.length == 0) {
-			return;
+		
+		host = host.toLowerCase(Locale.US);
+		
+		// for any non-null host, we take the list of non-null splits
+		List<LocatableInputSplit> localSplits = this.localPerHost.get(host);
+		
+		// if we have no list for this host yet, create one
+		if (localSplits == null) {
+			localSplits = new ArrayList<LocatableInputSplit>(16);
+			
+			// lock the list, to be sure that others have to wait for that host's local list
+			synchronized (localSplits) {
+				List<LocatableInputSplit> prior = this.localPerHost.putIfAbsent(host, localSplits);
+				
+				// if someone else beat us in the case to create this list, then we do not populate this one, but
+				// simply work with that other list
+				if (prior == null) {
+					// we are the first, we populate
+					
+					// first, copy the remaining splits to release the lock on the set early
+					// because that is shared among threads
+					LocatableInputSplit[] remaining;
+					synchronized (this.unassigned) {
+						remaining = (LocatableInputSplit[]) this.unassigned.toArray(new LocatableInputSplit[this.unassigned.size()]);
+					}
+					
+					for (LocatableInputSplit is : remaining) {
+						if (isLocal(host, is.getHostnames())) {
+							localSplits.add(is);
+						}
+					}
+				}
+				else {
+					// someone else was faster
+					localSplits = prior;
+				}
+			}
 		}
-
-		final LocatableInputSplitList splitStore = new LocatableInputSplitList();
-		if (this.vertexMap.putIfAbsent(groupVertex, splitStore) != null) {
-			LOG.error(groupVertex.getName()
-				+ " appears to be already registered with the locatable input split assigner, ignoring vertex...");
-			return;
+		
+		// at this point, we have a list of local splits (possibly empty)
+		// we need to make sure no one else operates in the current list (that protects against
+		// list creation races) and that the unassigned set is consistent
+		// NOTE: we need to obtain the locks in this order, strictly!!!
+		synchronized (localSplits) {
+			int size = localSplits.size();
+			if (size > 0) {
+				synchronized (this.unassigned) {
+					do {
+						--size;
+						LocatableInputSplit split = localSplits.remove(size);
+						if (this.unassigned.remove(split)) {
+							
+							if (LOG.isDebugEnabled()) {
+								LOG.debug("Assigning local split to host " + host);
+							}
+							
+							localAssignments++;
+							return split;
+						}
+					} while (size > 0);
+				}
+			}
 		}
-
-		synchronized (splitStore) {
-
-			for (int i = 0; i < inputSplits.length; ++i) {
-				// TODO: Improve this
-				final InputSplit inputSplit = inputSplits[i];
-				if (!(inputSplit instanceof LocatableInputSplit)) {
-					LOG.error("Input split " + i + " of vertex " + groupVertex.getName() + " is of type "
-						+ inputSplit.getClass() + ", ignoring split...");
-					continue;
+		
+		// we did not find a local split, return any
+		synchronized (this.unassigned) {
+			Iterator<LocatableInputSplit> iter = this.unassigned.iterator();
+			if (iter.hasNext()) {
+				LocatableInputSplit next = iter.next();
+				iter.remove();
+				
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Assigning remote split to host " + host);
 				}
-				splitStore.addSplit((LocatableInputSplit) inputSplit);
+				
+				remoteAssignments++;
+				return next;
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("No more input splits remaining.");
+				}
+				return null;
 			}
-
 		}
 	}
-
-
-	@Override
-	public void unregisterGroupVertex(final ExecutionGroupVertex groupVertex) {
-		this.vertexMap.remove(groupVertex);
-	}
-
-
-	@Override
-	public InputSplit getNextInputSplit(final ExecutionVertex vertex) {
-
-		final ExecutionGroupVertex groupVertex = vertex.getGroupVertex();
-		final LocatableInputSplitList splitStore = this.vertexMap.get(groupVertex);
-
-		if (splitStore == null) {
-			return null;
+	
+	private static final boolean isLocal(String host, String[] hosts) {
+		if (host == null || hosts == null) {
+			return false;
 		}
-
-		final Instance instance = vertex.getAllocatedResource().getInstance();
-		if (instance == null) {
-			LOG.error("Instance is null, returning random split");
-			return null;
+		
+		for (String h : hosts) {
+			if (h != null && host.equals(h.toLowerCase())) {
+				return true;
+			}
 		}
-
-		return splitStore.getNextInputSplit(instance);
+		
+		return false;
+	}
+	
+	public int getNumberOfLocalAssignments() {
+		return localAssignments;
+	}
+	
+	public int getNumberOfRemoteAssignments() {
+		return remoteAssignments;
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitList.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitList.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitList.java
deleted file mode 100644
index 71fbf7a..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitList.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.io.LocatableInputSplit;
-import org.apache.flink.runtime.instance.Instance;
-
-/**
- * The locatable input split list stores the locatable input splits for an input vertex that are still expected to be
- * consumed. Besides simply storing the splits, the locatable input split list also computes the distance all
- * {@link org.apache.flink.runtime.instance.Instance} objects which request an input split and its nearest storage location with respect to the
- * underlying network topology. That way input splits are always given to consuming vertices in a way that data locality
- * is preserved as well as possible.
- * <p>
- * This class is not thread-safe.
- * 
- */
-public final class LocatableInputSplitList {
-
-	/**
-	 * The logging object which is used to report information and errors.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(LocatableInputSplitList.class);
-
-	/**
-	 * The set containing all the locatable input splits that still must be consumed.
-	 */
-	private Set<LocatableInputSplit> masterSet = new HashSet<LocatableInputSplit>();
-
-	/**
-	 * The map caching the specific file input split lists for each {@link org.apache.flink.runtime.instance.Instance}.
-	 */
-	private Map<Instance, Queue<QueueElem>> instanceMap = new HashMap<Instance, Queue<QueueElem>>();
-
-	/**
-	 * This is an auxiliary class to store the minimum distance between a file input split's storage locations and an
-	 * {@link org.apache.flink.runtime.instance.Instance}.
-	 * 
-	 */
-	private final class QueueElem implements Comparable<QueueElem> {
-
-		/**
-		 * The locatable input split the distance applies to.
-		 */
-		final LocatableInputSplit inputSplit;
-
-		/**
-		 * The minimum distance between the file input split's storage locations and the instance this object has been
-		 * created for.
-		 */
-		final int distance;
-
-		/**
-		 * Creates a new queue element.
-		 * 
-		 * @param inputSplit
-		 *        the locatable input split to be stored
-		 * @param distance
-		 *        the minimum distance between the stored input split's storage locations and the instance this object
-		 *        has been created for
-		 */
-		private QueueElem(final LocatableInputSplit inputSplit, final int distance) {
-			this.inputSplit = inputSplit;
-			this.distance = distance;
-		}
-
-		/**
-		 * Returns the locatable input split stored within this object.
-		 * 
-		 * @return the locatable input split
-		 */
-		private LocatableInputSplit getInputSplit() {
-			return this.inputSplit;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public int compareTo(final QueueElem o) {
-
-			return (this.distance - o.distance);
-		}
-
-	}
-
-	/**
-	 * Adds the given locate input split to the set of locatable input splits to be consumed.
-	 * 
-	 * @param locatableInputSplit
-	 *        the locatable input split to be added
-	 */
-	synchronized void addSplit(final LocatableInputSplit locatableInputSplit) {
-
-		this.masterSet.add(locatableInputSplit);
-	}
-
-	/**
-	 * Returns the next locatable input split to be consumed by the given instance. The returned input split is selected
-	 * in a
-	 * way that the distance between the split's storage location and the requesting {@link org.apache.flink.runtime.instance.Instance} is as
-	 * short as possible.
-	 * 
-	 * @param instance
-	 *        the instance requesting the next file input split
-	 * @return the next input split to be consumed by the given instance or <code>null</code> if all input splits have
-	 *         already been consumed.
-	 */
-	synchronized LocatableInputSplit getNextInputSplit(final Instance instance) {
-
-		final Queue<QueueElem> instanceSplitList = getInstanceSplitList(instance);
-
-		while (true) {
-
-			final QueueElem candidate = instanceSplitList.poll();
-			if (candidate == null) {
-				return null;
-			}
-
-			if (this.masterSet.remove(candidate.getInputSplit())) {
-				if (LOG.isInfoEnabled()) {
-					if (candidate.distance == 0) {
-						LOG.info(instance + " receives local file input split");
-					} else {
-						LOG.info(instance + " receives remote file input split (distance " + candidate.distance + ")");
-					}
-				}
-				return candidate.getInputSplit();
-			}
-
-			if (this.masterSet.isEmpty()) {
-				return null;
-			}
-		}
-	}
-
-	/**
-	 * Returns a list of locatable input splits specifically ordered for the given {@link org.apache.flink.runtime.instance.Instance}. When the
-	 * list is initially created, it contains all the unconsumed located input splits at that point in time, ascendingly
-	 * ordered
-	 * by the minimum distance between the input splits' storage locations and the given {@link org.apache.flink.runtime.instance.Instance}.
-	 * 
-	 * @param instance
-	 *        the instance for which the locatable input split list has been computed
-	 * @return the list of file input splits ordered specifically for the given instance
-	 */
-	private Queue<QueueElem> getInstanceSplitList(final Instance instance) {
-
-		Queue<QueueElem> instanceSplitList = this.instanceMap.get(instance);
-		if (instanceSplitList == null) {
-
-			// Create and populate instance specific split list
-			instanceSplitList = new PriorityQueue<LocatableInputSplitList.QueueElem>();
-			final Iterator<LocatableInputSplit> it = this.masterSet.iterator();
-			while (it.hasNext()) {
-
-				final LocatableInputSplit split = it.next();
-				final String[] hostnames = split.getHostnames();
-				if (hostnames == null) {
-					instanceSplitList.add(new QueueElem(split, Integer.MAX_VALUE));
-
-				} else {
-
-					int minDistance = Integer.MAX_VALUE;
-					for (int i = 0; i < hostnames.length; ++i) {
-						final int distance = instance.getDistance(hostnames[i]);
-						if (LOG.isDebugEnabled()) {
-							LOG.debug("Distance between " + instance + " and " + hostnames[i] + " is " + distance);
-						}
-						if (distance < minDistance) {
-							minDistance = distance;
-						}
-					}
-
-					instanceSplitList.add(new QueueElem(split, minDistance));
-				}
-			}
-
-			this.instanceMap.put(instance, instanceSplitList);
-		}
-
-		return instanceSplitList;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitAssigner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitAssigner.java
deleted file mode 100644
index 383ed38..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitAssigner.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * 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.jobmanager.splitassigner.file;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobmanager.splitassigner.InputSplitAssigner;
-
-/**
- * The file input split assigner is a specific implementation of the {@link InputSplitAssigner} interface for
- * {@link FileInputSplit} objects. The file input split assigner offers to take the storage location of the individual
- * file input splits into account. It attempts to always assign the splits to vertices in a way that the data locality
- * is preserved as well as possible.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class FileInputSplitAssigner implements InputSplitAssigner {
-
-	/**
-	 * The logging object which is used to report information and errors.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(FileInputSplitAssigner.class);
-
-	private final ConcurrentMap<ExecutionGroupVertex, FileInputSplitList> vertexMap = new ConcurrentHashMap<ExecutionGroupVertex, FileInputSplitList>();
-
-
-	@Override
-	public void registerGroupVertex(final ExecutionGroupVertex groupVertex) {
-
-		if (!FileInputSplit.class.equals(groupVertex.getInputSplitType())) {
-			LOG.error(groupVertex.getName() + " produces input splits of type " + groupVertex.getInputSplitType()
-				+ " and cannot be handled by this split assigner");
-			return;
-		}
-
-		// Ignore vertices that do not produce splits
-		final InputSplit[] inputSplits = groupVertex.getInputSplits();
-		if (inputSplits == null) {
-			return;
-		}
-
-		if (inputSplits.length == 0) {
-			return;
-		}
-
-		final FileInputSplitList splitStore = new FileInputSplitList();
-		if (this.vertexMap.putIfAbsent(groupVertex, splitStore) != null) {
-			LOG.error(groupVertex.getName()
-				+ " appears to be already registered with the file input split assigner, ignoring vertex...");
-			return;
-		}
-
-		synchronized (splitStore) {
-
-			for (int i = 0; i < inputSplits.length; ++i) {
-				// TODO: Improve this
-				final InputSplit inputSplit = inputSplits[i];
-				if (!(inputSplit instanceof FileInputSplit)) {
-					LOG.error("Input split " + i + " of vertex " + groupVertex.getName() + " is of type "
-						+ inputSplit.getClass() + ", ignoring split...");
-					continue;
-				}
-				splitStore.addSplit((FileInputSplit) inputSplit);
-			}
-
-		}
-	}
-
-	@Override
-	public void unregisterGroupVertex(final ExecutionGroupVertex groupVertex) {
-		this.vertexMap.remove(groupVertex);
-	}
-
-	@Override
-	public InputSplit getNextInputSplit(final ExecutionVertex vertex) {
-
-		final ExecutionGroupVertex groupVertex = vertex.getGroupVertex();
-		final FileInputSplitList splitStore = this.vertexMap.get(groupVertex);
-
-		if (splitStore == null) {
-			return null;
-		}
-
-		final Instance instance = vertex.getAllocatedResource().getInstance();
-		if (instance == null) {
-			LOG.error("Instance is null, returning random split");
-			return null;
-		}
-
-		return splitStore.getNextInputSplit(instance);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitList.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitList.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitList.java
deleted file mode 100644
index 06cca24..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/file/FileInputSplitList.java
+++ /dev/null
@@ -1,209 +0,0 @@
-/**
- * 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.jobmanager.splitassigner.file;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.PriorityQueue;
-import java.util.Queue;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.runtime.instance.Instance;
-
-/**
- * The file input split list stores the file input splits for an input vertex that are still expected to be consumed.
- * Besides simply storing the splits, the file input split list also computes the distance all {@link org.apache.flink.runtime.instance.Instance}
- * objects which request a input split and its nearest storage location with respect to the underlying network topology.
- * That way input splits are always given to consuming vertices in a way that data locality is preserved as well as
- * possible.
- * <p>
- * This class is not thread-safe.
- * 
- */
-public final class FileInputSplitList {
-
-	/**
-	 * The logging object which is used to report information and errors.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(FileInputSplitList.class);
-
-	/**
-	 * The set containing all the file input splits that still must be consumed.
-	 */
-	private Set<FileInputSplit> masterSet = new HashSet<FileInputSplit>();
-
-	/**
-	 * The map caching the specific file input split lists for each {@link org.apache.flink.runtime.instance.Instance}.
-	 */
-	private Map<Instance, Queue<QueueElem>> instanceMap = new HashMap<Instance, Queue<QueueElem>>();
-
-	/**
-	 * This is an auxiliary class to store the minimum distance between a file input split's storage locations and an
-	 * {@link org.apache.flink.runtime.instance.Instance}.
-	 * 
-	 */
-	private final class QueueElem implements Comparable<QueueElem> {
-
-		/**
-		 * The file input split the distance applies to.
-		 */
-		final FileInputSplit inputSplit;
-
-		/**
-		 * The minimum distance between the file input split's storage locations and the instance this object has been
-		 * created for.
-		 */
-		final int distance;
-
-		/**
-		 * Creates a new queue element.
-		 * 
-		 * @param inputSplit
-		 *        the file input split to be stored
-		 * @param distance
-		 *        the minimum distance between the stored input split's storage locations and the instance this object
-		 *        has been created for
-		 */
-		private QueueElem(final FileInputSplit inputSplit, final int distance) {
-			this.inputSplit = inputSplit;
-			this.distance = distance;
-		}
-
-		/**
-		 * Returns the file input split stored within this object.
-		 * 
-		 * @return the file input split
-		 */
-		private FileInputSplit getInputSplit() {
-			return this.inputSplit;
-		}
-
-		/**
-		 * {@inheritDoc}
-		 */
-		@Override
-		public int compareTo(final QueueElem o) {
-
-			return (this.distance - o.distance);
-		}
-
-	}
-
-	/**
-	 * Adds the given file input split to the set of file input splits to be consumed.
-	 * 
-	 * @param fileInputSplit
-	 *        the file input split to be added
-	 */
-	synchronized void addSplit(final FileInputSplit fileInputSplit) {
-
-		this.masterSet.add(fileInputSplit);
-	}
-
-	/**
-	 * Returns the next file input split to be consumed by the given instance. The returned input split is selected in a
-	 * way that the distance between the split's storage location and the requesting {@link org.apache.flink.runtime.instance.Instance} is as
-	 * short as possible.
-	 * 
-	 * @param instance
-	 *        the instance requesting the next file input split
-	 * @return the next input split to be consumed by the given instance or <code>null</code> if all input splits have
-	 *         already been consumed.
-	 */
-	synchronized FileInputSplit getNextInputSplit(final Instance instance) {
-
-		final Queue<QueueElem> instanceSplitList = getInstanceSplitList(instance);
-
-		while (true) {
-
-			final QueueElem candidate = instanceSplitList.poll();
-			if (candidate == null) {
-				return null;
-			}
-
-			if (this.masterSet.remove(candidate.getInputSplit())) {
-				if (LOG.isInfoEnabled()) {
-					if (candidate.distance == 0) {
-						LOG.info(instance + " receives local file input split");
-					} else {
-						LOG.info(instance + " receives remote file input split (distance " + candidate.distance + ")");
-					}
-				}
-				return candidate.getInputSplit();
-			}
-
-			if (this.masterSet.isEmpty()) {
-				return null;
-			}
-		}
-	}
-
-	/**
-	 * Returns a list of file input splits specifically ordered for the given {@link org.apache.flink.runtime.instance.Instance}. When the list is
-	 * initially created, it contains all the unconsumed file input splits at that point in time, ascendingly ordered by
-	 * the minimum distance between the input splits' storage locations and the given {@link org.apache.flink.runtime.instance.Instance}.
-	 * 
-	 * @param instance
-	 *        the instance for which the file input split list has been computed
-	 * @return the list of file input splits ordered specifically for the given instance
-	 */
-	private Queue<QueueElem> getInstanceSplitList(final Instance instance) {
-
-		Queue<QueueElem> instanceSplitList = this.instanceMap.get(instance);
-		if (instanceSplitList == null) {
-
-			// Create and populate instance specific split list
-			instanceSplitList = new PriorityQueue<FileInputSplitList.QueueElem>();
-			final Iterator<FileInputSplit> it = this.masterSet.iterator();
-			while (it.hasNext()) {
-
-				final FileInputSplit split = it.next();
-				final String[] hostNames = split.getHostNames();
-				if (hostNames == null) {
-					instanceSplitList.add(new QueueElem(split, Integer.MAX_VALUE));
-
-				} else {
-
-					int minDistance = Integer.MAX_VALUE;
-					for (int i = 0; i < hostNames.length; ++i) {
-						final int distance = instance.getDistance(hostNames[i]);
-						if (LOG.isDebugEnabled()) {
-							LOG.debug("Distance between " + instance + " and " + hostNames[i] + " is " + distance);
-						}
-						if (distance < minDistance) {
-							minDistance = distance;
-						}
-					}
-
-					instanceSplitList.add(new QueueElem(split, minDistance));
-				}
-			}
-
-			this.instanceMap.put(instance, instanceSplitList);
-		}
-
-		return instanceSplitList;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java
index 82e7213..0bec1f3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java
@@ -28,12 +28,8 @@ import static org.junit.Assert.fail;
 import java.net.InetAddress;
 
 import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.apache.flink.util.LogUtils;
-
-import org.apache.log4j.Level;
 
 import org.junit.Assert;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
index 9380e26..290326c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
@@ -57,10 +57,9 @@ import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 import org.apache.flink.runtime.taskmanager.transferenvelope.RegisterTaskManagerResult;
 import org.apache.flink.runtime.types.IntegerRecord;
-import org.apache.flink.util.LogUtils;
+
 import org.junit.After;
 import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -71,12 +70,6 @@ public class LocalInstanceManagerTest {
 	
 	private int port;
 	
-	
-	@BeforeClass
-	public static void initLogger() {
-		LogUtils.initializeDefaultTestConsoleLogger();
-	}
-	
 	@Before
 	public void startJobManagerServer() {
 		try {


[23/63] [abbrv] Redesign Scheduler part 2

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
new file mode 100644
index 0000000..afc0db9
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
@@ -0,0 +1,633 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance;
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+/**
+ * Tests for the scheduler when scheduling tasks in slot sharing groups.
+ */
+public class SchedulerSlotSharingTest {
+	
+	@Test
+	public void scheduleSingleVertexType() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1);
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			
+			// schedule 4 tasks from the first vertex group
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 8), sharingGroup));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 8), sharingGroup));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 8), sharingGroup));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 8), sharingGroup));
+			
+			assertNotNull(s1);
+			assertNotNull(s2);
+			assertNotNull(s3);
+			assertNotNull(s4);
+			
+			assertTrue(areAllDistinct(s1, s2, s3, s4));
+			
+			// we cannot schedule another task from the first vertex group
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			// release something
+			s3.releaseSlot();
+			
+			// allocate another slot from that group
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 8), sharingGroup));
+			assertNotNull(s5);
+			
+			// release all old slots
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s4.releaseSlot();
+			
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 8), sharingGroup));
+			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 6, 8), sharingGroup));
+			AllocatedSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 7, 8), sharingGroup));
+			
+			assertNotNull(s6);
+			assertNotNull(s7);
+			assertNotNull(s8);
+			
+			// make sure we have two slots on the first instance, and two on the second
+			int c = 0;
+			c += (s5.getInstance() == i1) ? 1 : -1;
+			c += (s6.getInstance() == i1) ? 1 : -1;
+			c += (s7.getInstance() == i1) ? 1 : -1;
+			c += (s8.getInstance() == i1) ? 1 : -1;
+			assertEquals(0, c);
+			
+			// release all
+			s5.releaseSlot();
+			s6.releaseSlot();
+			s7.releaseSlot();
+			s8.releaseSlot();
+			
+			// test that everything is released
+			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void scheduleImmediatelyWithSharing() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			
+			// schedule 4 tasks from the first vertex group
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 5), sharingGroup));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 5), sharingGroup));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 5), sharingGroup));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 5), sharingGroup));
+			
+			assertNotNull(s1);
+			assertNotNull(s2);
+			assertNotNull(s3);
+			assertNotNull(s4);
+			
+			assertTrue(areAllDistinct(s1, s2, s3, s4));
+			
+			// we cannot schedule another task from the first vertex group
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			// schedule some tasks from the second ID group
+			AllocatedSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 5), sharingGroup));
+			AllocatedSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 5), sharingGroup));
+			AllocatedSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 5), sharingGroup));
+			AllocatedSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 5), sharingGroup));
+			
+			assertNotNull(s1_2);
+			assertNotNull(s2_2);
+			assertNotNull(s3_2);
+			assertNotNull(s4_2);
+			
+			// we cannot schedule another task from the second vertex group
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			// now, we release some vertices (sub-slots) from the first group.
+			// that should allow us to schedule more vertices from the first group
+			s1.releaseSlot();
+			s4.releaseSlot();
+			
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(2, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+			
+			// we can still not schedule anything from the second group of vertices
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			// we can schedule something from the first vertex group
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup));
+			assertNotNull(s5);
+			
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+			
+			
+			// now we release a slot from the second vertex group and schedule another task from that group
+			s2_2.releaseSlot();
+			AllocatedSlot s5_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 5), sharingGroup));
+			assertNotNull(s5_2);
+			
+			// release all slots
+			s2.releaseSlot();
+			s3.releaseSlot();
+			s5.releaseSlot();
+			
+			s1_2.releaseSlot();
+			s3_2.releaseSlot();
+			s4_2.releaseSlot();
+			s5_2.releaseSlot();
+			
+			// test that everything is released
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			JobVertexID jid3 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2, jid3);
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			
+			// schedule 4 tasks from the first vertex group
+			AllocatedSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup));
+			AllocatedSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup));
+			AllocatedSlot s3_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
+			AllocatedSlot s4_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup));
+			
+			assertNotNull(s1_1);
+			assertNotNull(s2_1);
+			assertNotNull(s3_1);
+			assertNotNull(s4_1);
+			
+			assertTrue(areAllDistinct(s1_1, s2_1, s3_1, s4_1));
+			
+			// schedule 4 tasks from the second vertex group
+			AllocatedSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 7), sharingGroup));
+			AllocatedSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 7), sharingGroup));
+			AllocatedSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 7), sharingGroup));
+			AllocatedSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 7), sharingGroup));
+			
+			assertNotNull(s1_2);
+			assertNotNull(s2_2);
+			assertNotNull(s3_2);
+			assertNotNull(s4_2);
+			
+			assertTrue(areAllDistinct(s1_2, s2_2, s3_2, s4_2));
+			
+			// schedule 4 tasks from the third vertex group
+			AllocatedSlot s1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 4), sharingGroup));
+			AllocatedSlot s2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 4), sharingGroup));
+			AllocatedSlot s3_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 4), sharingGroup));
+			AllocatedSlot s4_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 4), sharingGroup));
+			
+			assertNotNull(s1_3);
+			assertNotNull(s2_3);
+			assertNotNull(s3_3);
+			assertNotNull(s4_3);
+			
+			assertTrue(areAllDistinct(s1_3, s2_3, s3_3, s4_3));
+			
+			
+			// we cannot schedule another task from the second vertex group
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 5), sharingGroup));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			// release the second vertex group
+			s1_2.releaseSlot();
+			s2_2.releaseSlot();
+			s3_2.releaseSlot();
+			s4_2.releaseSlot();
+			
+			AllocatedSlot s5_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 7), sharingGroup));
+			AllocatedSlot s6_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 6, 7), sharingGroup));
+			AllocatedSlot s7_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 7, 7), sharingGroup));
+			
+			assertNotNull(s5_2);
+			assertNotNull(s6_2);
+			assertNotNull(s7_2);
+			
+			// release the slots
+			s1_1.releaseSlot();
+			s2_1.releaseSlot();
+			s3_1.releaseSlot();
+			s4_1.releaseSlot();
+			
+			s5_2.releaseSlot();
+			s6_2.releaseSlot();
+			s7_2.releaseSlot();
+			
+			// test that everything is released
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+			
+			s1_3.releaseSlot();
+			s2_3.releaseSlot();
+			s3_3.releaseSlot();
+			s4_3.releaseSlot();
+			
+			// test that everything is released
+			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup2() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			JobVertexID jid3 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			
+			// schedule 1 tasks from the first vertex group and 2 from the second
+			AllocatedSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 2), sharingGroup));
+			AllocatedSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 2), sharingGroup));
+			AllocatedSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 2), sharingGroup));
+			
+			assertNotNull(s1_1);
+			assertNotNull(s2_1);
+			assertNotNull(s2_2);
+			
+			assertEquals(2, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+			
+			// release the two from the second
+			s2_1.releaseSlot();
+			s2_2.releaseSlot();
+			
+			
+			// this should free one slot so we can allocate one non-shared
+			AllocatedSlot sx = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1)));
+			assertNotNull(sx);
+			
+			assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void scheduleMixedSharingAndNonSharing() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			JobVertexID jidA = new JobVertexID();
+			JobVertexID jidB= new JobVertexID();
+			JobVertexID jidC = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+			scheduler.newInstanceAvailable(getRandomInstance(3));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			
+			// schedule some individual vertices
+			AllocatedSlot sA1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidA, 0, 2)));
+			AllocatedSlot sA2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidA, 1, 2)));
+			assertNotNull(sA1);
+			assertNotNull(sA2);
+			
+			// schedule some vertices in the sharing group
+			AllocatedSlot s1_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup));
+			AllocatedSlot s1_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup));
+			AllocatedSlot s2_0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup));
+			AllocatedSlot s2_1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup));
+			assertNotNull(s1_0);
+			assertNotNull(s1_1);
+			assertNotNull(s2_0);
+			assertNotNull(s2_1);
+			
+			// schedule another isolated vertex
+			AllocatedSlot sB1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 1, 3)));
+			assertNotNull(sB1);
+			
+			// should not be able to schedule more vertices
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 0, 3)));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 0, 1)));
+				fail("Scheduler accepted too many tasks at the same time");
+			}
+			catch (NoResourceAvailableException e) {
+				// good!
+			}
+			catch (Exception e) {
+				fail("Wrong exception.");
+			}
+			
+			// release some isolated task and check that the sharing group may grow
+			sA1.releaseSlot();
+			
+			AllocatedSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
+			AllocatedSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup));
+			assertNotNull(s1_2);
+			assertNotNull(s2_2);
+			
+			// release three of the previously allocated sub slots, which guarantees to return one shared slot
+			s1_0.releaseSlot();
+			s1_1.releaseSlot();
+			s2_0.releaseSlot();
+			
+			assertEquals(1, scheduler.getNumberOfAvailableSlots());
+			
+			// schedule one more no-shared task
+			AllocatedSlot sB0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 0, 3)));
+			assertNotNull(sB0);
+			
+			// release the last of the original shared slots and allocate one more non-shared slot
+			s2_1.releaseSlot();
+			AllocatedSlot sB2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidB, 2, 3)));
+			assertNotNull(sB2);
+			
+			
+			// release on non-shared and add some shared slots
+			sA2.releaseSlot();
+			AllocatedSlot s1_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup));
+			AllocatedSlot s2_3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup));
+			assertNotNull(s1_3);
+			assertNotNull(s2_3);
+			
+			// release all shared and allocate all in non-shared
+			s1_2.releaseSlot();
+			s2_2.releaseSlot();
+			s1_3.releaseSlot();
+			s2_3.releaseSlot();
+			
+			AllocatedSlot sC0 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 1, 2)));
+			AllocatedSlot sC1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jidC, 0, 2)));
+			assertNotNull(sC0);
+			assertNotNull(sC1);
+			
+			
+			sB0.releaseSlot();
+			sB1.releaseSlot();
+			sB2.releaseSlot();
+			sC0.releaseSlot();
+			sC1.releaseSlot();
+			
+			// test that everything is released
+			assertEquals(5, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	/**
+	 * Tests that the scheduler assigns the correct existing shared slots
+	 */
+	@Test
+	public void testLocalizedAssignment1() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			
+			
+			// schedule one to each instance
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup));
+			assertNotNull(s1);
+			assertNotNull(s2);
+			
+			assertEquals(2, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(1, i1.getNumberOfAvailableSlots());
+			assertEquals(1, i2.getNumberOfAvailableSlots());
+			
+			// schedule one from the other group to each instance
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i2), sharingGroup));
+			assertNotNull(s3);
+			assertNotNull(s4);
+			
+			assertEquals(2, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(1, i1.getNumberOfAvailableSlots());
+			assertEquals(1, i2.getNumberOfAvailableSlots());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	/**
+	 * Tests that the scheduler assigns to new local slots, rather than to existing non-local slots
+	 */
+	@Test
+	public void testLocalizedAssignment2() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			
+			
+			// schedule one to each instance
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup));
+			assertNotNull(s1);
+			assertNotNull(s2);
+			
+			assertEquals(2, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, i1.getNumberOfAvailableSlots());
+			assertEquals(2, i2.getNumberOfAvailableSlots());
+			
+			// schedule one from the other group to each instance
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i2), sharingGroup));
+			assertNotNull(s3);
+			assertNotNull(s4);
+			
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, i1.getNumberOfAvailableSlots());
+			assertEquals(0, i2.getNumberOfAvailableSlots());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	/**
+	 * Tests that the scheduler can fall back to non-local
+	 */
+	@Test
+	public void testLocalizedAssignment3() {
+		try {
+			
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
new file mode 100644
index 0000000..78a397e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
@@ -0,0 +1,145 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+
+public class SchedulerTestUtils {
+	
+	private static final AtomicInteger port = new AtomicInteger(10000);
+
+	// --------------------------------------------------------------------------------------------
+	
+	public static Instance getRandomInstance(int numSlots) {
+		if (numSlots <= 0) {
+			throw new IllegalArgumentException();
+		}
+		
+		InetAddress address;
+		try {
+			address = InetAddress.getByName("127.0.0.1");
+		} catch (UnknownHostException e) {
+			throw new RuntimeException("Test could not create IP address for localhost loopback.");
+		}
+		
+		int ipcPort = port.getAndIncrement();
+		int dataPort = port.getAndIncrement();
+		
+		InstanceConnectionInfo ci = new InstanceConnectionInfo(address, ipcPort, dataPort);
+		
+		final long GB = 1024L*1024*1024;
+		HardwareDescription resources = new HardwareDescription(4, 4*GB, 3*GB, 2*GB);
+		
+		return new Instance(ci, new InstanceID(), resources, numSlots);
+	}
+	
+	
+	public static ExecutionVertex2 getDummyVertex() {
+		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
+		
+		when(vertex.getJobId()).thenReturn(new JobID());
+		when(vertex.toString()).thenReturn("TEST-VERTEX");
+		
+		return vertex;
+	}
+	
+	public static ExecutionVertex2 getTestVertex(JobVertexID jid, int taskIndex, int numTasks) {
+		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
+		
+		when(vertex.getPreferredLocations()).thenReturn(null);
+		when(vertex.getJobId()).thenReturn(new JobID());
+		when(vertex.getJobvertexId()).thenReturn(jid);
+		when(vertex.getParallelSubtaskIndex()).thenReturn(taskIndex);
+		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks);
+		when(vertex.toString()).thenReturn("TEST-VERTEX");
+		
+		return vertex;
+	}
+	
+	public static ExecutionVertex2 getTestVertexWithLocation(JobVertexID jid, int taskIndex, int numTasks, Instance... locations) {
+		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
+		
+		when(vertex.getPreferredLocations()).thenReturn(Arrays.asList(locations));
+		when(vertex.getJobId()).thenReturn(new JobID());
+		when(vertex.getJobvertexId()).thenReturn(jid);
+		when(vertex.getParallelSubtaskIndex()).thenReturn(taskIndex);
+		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks);
+		when(vertex.toString()).thenReturn("TEST-VERTEX");
+		
+		return vertex;
+	}
+	
+	public static ExecutionVertex2 getTestVertex(Iterable<Instance> preferredLocations) {
+		ExecutionVertex2 vertex = mock(ExecutionVertex2.class);
+		
+		when(vertex.getPreferredLocations()).thenReturn(preferredLocations);
+		when(vertex.getJobId()).thenReturn(new JobID());
+		when(vertex.toString()).thenReturn("TEST-VERTEX");
+		
+		return vertex;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static boolean areAllDistinct(Object ... obj) {
+		if (obj == null) {
+			return true;
+		}
+		
+		HashSet<Object> set = new HashSet<Object>();
+		for (Object o : obj) {
+			set.add(o);
+		}
+		
+		return set.size() == obj.length;
+	}
+	
+	public static boolean areSameSets(Collection<Object> set1, Collection<Object> set2) {
+		if (set1 == null || set2 == null) {
+			throw new IllegalArgumentException();
+		}
+		
+		HashSet<Object> set = new HashSet<Object>(set1);
+		for (Object o : set2) {
+			if (!set.remove(o)) {
+				return false;
+			}
+		}
+		
+		return set.isEmpty();
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueueTest.java
new file mode 100644
index 0000000..9b9a899
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueueTest.java
@@ -0,0 +1,110 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.junit.Assert.*;
+
+import java.util.Queue;
+
+import org.junit.Test;
+
+/**
+ * Test for the basic functionality of the {@link LifoSetQueue}.
+ */
+public class SetQueueTest {
+
+	@Test
+	public void testSizeAddPollAndPeek() {
+		try {
+			Queue<Integer> queue = new SetQueue<Integer>();
+			
+			// empty queue
+			assertEquals(0, queue.size());
+			assertNull(queue.poll());
+			assertNull(queue.peek());
+			
+			// add some elements
+			assertTrue(queue.add(1));
+			assertTrue(queue.offer(2));
+			assertTrue(queue.offer(3));
+			assertEquals(3, queue.size());
+			
+			assertEquals(1, queue.peek().intValue());
+			
+			// prevent duplicates. note that the methods return true, because no capacity constraint is violated
+			assertTrue(queue.add(1));
+			assertTrue(queue.offer(1));
+			assertTrue(queue.add(3));
+			assertTrue(queue.offer(3));
+			assertTrue(queue.add(2));
+			assertTrue(queue.offer(2));
+			assertEquals(3, queue.size());
+			
+			// peek and poll some elements
+			assertEquals(1, queue.peek().intValue());
+			assertEquals(3, queue.size());
+			assertEquals(1, queue.poll().intValue());
+			assertEquals(2, queue.size());
+			assertEquals(2, queue.peek().intValue());
+			assertEquals(2, queue.size());
+			assertEquals(2, queue.poll().intValue());
+			assertEquals(1, queue.size());
+			assertEquals(3, queue.peek().intValue());
+			assertEquals(1, queue.size());
+			assertEquals(3, queue.poll().intValue());
+			assertEquals(0, queue.size());
+			assertTrue(queue.isEmpty());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getClass().getSimpleName() + " : " + e.getMessage());
+		}
+	}
+	
+
+	@Test
+	public void testOrder() {
+		try {
+			Queue<Integer> queue = new SetQueue<Integer>();
+			
+			queue.add(1);
+			queue.add(2);
+			queue.add(3);
+			
+			assertEquals(1, queue.peek().intValue());
+			
+			queue.add(4);
+			assertEquals(1, queue.peek().intValue());
+			
+			queue.remove(2);
+			assertEquals(1, queue.peek().intValue());
+			
+			queue.remove(4);
+			assertEquals(1, queue.peek().intValue());
+			
+			queue.remove(2);
+			assertEquals(1, queue.poll().intValue());
+			assertEquals(3, queue.poll().intValue());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getClass().getSimpleName() + " : " + e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
new file mode 100644
index 0000000..de22999
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
@@ -0,0 +1,118 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.eq;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+public class SharedSlotsTest {
+
+	@Test
+	public void createAndDoNotRelease() {
+		try {
+			SlotSharingGroupAssignment assignment = mock(SlotSharingGroupAssignment.class);
+			when(assignment.sharedSlotAvailableForJid(any(SharedSlot.class), any(JobVertexID.class), any(boolean.class))).thenReturn(false);
+			
+			Instance instance = SchedulerTestUtils.getRandomInstance(1);
+			
+			SharedSlot slot = new SharedSlot(instance.allocateSlot(new JobID()), assignment);
+			assertFalse(slot.isDisposed());
+			
+			SubSlot ss1 = slot.allocateSubSlot(new JobVertexID());
+			assertNotNull(ss1);
+			
+			// verify resources
+			assertEquals(instance, ss1.getInstance());
+			assertEquals(0, ss1.getSlotNumber());
+			assertEquals(slot.getAllocatedSlot().getJobID(), ss1.getJobID());
+			
+			SubSlot ss2 = slot.allocateSubSlot(new JobVertexID());
+			assertNotNull(ss2);
+			
+			assertEquals(2, slot.getNumberOfAllocatedSubSlots());
+			
+			// release first slot, should not trigger release
+			ss1.releaseSlot();
+			assertFalse(slot.isDisposed());
+			
+			ss2.releaseSlot();
+			assertFalse(slot.isDisposed());
+			
+			// the shared slot should now dispose itself
+			assertEquals(0, slot.getNumberOfAllocatedSubSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void createAndRelease() {
+		try {
+			SlotSharingGroupAssignment assignment = mock(SlotSharingGroupAssignment.class);
+			when(assignment.sharedSlotAvailableForJid(any(SharedSlot.class), any(JobVertexID.class), eq(false))).thenReturn(false);
+			when(assignment.sharedSlotAvailableForJid(any(SharedSlot.class), any(JobVertexID.class), eq(true))).thenReturn(true);
+			
+			Instance instance = SchedulerTestUtils.getRandomInstance(1);
+			
+			SharedSlot slot = new SharedSlot(instance.allocateSlot(new JobID()), assignment);
+			assertFalse(slot.isDisposed());
+			
+			SubSlot ss1 = slot.allocateSubSlot(new JobVertexID());
+			assertNotNull(ss1);
+			
+			// verify resources
+			assertEquals(instance, ss1.getInstance());
+			assertEquals(0, ss1.getSlotNumber());
+			assertEquals(slot.getAllocatedSlot().getJobID(), ss1.getJobID());
+			
+			SubSlot ss2 = slot.allocateSubSlot(new JobVertexID());
+			assertNotNull(ss2);
+			
+			assertEquals(2, slot.getNumberOfAllocatedSubSlots());
+			
+			// release first slot, should not trigger release
+			ss1.releaseSlot();
+			assertFalse(slot.isDisposed());
+			
+			ss2.releaseSlot();
+			assertTrue(slot.isDisposed());
+			
+			// the shared slot should now dispose itself
+			assertEquals(0, slot.getNumberOfAllocatedSubSlots());
+			
+			assertNull(slot.allocateSubSlot(new JobVertexID()));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
new file mode 100644
index 0000000..423004f
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureTest.java
@@ -0,0 +1,177 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.junit.Assert.*;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.junit.Test;
+
+public class SlotAllocationFutureTest {
+
+	@Test
+	public void testInvalidActions() {
+		try {
+			final SlotAllocationFuture future = new SlotAllocationFuture();
+			
+			SlotAllocationFutureAction action = new SlotAllocationFutureAction() {
+				@Override
+				public void slotAllocated(AllocatedSlot slot) {}
+			};
+			
+			future.setFutureAction(action);
+			try {
+				future.setFutureAction(action);
+				fail();
+			} catch (IllegalStateException e) {
+				// expected
+			}
+			
+			final AllocatedSlot slot1 = new AllocatedSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0);
+			final AllocatedSlot slot2 = new AllocatedSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0);
+			
+			future.setSlot(slot1);
+			try {
+				future.setSlot(slot2);
+				fail();
+			} catch (IllegalStateException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void setWithAction() {
+		try {
+			
+			// action before the slot
+			{
+				final AtomicInteger invocations = new AtomicInteger();
+				final AllocatedSlot thisSlot = new AllocatedSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0);
+				
+				SlotAllocationFuture future = new SlotAllocationFuture();
+				
+				future.setFutureAction(new SlotAllocationFutureAction() {
+					@Override
+					public void slotAllocated(AllocatedSlot slot) {
+						assertEquals(thisSlot, slot);
+						invocations.incrementAndGet();
+					}
+				});
+				
+				future.setSlot(thisSlot);
+				
+				assertEquals(1, invocations.get());
+			}
+			
+			// slot before action
+			{
+				final AtomicInteger invocations = new AtomicInteger();
+				final AllocatedSlot thisSlot = new AllocatedSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0);
+				
+				SlotAllocationFuture future = new SlotAllocationFuture();
+				future.setSlot(thisSlot);
+				
+				future.setFutureAction(new SlotAllocationFutureAction() {
+					@Override
+					public void slotAllocated(AllocatedSlot slot) {
+						assertEquals(thisSlot, slot);
+						invocations.incrementAndGet();
+					}
+				});
+				
+				assertEquals(1, invocations.get());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void setSync() {
+		try {
+			// sync before setting the slot
+			{
+				final AtomicInteger invocations = new AtomicInteger();
+				final AtomicBoolean error = new AtomicBoolean();
+				
+				final AllocatedSlot thisSlot = new AllocatedSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0);
+				
+				final SlotAllocationFuture future = new SlotAllocationFuture();
+				
+				
+				Runnable r = new Runnable() {
+					@Override
+					public void run() {
+						try {
+							AllocatedSlot syncSlot = future.waitTillAllocated();
+							if (syncSlot == null || syncSlot != thisSlot) {
+								error.set(true);
+								return;
+							}
+							invocations.incrementAndGet();
+						}
+						catch (Throwable t) {
+							error.set(true);
+						}
+					}
+				};
+				
+				Thread syncer = new Thread(r);
+				syncer.start();
+				
+				// wait, and give the sync thread a chance to sync
+				Thread.sleep(10);
+				future.setSlot(thisSlot);
+				
+				syncer.join();
+				
+				assertFalse(error.get());
+				assertEquals(1, invocations.get());
+			}
+			
+			// setting slot before syncing
+			{
+				final AllocatedSlot thisSlot = new AllocatedSlot(new JobID(), SchedulerTestUtils.getRandomInstance(1), 0);
+				final SlotAllocationFuture future = new SlotAllocationFuture();
+
+				future.setSlot(thisSlot);
+				
+				AllocatedSlot retrieved = future.waitTillAllocated();
+				
+				assertNotNull(retrieved);
+				assertEquals(thisSlot, retrieved);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[47/63] [abbrv] git commit: Add options strict co-location constraints to scheduler

Posted by se...@apache.org.
Add options strict co-location constraints to scheduler


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/91871757
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/91871757
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/91871757

Branch: refs/heads/master
Commit: 91871757bebf7404324334d79755e3f117752966
Parents: b3c30ca
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 00:39:00 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../scheduler/CoLocationConstraint.java         |  64 ++++++
 .../jobmanager/scheduler/ScheduledUnit.java     |  36 +--
 .../runtime/jobmanager/scheduler/Scheduler.java |  58 ++++-
 .../jobmanager/scheduler/SharedSlot.java        |  27 ++-
 .../runtime/jobmanager/scheduler/SubSlot.java   |  11 +-
 .../ScheduleWithCoLocationHintTest.java         | 225 +++++++++++++++++++
 6 files changed, 400 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91871757/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
new file mode 100644
index 0000000..26332c8
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
@@ -0,0 +1,64 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+public class CoLocationConstraint {
+	
+	private static final AtomicReferenceFieldUpdater<CoLocationConstraint, SharedSlot> UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(CoLocationConstraint.class, SharedSlot.class, "slot");
+	
+	private volatile SharedSlot slot;
+
+	
+	public boolean isUnassigned() {
+		return slot == null;
+	}
+	
+	public SharedSlot getSlot() {
+		return slot;
+	}
+	
+	public SharedSlot swapInNewSlot(AllocatedSlot newSlot) {
+		SharedSlot newShared = new SharedSlot(newSlot);
+		
+		// atomic swap/release-other to prevent resource leaks
+		while (true) {
+			SharedSlot current = this.slot;
+			if (UPDATER.compareAndSet(this, current, newShared)) {
+				if (current != null) {
+					current.rease();
+				}
+				return newShared;
+			}
+		}
+	}
+	
+	public SubSlot allocateSubSlot(JobVertexID jid) {
+		if (this.slot == null) {
+			throw new IllegalStateException("Location constraint has not yet been assigned a slot.");
+		}
+		
+		return slot.allocateSubSlot(jid);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91871757/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
index 10190f5..28fd916 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
@@ -21,35 +21,40 @@ package org.apache.flink.runtime.jobmanager.scheduler;
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
+import com.google.common.base.Preconditions;
+
 public class ScheduledUnit {
 	
 	private final Execution vertexExecution;
 	
 	private final SlotSharingGroup sharingGroup;
 	
+	private final CoLocationConstraint locationConstraint;
+	
 	// --------------------------------------------------------------------------------------------
 	
-	public ScheduledUnit(Execution taskVertex) {
-		if (taskVertex == null) {
-			throw new NullPointerException();
-		}
+	public ScheduledUnit(Execution task) {
+		Preconditions.checkNotNull(task);
 		
-		this.vertexExecution = taskVertex;
+		this.vertexExecution = task;
 		this.sharingGroup = null;
+		this.locationConstraint = null;
 	}
 	
-	public ScheduledUnit(Execution taskVertex, SlotSharingGroup sharingUnit) {
-		if (taskVertex == null) {
-			throw new NullPointerException();
-		}
+	public ScheduledUnit(Execution task, SlotSharingGroup sharingUnit) {
+		Preconditions.checkNotNull(task);
 		
-		this.vertexExecution = taskVertex;
+		this.vertexExecution = task;
 		this.sharingGroup = sharingUnit;
+		this.locationConstraint = null;
 	}
 	
-	ScheduledUnit() {
-		this.vertexExecution = null;
+	public ScheduledUnit(Execution task, CoLocationConstraint locationConstraint) {
+		Preconditions.checkNotNull(task);
+		
+		this.vertexExecution = task;
 		this.sharingGroup = null;
+		this.locationConstraint = locationConstraint;
 	}
 
 	// --------------------------------------------------------------------------------------------
@@ -65,11 +70,16 @@ public class ScheduledUnit {
 	public SlotSharingGroup getSlotSharingGroup() {
 		return sharingGroup;
 	}
+	
+	public CoLocationConstraint getLocationConstraint() {
+		return locationConstraint;
+	}
 
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
 	public String toString() {
-		return "{vertex=" + vertexExecution.getVertexWithAttempt() + ", sharingUnit=" + sharingGroup + '}';
+		return "{task=" + vertexExecution.getVertexWithAttempt() + ", sharingUnit=" + sharingGroup + 
+				", locationConstraint=" + locationConstraint + '}';
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91871757/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index cec8fb7..eb2c0a5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -27,7 +27,6 @@ import java.util.concurrent.ExecutorService;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
@@ -149,13 +148,66 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 		final ExecutionVertex vertex = task.getTaskToExecute().getVertex();
 	
 		synchronized (globalLock) {
+			
 			// 1)  === If the task has a strict co-schedule hint, obey it ===
-
+			
+			CoLocationConstraint locationConstraint = task.getLocationConstraint();
+			if (locationConstraint != null) {
+				// location constraints can never be scheduled in a queued fashion
+				if (queueIfNoResource) {
+					throw new IllegalArgumentException("A task with a location constraint was scheduled in a queued fashion.");
+				}
+				
+				// since we are inside the global lock scope, we can check, allocate, and assign
+				// in one atomic action. however, slots may die and be deallocated
+				
+				// (a) is the constraint has not yet has a slot, get one
+				if (locationConstraint.isUnassigned()) {
+					// try and get a slot
+					AllocatedSlot newSlot = getFreeSlotForTask(vertex);
+					if (newSlot == null) {
+						throw new NoResourceAvailableException();
+					}
+					SharedSlot sl = locationConstraint.swapInNewSlot(newSlot);
+					SubSlot slot = sl.allocateSubSlot(vertex.getJobvertexId());
+					
+					updateLocalityCounters(newSlot.getLocality());
+					return slot;
+				}
+				else {
+					// try to get a subslot. returns null, if the location's slot has been released
+					// in the meantime
+					SubSlot slot = locationConstraint.allocateSubSlot(vertex.getJobvertexId());
+					if (slot == null) {
+						// get a new slot. at the same instance!!!
+						Instance location = locationConstraint.getSlot().getAllocatedSlot().getInstance();
+						AllocatedSlot newSlot;
+						try {
+							newSlot = location.allocateSlot(vertex.getJobId());
+						} catch (InstanceDiedException e) {
+							throw new NoResourceAvailableException("The instance of the required location died.");
+						}
+						if (newSlot == null) {
+							throw new NoResourceAvailableException();
+						}
+						SharedSlot sharedSlot = locationConstraint.swapInNewSlot(newSlot);
+						slot = sharedSlot.allocateSubSlot(vertex.getJobvertexId());
+					}
+					
+					updateLocalityCounters(Locality.LOCAL);
+					return slot;
+				}
+			}
 		
 			// 2)  === If the task has a slot sharing group, schedule with shared slots ===
 			
 			SlotSharingGroup sharingUnit = task.getSlotSharingGroup();
 			if (sharingUnit != null) {
+				
+				if (queueIfNoResource) {
+					throw new IllegalArgumentException("A task with a vertex sharing group was scheduled in a queued fashion.");
+				}
+				
 				final SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
 				
 				AllocatedSlot newSlot = null;
@@ -263,7 +315,7 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 					}
 				}
 				
-				if (instanceToUse == null) {					
+				if (instanceToUse == null) {
 					instanceToUse = this.instancesWithAvailableResources.poll();
 					locality = Locality.NON_LOCAL;
 					if (LOG.isDebugEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91871757/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
index 36d8a8b..0f3687a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
@@ -48,6 +48,16 @@ public class SharedSlot {
 		this.subSlots = new HashSet<SubSlot>();
 	}
 	
+	public SharedSlot(AllocatedSlot allocatedSlot) {
+		if (allocatedSlot == null) {
+			throw new NullPointerException();
+		}
+		
+		this.allocatedSlot = allocatedSlot;
+		this.assignmentGroup = null;;
+		this.subSlots = new HashSet<SubSlot>();
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	
 	public AllocatedSlot getAllocatedSlot() {
@@ -76,6 +86,17 @@ public class SharedSlot {
 		}
 	}
 	
+	public void rease() {
+		synchronized (this.subSlots) {
+			disposed = true;
+			for (SubSlot ss : subSlots) {
+				ss.releaseSlot();
+			}
+		}
+		
+		allocatedSlot.releaseSlot();
+	}
+	
 	void returnAllocatedSlot(SubSlot slot) {
 		boolean release;
 		
@@ -84,7 +105,11 @@ public class SharedSlot {
 				throw new IllegalArgumentException("Wrong shared slot for subslot.");
 			}
 			
-			release = assignmentGroup.sharedSlotAvailableForJid(this, slot.getJobVertexId(), this.subSlots.isEmpty());
+			if (assignmentGroup != null) {
+				release = assignmentGroup.sharedSlotAvailableForJid(this, slot.getJobVertexId(), this.subSlots.isEmpty());
+			} else {
+				release = subSlots.isEmpty();
+			}
 			
 			if (release) {
 				disposed = true;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91871757/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
index 003239d..ca2fb5e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
@@ -45,10 +45,13 @@ public class SubSlot extends AllocatedSlot {
 	public void releaseSlot() {
 		// cancel everything, if there is something. since this is atomically status based,
 		// it will not happen twice if another attempt happened before or concurrently
-		cancel();
-		
-		if (markReleased()) {
-			this.sharedSlot.returnAllocatedSlot(this);
+		try {
+			cancel();
+		}
+		finally {
+			if (markReleased()) {
+				this.sharedSlot.returnAllocatedSlot(this);
+			}
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91871757/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
new file mode 100644
index 0000000..0ee9346
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
@@ -0,0 +1,225 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.junit.Test;
+
+public class ScheduleWithCoLocationHintTest {
+
+	@Test
+	public void schedule() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			Instance i3 = getRandomInstance(2);
+			
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i3);
+			
+			assertEquals(6, scheduler.getNumberOfAvailableSlots());
+			
+			CoLocationConstraint c1 = new CoLocationConstraint();
+			CoLocationConstraint c2 = new CoLocationConstraint();
+			CoLocationConstraint c3 = new CoLocationConstraint();
+			CoLocationConstraint c4 = new CoLocationConstraint();
+			CoLocationConstraint c5 = new CoLocationConstraint();
+			CoLocationConstraint c6 = new CoLocationConstraint();
+			
+			// schedule 4 tasks from the first vertex group
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), c1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 6), c2));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 6), c3));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 6), c4));
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 6), c1));
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 6), c2));
+			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 6), c3));
+			AllocatedSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 6), c5));
+			AllocatedSlot s9 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 6), c6));
+			AllocatedSlot s10 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 6), c4));
+			AllocatedSlot s11 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 6), c5));
+			AllocatedSlot s12 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 6), c6));
+
+			assertNotNull(s1);
+			assertNotNull(s2);
+			assertNotNull(s3);
+			assertNotNull(s4);
+			assertNotNull(s5);
+			assertNotNull(s6);
+			assertNotNull(s7);
+			assertNotNull(s8);
+			assertNotNull(s9);
+			assertNotNull(s10);
+			assertNotNull(s11);
+			assertNotNull(s12);
+			
+			assertEquals(s1.getInstance(), s5.getInstance());
+			assertEquals(s2.getInstance(), s6.getInstance());
+			assertEquals(s3.getInstance(), s7.getInstance());
+			assertEquals(s4.getInstance(), s10.getInstance());
+			assertEquals(s8.getInstance(), s11.getInstance());
+			assertEquals(s9.getInstance(), s12.getInstance());
+			
+			assertEquals(c1.getSlot().getAllocatedSlot().getInstance(), s1.getInstance());
+			assertEquals(c2.getSlot().getAllocatedSlot().getInstance(), s2.getInstance());
+			assertEquals(c3.getSlot().getAllocatedSlot().getInstance(), s3.getInstance());
+			assertEquals(c4.getSlot().getAllocatedSlot().getInstance(), s4.getInstance());
+			assertEquals(c5.getSlot().getAllocatedSlot().getInstance(), s8.getInstance());
+			assertEquals(c6.getSlot().getAllocatedSlot().getInstance(), s9.getInstance());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+			
+			// the first assignments are unconstrained, co.-schedulings are constrained
+			assertEquals(6, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(6, scheduler.getNumberOfUnconstrainedAssignments());
+			
+			// release some slots, be sure that new available ones come up
+			s4.releaseSlot();
+			s10.releaseSlot();
+			assertEquals(1, scheduler.getNumberOfAvailableSlots());
+			
+			AllocatedSlot single = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1)));
+			assertNotNull(single);
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s3.releaseSlot();
+			s5.releaseSlot();
+			s6.releaseSlot();
+			s7.releaseSlot();
+			s8.releaseSlot();
+			s9.releaseSlot();
+			s11.releaseSlot();
+			s12.releaseSlot();
+			
+			assertEquals(5, scheduler.getNumberOfAvailableSlots());
+			
+			assertEquals(6, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(7, scheduler.getNumberOfUnconstrainedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void scheduleWithIntermediateRelease() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			JobVertexID jid3 = new JobVertexID();
+			JobVertexID jid4 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(1);
+			Instance i2 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			CoLocationConstraint c1 = new CoLocationConstraint();
+			
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), c1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), c1));
+			
+			AllocatedSlot sSolo = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 1)));
+			
+			Instance loc = s1.getInstance();
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			sSolo.releaseSlot();
+			
+			AllocatedSlot sNew = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), c1));
+			assertEquals(loc, sNew.getInstance());
+			
+			assertEquals(2, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(2, scheduler.getNumberOfUnconstrainedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void scheduleWithReleaseNoResource() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			JobVertexID jid3 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(1);
+			Instance i2 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			CoLocationConstraint c1 = new CoLocationConstraint();
+			
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), c1));
+			s1.releaseSlot();
+			
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1)));
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 2)));
+			
+			
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), c1));
+				fail("Scheduled even though no resource was available.");
+			} catch (NoResourceAvailableException e) {
+				// expected
+			}
+			
+			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(3, scheduler.getNumberOfUnconstrainedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[37/63] [abbrv] Finalize ExecutionGraph state machine and calls

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
index 3d891af..e73102a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/concurrent/SolutionSetUpdateBarrierBroker.java
@@ -16,12 +16,12 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.iterative.concurrent;
 
 /**
- * Broker to hand over {@link SolutionSetUpdateBarrier} from {@link IterationHeadPactTask} to
- * {@link IterationTailPactTask}.
+ * Broker to hand over {@link SolutionSetUpdateBarrier} from 
+ * {@link org.apache.flink.runtime.iterative.task.IterationHeadPactTask} to
+ * {@link org.apache.flink.runtime.iterative.task.IterationTailPactTask}.
  */
 public class SolutionSetUpdateBarrierBroker extends Broker<SolutionSetUpdateBarrier> {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
index 3bd24a6..a45507a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
@@ -33,7 +33,7 @@ import org.apache.flink.runtime.event.job.ManagementEvent;
 import org.apache.flink.runtime.event.job.RecentJobEvent;
 import org.apache.flink.runtime.event.job.VertexEvent;
 import org.apache.flink.runtime.execution.ExecutionListener;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
@@ -74,7 +74,7 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 
 		@Override
 		public void executionStateChanged(JobID jobID, JobVertexID vertexId, int subtask, ExecutionAttemptID executionId,
-				ExecutionState2 newExecutionState, String optionalMessage)
+				ExecutionState newExecutionState, String optionalMessage)
 		{
 			final long timestamp = System.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index 880abd5..d3a920c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -415,18 +415,18 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 	}
 	
 	@Override
-	public void updateTaskExecutionState(TaskExecutionState executionState) throws IOException {
+	public boolean updateTaskExecutionState(TaskExecutionState executionState) throws IOException {
 		Preconditions.checkNotNull(executionState);
 
 
 		final ExecutionGraph eg = this.currentJobs.get(executionState.getJobID());
 		if (eg == null) {
-			LOG.error("Cannot find execution graph for ID " + executionState.getJobID() + " to change state to "
-				+ executionState.getExecutionState());
-			return;
+			LOG.debug("Orphaned execution task: UpdateTaskExecutionState call cannot find execution graph for ID " + executionState.getJobID() +
+					" to change state to " + executionState.getExecutionState());
+			return false;
 		}
 
-		eg.updateState(executionState);
+		return eg.updateState(executionState);
 	}
 	
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
index 06f0eab..54e16b9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
@@ -27,7 +27,7 @@ import java.util.Set;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceDiedException;
@@ -39,7 +39,7 @@ import org.apache.flink.runtime.instance.InstanceListener;
  */
 public class DefaultScheduler implements InstanceListener, SlotAvailablilityListener {
 
-	protected static final Logger LOG = LoggerFactory.getLogger(DefaultScheduler.class);
+	private static final Logger LOG = LoggerFactory.getLogger(DefaultScheduler.class);
 	
 	
 	private final Object globalLock = new Object();
@@ -136,6 +136,8 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 		if (LOG.isDebugEnabled()) {
 			LOG.debug("Scheduling task " + task);
 		}
+		
+		final ExecutionVertex vertex = task.getTaskToExecute().getVertex();
 	
 		synchronized (globalLock) {
 			// 1) If the task has a strict co-schedule hint, obey it, if it has been assigned.
@@ -154,7 +156,7 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 			if (sharingUnit != null) {
 				// see if we can add the task to the current sharing group.
 				SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
-				AllocatedSlot slot = assignment.getSlotForTask(task.getJobVertexId(), task.getTaskVertex());
+				AllocatedSlot slot = assignment.getSlotForTask(vertex.getJobvertexId(), vertex);
 				if (slot != null) {
 					return slot;
 				}
@@ -165,11 +167,13 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 			// we need potentially to loop multiple times, because there may be false positives
 			// in the set-with-available-instances
 			while (true) {
-				Instance instanceToUse = getFreeInstanceForTask(task.getTaskVertex());
+				
+				
+				Instance instanceToUse = getFreeInstanceForTask(task.getTaskToExecute().getVertex());
 			
 				if (instanceToUse != null) {
 					try {
-						AllocatedSlot slot = instanceToUse.allocateSlot(task.getTaskVertex().getJobId());
+						AllocatedSlot slot = instanceToUse.allocateSlot(vertex.getJobId());
 						
 						// if the instance has further available slots, re-add it to the set of available resources.
 						if (instanceToUse.hasResourcesAvailable()) {
@@ -217,7 +221,7 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 	 * @param vertex The task to run. 
 	 * @return The instance to run the vertex on, it {@code null}, if no instance is available.
 	 */
-	protected Instance getFreeInstanceForTask(ExecutionVertex2 vertex) {
+	protected Instance getFreeInstanceForTask(ExecutionVertex vertex) {
 		if (this.instancesWithAvailableResources.isEmpty()) {
 			return null;
 		}
@@ -270,9 +274,10 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 			
 			if (queued != null) {
 				ScheduledUnit task = queued.getTask();
+				ExecutionVertex vertex = task.getTaskToExecute().getVertex();
 				
 				try {
-					AllocatedSlot newSlot = instance.allocateSlot(task.getTaskVertex().getJobId());
+					AllocatedSlot newSlot = instance.allocateSlot(vertex.getJobId());
 					if (newSlot != null) {
 						
 						// success, remove from the task queue and notify the future
@@ -282,8 +287,8 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 								queued.getFuture().setSlot(newSlot);
 							}
 							catch (Throwable t) {
-								LOG.error("Error calling allocation future for task " + task.getTaskVertex().getSimpleName(), t);
-								task.getTaskVertex().fail(t);
+								LOG.error("Error calling allocation future for task " + vertex.getSimpleName(), t);
+								task.getTaskToExecute().fail(t);
 							}
 						}
 					}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
index 2b0de6f..10190f5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
@@ -18,48 +18,48 @@
 
 package org.apache.flink.runtime.jobmanager.scheduler;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 public class ScheduledUnit {
 	
-	private final ExecutionVertex2 taskVertex;
+	private final Execution vertexExecution;
 	
 	private final SlotSharingGroup sharingGroup;
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public ScheduledUnit(ExecutionVertex2 taskVertex) {
+	public ScheduledUnit(Execution taskVertex) {
 		if (taskVertex == null) {
 			throw new NullPointerException();
 		}
 		
-		this.taskVertex = taskVertex;
+		this.vertexExecution = taskVertex;
 		this.sharingGroup = null;
 	}
 	
-	public ScheduledUnit(ExecutionVertex2 taskVertex, SlotSharingGroup sharingUnit) {
+	public ScheduledUnit(Execution taskVertex, SlotSharingGroup sharingUnit) {
 		if (taskVertex == null) {
 			throw new NullPointerException();
 		}
 		
-		this.taskVertex = taskVertex;
+		this.vertexExecution = taskVertex;
 		this.sharingGroup = sharingUnit;
 	}
 	
 	ScheduledUnit() {
-		this.taskVertex = null;
+		this.vertexExecution = null;
 		this.sharingGroup = null;
 	}
 
 	// --------------------------------------------------------------------------------------------
 	
 	public JobVertexID getJobVertexId() {
-		return this.taskVertex.getJobvertexId();
+		return this.vertexExecution.getVertex().getJobvertexId();
 	}
 	
-	public ExecutionVertex2 getTaskVertex() {
-		return taskVertex;
+	public Execution getTaskToExecute() {
+		return vertexExecution;
 	}
 	
 	public SlotSharingGroup getSlotSharingGroup() {
@@ -70,6 +70,6 @@ public class ScheduledUnit {
 	
 	@Override
 	public String toString() {
-		return "{vertex=" + taskVertex.getSimpleName() + ", sharingUnit=" + sharingGroup + '}';
+		return "{vertex=" + vertexExecution.getVertexWithAttempt() + ", sharingUnit=" + sharingGroup + '}';
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
index e7968ed..4599d68 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
@@ -19,17 +19,15 @@
 package org.apache.flink.runtime.jobmanager.scheduler;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Queue;
 import java.util.Set;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -42,9 +40,6 @@ public class SlotSharingGroupAssignment {
 	/** The slots available per vertex type (jid), keyed by instance, to make them locatable */
 	private final Map<JobVertexID, Map<Instance, List<SharedSlot>>> availableSlotsPerJid = new LinkedHashMap<JobVertexID, Map<Instance, List<SharedSlot>>>();
 	
-	/** The tasks that are waiting, per vertex type (jid) */
-	private final Map<JobVertexID, Queue<ExecutionVertex2>> pendingTasks = new HashMap<JobVertexID, Queue<ExecutionVertex2>>();
-	
 	
 	// --------------------------------------------------------------------------------------------
 	
@@ -84,7 +79,7 @@ public class SlotSharingGroupAssignment {
 		}
 	}
 	
-	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex2 vertex) {
+	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex vertex) {
 		synchronized (allSlots) {
 			return getSlotForTaskInternal(jid, vertex.getPreferredLocations());
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
index b086fc1..f4c67df 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
@@ -39,10 +39,11 @@ import org.apache.flink.runtime.event.job.AbstractEvent;
 import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent;
 import org.apache.flink.runtime.event.job.JobEvent;
 import org.apache.flink.runtime.event.job.RecentJobEvent;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -248,20 +249,24 @@ public class JobmanagerInfoServlet extends HttpServlet {
 			if (jobEvent.getJobStatus() == JobStatus.FAILED) {
 				wrt.write("\"failednodes\": [");
 				boolean first = true;
-				for (ExecutionVertex2 vertex : graph.getAllExecutionVertices()) {
-					if (vertex.getExecutionState() == ExecutionState2.FAILED) {
-					if (first) {
-						first = false;
-					} else {
-						wrt.write(",");
+				for (ExecutionVertex vertex : graph.getAllExecutionVertices()) {
+					if (vertex.getExecutionState() == ExecutionState.FAILED) {
+						AllocatedSlot slot = vertex.getCurrentAssignedResource();
+						Throwable failureCause = vertex.getFailureCause();
+						if (slot != null || failureCause != null) {
+							if (first) {
+								first = false;
+							} else {
+								wrt.write(",");
+							}
+							wrt.write("{");
+							wrt.write("\"node\": \"" + (slot == null ? "(none)" : slot.getInstance().getInstanceConnectionInfo().hostname()) + "\",");
+							wrt.write("\"message\": \"" + (failureCause == null ? "" : StringUtils.escapeHtml(ExceptionUtils.stringifyException(failureCause))) + "\"");
+							wrt.write("}");
+						}
 					}
-					wrt.write("{");
-					wrt.write("\"node\": \"" + vertex.getAssignedResource().getInstance().getInstanceConnectionInfo().hostname() + "\",");
-					wrt.write("\"message\": \"" + StringUtils.escapeHtml(ExceptionUtils.stringifyException(vertex.getFailureCause())) + "\"");
-					wrt.write("}");
 				}
-			}
-			wrt.write("],");
+				wrt.write("],");
 			}
 
 			// Serialize ManagementGraph to json
@@ -307,16 +312,16 @@ public class JobmanagerInfoServlet extends HttpServlet {
 				long ended = 0;
 				
 				// Take earliest running state and latest endstate of groupmembers
-				for (ExecutionVertex2 vertex : groupVertex.getTaskVertices()) {
+				for (ExecutionVertex vertex : groupVertex.getTaskVertices()) {
 					
-					long running = vertex.getStateTimestamp(ExecutionState2.RUNNING);
+					long running = vertex.getStateTimestamp(ExecutionState.RUNNING);
 					if (running != 0 && running < started) {
 						started = running;
 					}
 					
-					long finished = vertex.getStateTimestamp(ExecutionState2.FINISHED);
-					long canceled = vertex.getStateTimestamp(ExecutionState2.CANCELED);
-					long failed = vertex.getStateTimestamp(ExecutionState2.FAILED);
+					long finished = vertex.getStateTimestamp(ExecutionState.FINISHED);
+					long canceled = vertex.getStateTimestamp(ExecutionState.CANCELED);
+					long failed = vertex.getStateTimestamp(ExecutionState.FAILED);
 					
 					if(finished != 0 && finished > ended) {
 						ended = finished;
@@ -460,7 +465,7 @@ public class JobmanagerInfoServlet extends HttpServlet {
 			for (ExecutionJobVertex groupVertex : graph.getAllVertices().values()) {
 				
 				int num = 0;
-				for (ExecutionVertex2 vertex : groupVertex.getTaskVertices()) {
+				for (ExecutionVertex vertex : groupVertex.getTaskVertices()) {
 					
 					if(first) {
 						first = false;
@@ -470,14 +475,14 @@ public class JobmanagerInfoServlet extends HttpServlet {
 					wrt.write("\""+jobVertex.getJobVertex()+"-"+num +"\": {");
 					wrt.write("\"vertexid\": \"" + vertex.getJobvertexId() + "\",");
 					wrt.write("\"vertexname\": \"" + vertex + "\",");
-					wrt.write("\"CREATED\": "+ vertex.getStateTimestamp(ExecutionState2.CREATED) + ",");
-					wrt.write("\"SCHEDULED\": "+ vertex.getStateTimestamp(ExecutionState2.SCHEDULED) + ",");
-					wrt.write("\"STARTING\": "+ vertex.getStateTimestamp(ExecutionState2.DEPLOYING) + ",");
-					wrt.write("\"RUNNING\": "+ vertex.getStateTimestamp(ExecutionState2.RUNNING) + ",");
-					wrt.write("\"FINISHED\": "+ vertex.getStateTimestamp(ExecutionState2.FINISHED) + ",");
-					wrt.write("\"CANCELING\": "+ vertex.getStateTimestamp(ExecutionState2.CANCELING) + ",");
-					wrt.write("\"CANCELED\": "+ vertex.getStateTimestamp(ExecutionState2.CANCELED) + ",");
-					wrt.write("\"FAILED\": "+ vertex.getStateTimestamp(ExecutionState2.FAILED) + "");
+					wrt.write("\"CREATED\": "+ vertex.getStateTimestamp(ExecutionState.CREATED) + ",");
+					wrt.write("\"SCHEDULED\": "+ vertex.getStateTimestamp(ExecutionState.SCHEDULED) + ",");
+					wrt.write("\"STARTING\": "+ vertex.getStateTimestamp(ExecutionState.DEPLOYING) + ",");
+					wrt.write("\"RUNNING\": "+ vertex.getStateTimestamp(ExecutionState.RUNNING) + ",");
+					wrt.write("\"FINISHED\": "+ vertex.getStateTimestamp(ExecutionState.FINISHED) + ",");
+					wrt.write("\"CANCELING\": "+ vertex.getStateTimestamp(ExecutionState.CANCELING) + ",");
+					wrt.write("\"CANCELED\": "+ vertex.getStateTimestamp(ExecutionState.CANCELED) + ",");
+					wrt.write("\"FAILED\": "+ vertex.getStateTimestamp(ExecutionState.FAILED) + "");
 					wrt.write("}");
 					
 					num++;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
index 9084010..745a9f8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
@@ -22,9 +22,9 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.executiongraph.IntermediateResult;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.io.network.channels.ChannelType;
@@ -32,14 +32,14 @@ import org.apache.flink.util.StringUtils;
 
 public class JsonFactory {
 
-	public static String toJson(ExecutionVertex2 vertex) {
+	public static String toJson(ExecutionVertex vertex) {
 		StringBuilder json = new StringBuilder("");
 		json.append("{");
 		json.append("\"vertexid\": \"" + vertex.getJobvertexId() + "\",");
 		json.append("\"vertexname\": \"" + StringUtils.escapeHtml(vertex.getSimpleName()) + "\",");
 		json.append("\"vertexstatus\": \"" + vertex.getExecutionState() + "\",");
 		
-		AllocatedSlot slot = vertex.getAssignedResource();
+		AllocatedSlot slot = vertex.getCurrentAssignedResource();
 		String instanceName = slot == null ? "(null)" : slot.getInstance().getInstanceConnectionInfo().hostname();
 		
 		json.append("\"vertexinstancename\": \"" + instanceName + "\"");
@@ -57,17 +57,17 @@ public class JsonFactory {
 		json.append("\"groupmembers\": [");
 		
 		// Count state status of group members
-		Map<ExecutionState2, Integer> stateCounts = new HashMap<ExecutionState2, Integer>();
+		Map<ExecutionState, Integer> stateCounts = new HashMap<ExecutionState, Integer>();
 		
 		// initialize with 0
-		for (ExecutionState2 state : ExecutionState2.values()) {
+		for (ExecutionState state : ExecutionState.values()) {
 			stateCounts.put(state, new Integer(0));
 		}
 		
-		ExecutionVertex2[] vertices = jobVertex.getTaskVertices();
+		ExecutionVertex[] vertices = jobVertex.getTaskVertices();
 		
 		for(int j = 0; j < vertices.length; j++) {
-			ExecutionVertex2 vertex = vertices[j];
+			ExecutionVertex vertex = vertices[j];
 			
 			json.append(toJson(vertex));
 			
@@ -103,7 +103,7 @@ public class JsonFactory {
 		json.append("]");
 		
 		// list number of members for each status
-		for (Map.Entry<ExecutionState2, Integer> stateCount : stateCounts.entrySet()) {
+		for (Map.Entry<ExecutionState, Integer> stateCount : stateCounts.entrySet()) {
 			json.append(",\""+stateCount.getKey()+"\": " + stateCount.getValue());
 		}
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java
index 9c443ad..2cc1377 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/AbstractMutableHashTable.java
@@ -74,14 +74,7 @@ public abstract class AbstractMutableHashTable<T> {
 	// ------------- Accessors -------------
 	
 	public abstract MutableObjectIterator<T> getEntryIterator();
-	
-	/**
-	 * 
-	 * @param probeSideComparator
-	 * @param pairComparator
-	 * @param <PT> The type of the probe side.
-	 * @return
-	 */
+
 	public abstract <PT> AbstractHashTableProber<PT, T> getProber(TypeComparator<PT> probeSideComparator, TypePairComparator<PT, T> pairComparator);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
index 2eaff60..ec41408 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.sort;
 
 import java.io.File;
@@ -754,11 +753,6 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		private final ExceptionHandler<IOException> exceptionHandler;
 
 		/**
-		 * The parent task at whom the thread needs to register.
-		 */
-		private final AbstractInvokable parentTask;
-
-		/**
 		 * The flag marking this thread as alive.
 		 */
 		private volatile boolean alive;
@@ -783,7 +777,6 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 			this.setUncaughtExceptionHandler(this);
 
 			this.queues = queues;
-			this.parentTask = parentTask;
 			this.alive = true;
 		}
 
@@ -1170,7 +1163,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		 * @param exceptionHandler The exception handler to call for all exceptions.
 		 * @param queues The queues used to pass buffers between the threads.
 		 * @param parentTask The task that started this thread. If non-null, it is used to register this thread.
-		 * @param memoryManager The memory manager used to allocate buffers for the readers and writers.
+		 * @param memManager The memory manager used to allocate buffers for the readers and writers.
 		 * @param ioManager The I/I manager used to instantiate readers and writers from.
 		 * @param serializer
 		 * @param comparator
@@ -1449,12 +1442,9 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		 * 
 		 * @param channelIDs The IDs of the sorted runs that need to be merged.
 		 * @param writeBuffers The buffers to be used by the writers.
-		 * @param writeBufferSize The size of the write buffers.
-		 * @param  readMemorySize The amount of memory dedicated to the readers.
+
 		 * @return A list of the IDs of the merged channels.
 		 * @throws IOException Thrown, if the readers or writers encountered an I/O problem.
-		 * @throws MemoryAllocationException Thrown, if the specified memory is insufficient to merge the channels
-		 *                                   or if the memory manager could not provide the requested memory.
 		 */
 		protected final List<ChannelWithBlockCount> mergeChannelList(final List<ChannelWithBlockCount> channelIDs,
 					final List<MemorySegment> allReadBuffers, final List<MemorySegment> writeBuffers)
@@ -1548,7 +1538,6 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		 * @param memory A list containing the memory buffers to be distributed. The buffers are not
 		 *               removed from this list.
 		 * @param numChannels The number of channels for which to allocate buffers. Must not be zero.
-		 * @return A list with all memory segments that were allocated.
 		 */
 		protected final void getSegmentsForReaders(List<List<MemorySegment>> target,
 			List<MemorySegment> memory, int numChannels)
@@ -1586,7 +1575,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		/**
 		 * Adds a channel to the list of channels that are to be removed at shutdown.
 		 * 
-		 * @param s The channel id.
+		 * @param channel The channel id.
 		 */
 		protected void registerChannelToBeRemovedAtShudown(Channel.ID channel) {
 			UnilateralSortMerger.this.channelsToDeleteAtShutdown.add(channel);
@@ -1595,7 +1584,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		/**
 		 * Removes a channel from the list of channels that are to be removed at shutdown.
 		 * 
-		 * @param s The channel id.
+		 * @param channel The channel id.
 		 */
 		protected void unregisterChannelToBeRemovedAtShudown(Channel.ID channel) {
 			UnilateralSortMerger.this.channelsToDeleteAtShutdown.remove(channel);
@@ -1604,7 +1593,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		/**
 		 * Adds a channel reader/writer to the list of channels that are to be removed at shutdown.
 		 * 
-		 * @param s The channel reader/writer.
+		 * @param channel The channel reader/writer.
 		 */
 		protected void registerOpenChannelToBeRemovedAtShudown(BlockChannelAccess<?, ?> channel) {
 			UnilateralSortMerger.this.openChannels.add(channel);
@@ -1613,7 +1602,7 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		/**
 		 * Removes a channel reader/writer from the list of channels that are to be removed at shutdown.
 		 * 
-		 * @param s The channel reader/writer.
+		 * @param channel The channel reader/writer.
 		 */
 		protected void unregisterOpenChannelToBeRemovedAtShudown(BlockChannelAccess<?, ?> channel) {
 			UnilateralSortMerger.this.openChannels.remove(channel);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
index 4a14e13..e11fc78 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.profiling.impl;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.runtime.execution.ExecutionListener;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
@@ -43,7 +43,7 @@ public class EnvironmentListenerImpl implements ExecutionListener {
 
 
 	@Override
-	public void executionStateChanged(JobID jobID, JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, ExecutionState2 newExecutionState, String optionalMessage) {
+	public void executionStateChanged(JobID jobID, JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, ExecutionState newExecutionState, String optionalMessage) {
 
 		switch (newExecutionState) {
 		case RUNNING:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
index 16c2bf5..8e85f2f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
@@ -25,7 +25,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
@@ -58,8 +58,8 @@ public class JobProfilingData {
 
 	public boolean addIfInstanceIsAllocatedByJob(InternalInstanceProfilingData instanceProfilingData) {
 
-		for (ExecutionVertex2 executionVertex : this.executionGraph.getAllExecutionVertices()) {
-			AllocatedSlot slot = executionVertex.getAssignedResource();
+		for (ExecutionVertex executionVertex : this.executionGraph.getAllExecutionVertices()) {
+			AllocatedSlot slot = executionVertex.getCurrentAssignedResource();
 			if (slot != null && slot.getInstance().getInstanceConnectionInfo().equals(
 					instanceProfilingData.getInstanceConnectionInfo()))
 			{
@@ -75,8 +75,8 @@ public class JobProfilingData {
 
 		final Set<Instance> tempSet = new HashSet<Instance>();
 		
-		for (ExecutionVertex2 executionVertex : this.executionGraph.getAllExecutionVertices()) {
-			AllocatedSlot slot = executionVertex.getAssignedResource();
+		for (ExecutionVertex executionVertex : this.executionGraph.getAllExecutionVertices()) {
+			AllocatedSlot slot = executionVertex.getCurrentAssignedResource();
 			if (slot != null) {
 				tempSet.add(slot.getInstance());
 			}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
index 9c46ac5..26fd095 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
@@ -53,12 +53,13 @@ public interface JobManagerProtocol extends VersionedProtocol {
 	InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) throws IOException;
 
 	/**
-	 * Reports an update of a task's execution state to the job manager.
+	 * Reports an update of a task's execution state to the job manager. This method returns true, if the state was
+	 * correctly registered. It it returns false, the calling task manager should cancel its execution of the task.
 	 * 
-	 * @param taskExecutionState
-	 *        the new task execution state
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
+	 * @param taskExecutionState The new task execution state.
+	 * @return True if everything is all right, false if the caller should cancel the task execution.
+	 * 
+	 * @throws IOException Thrown, if an error occurs during this remote procedure call
 	 */
-	void updateTaskExecutionState(TaskExecutionState taskExecutionState) throws IOException;
+	boolean updateTaskExecutionState(TaskExecutionState taskExecutionState) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
index 1800e3f..a71f760 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
@@ -26,7 +26,6 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileReques
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileResponse;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheUpdate;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
 
 /**
@@ -38,7 +37,7 @@ public interface TaskOperationProtocol extends VersionedProtocol {
 
 	TaskOperationResult submitTask(TaskDeploymentDescriptor task) throws IOException;
 
-	TaskOperationResult cancelTask(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId) throws IOException;
+	TaskOperationResult cancelTask(ExecutionAttemptID executionId) throws IOException;
 
 	LibraryCacheProfileResponse getLibraryCacheProfile(LibraryCacheProfileRequest request) throws IOException;
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 7e48d7c..7b692e3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -27,7 +27,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.execution.ExecutionListener;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
@@ -39,8 +39,8 @@ import org.apache.flink.util.ExceptionUtils;
 public final class Task {
 
 	/** For atomic state updates */
-	private static final AtomicReferenceFieldUpdater<Task, ExecutionState2> STATE_UPDATER = 
-			AtomicReferenceFieldUpdater.newUpdater(Task.class, ExecutionState2.class, "executionState");
+	private static final AtomicReferenceFieldUpdater<Task, ExecutionState> STATE_UPDATER = 
+			AtomicReferenceFieldUpdater.newUpdater(Task.class, ExecutionState.class, "executionState");
 			
 	/** The log object used for debugging. */
 	private static final Logger LOG = LoggerFactory.getLogger(Task.class);
@@ -68,7 +68,7 @@ public final class Task {
 	private volatile RuntimeEnvironment environment;
 	
 	/** The current execution state of the task */
-	private volatile ExecutionState2 executionState = ExecutionState2.DEPLOYING;
+	private volatile ExecutionState executionState = ExecutionState.DEPLOYING;
 
 	// --------------------------------------------------------------------------------------------	
 	
@@ -135,7 +135,7 @@ public final class Task {
 	 * 
 	 * @return the current execution state of the task
 	 */
-	public ExecutionState2 getExecutionState() {
+	public ExecutionState getExecutionState() {
 		return this.executionState;
 	}
 	
@@ -148,8 +148,8 @@ public final class Task {
 	}
 	
 	public boolean isCanceled() {
-		return executionState == ExecutionState2.CANCELING ||
-				executionState == ExecutionState2.CANCELED;
+		return executionState == ExecutionState.CANCELING ||
+				executionState == ExecutionState.CANCELED;
 	}
 	
 	public String getTaskName() {
@@ -172,9 +172,9 @@ public final class Task {
 	 * @return True, if the task correctly enters the state FINISHED.
 	 */
 	public boolean markAsFinished() {
-		if (STATE_UPDATER.compareAndSet(this, ExecutionState2.RUNNING, ExecutionState2.FINISHED)) {
-			notifyObservers(ExecutionState2.FINISHED, null);
-			taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.FINISHED, null);
+		if (STATE_UPDATER.compareAndSet(this, ExecutionState.RUNNING, ExecutionState.FINISHED)) {
+			notifyObservers(ExecutionState.FINISHED, null);
+			taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.FINISHED, null);
 			return true;
 		} else {
 			return false;
@@ -183,16 +183,15 @@ public final class Task {
 	
 	public void markFailed(Throwable error) {
 		while (true) {
-			ExecutionState2 current = this.executionState;
+			ExecutionState current = this.executionState;
 			
-			if (current == ExecutionState2.CANCELED || current == ExecutionState2.CANCELING) {
+			if (current == ExecutionState.CANCELED || current == ExecutionState.CANCELING) {
 				return;
 			}
 			
-			if (STATE_UPDATER.compareAndSet(this, current, ExecutionState2.FAILED)) {
-				String message = ExceptionUtils.stringifyException(error);
-				notifyObservers(ExecutionState2.FAILED, message);
-				taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.FAILED, message);
+			if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) {
+				notifyObservers(ExecutionState.FAILED, ExceptionUtils.stringifyException(error));
+				taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.FAILED, error);
 				return;
 			}
 		}
@@ -200,27 +199,27 @@ public final class Task {
 	
 	public void cancelExecution() {
 		while (true) {
-			ExecutionState2 current = this.executionState;
+			ExecutionState current = this.executionState;
 			
 			// if the task is already canceled (or canceling) or finished, then we
 			// need not do anything
-			if (current == ExecutionState2.FINISHED || current == ExecutionState2.CANCELED ||
-					current == ExecutionState2.CANCELING) {
+			if (current == ExecutionState.FINISHED || current == ExecutionState.CANCELED ||
+					current == ExecutionState.CANCELING) {
 				return;
 			}
 			
-			if (current == ExecutionState2.DEPLOYING) {
+			if (current == ExecutionState.DEPLOYING) {
 				// directly set to canceled
-				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState2.CANCELED)) {
-					notifyObservers(ExecutionState2.CANCELED, null);
-					taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.CANCELED, null);
+				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.CANCELED)) {
+					notifyObservers(ExecutionState.CANCELED, null);
+					taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.CANCELED, null);
 					return;
 				}
 			}
-			else if (current == ExecutionState2.RUNNING) {
+			else if (current == ExecutionState.RUNNING) {
 				// go to canceling and perform the actual task canceling
-				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState2.CANCELING)) {
-					notifyObservers(ExecutionState2.CANCELING, null);
+				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.CANCELING)) {
+					notifyObservers(ExecutionState.CANCELING, null);
 					try {
 						this.environment.cancelExecution();
 					} catch (Throwable e) {
@@ -237,9 +236,9 @@ public final class Task {
 	}
 	
 	public void cancelingDone() {
-		if (STATE_UPDATER.compareAndSet(this, ExecutionState2.CANCELING, ExecutionState2.CANCELED)) {
-			notifyObservers(ExecutionState2.CANCELED, null);
-			taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.CANCELED, null);
+		if (STATE_UPDATER.compareAndSet(this, ExecutionState.CANCELING, ExecutionState.CANCELED)) {
+			notifyObservers(ExecutionState.CANCELED, null);
+			taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.CANCELED, null);
 		}
 	}
 
@@ -247,7 +246,7 @@ public final class Task {
 	 * Starts the execution of this task.
 	 */
 	public boolean startExecution() {
-		if (STATE_UPDATER.compareAndSet(this, ExecutionState2.DEPLOYING, ExecutionState2.RUNNING)) {
+		if (STATE_UPDATER.compareAndSet(this, ExecutionState.DEPLOYING, ExecutionState.RUNNING)) {
 			final Thread thread = this.environment.getExecutingThread();
 			thread.start();
 			return true;
@@ -314,7 +313,7 @@ public final class Task {
 		this.executionListeners.remove(listener);
 	}
 	
-	private void notifyObservers(ExecutionState2 newState, String message) {
+	private void notifyObservers(ExecutionState newState, String message) {
 		for (ExecutionListener listener : this.executionListeners) {
 			try {
 				listener.executionStateChanged(jobId, vertexId, subtaskIndex, executionId, newState, message);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
index 8ab67bc..6e0a61d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
@@ -18,15 +18,18 @@
 
 package org.apache.flink.runtime.taskmanager;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.types.StringValue;
 
 /**
  * This class represents an update about a task's execution state.
@@ -39,10 +42,15 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 
 	private ExecutionAttemptID executionId;
 
-	private ExecutionState2 executionState;
+	private ExecutionState executionState;
 
-	private String description;
+	private Throwable error;
 
+	
+	public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, ExecutionState executionState) {
+		this(jobID, executionId, executionState, null);
+	}
+	
 	/**
 	 * Creates a new task execution state.
 	 * 
@@ -52,10 +60,10 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 	 *        the ID of the task execution whose state is to be reported
 	 * @param executionState
 	 *        the execution state to be reported
-	 * @param description
-	 *        an optional description
+	 * @param error
+	 *        an optional error
 	 */
-	public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, ExecutionState2 executionState, String description) {
+	public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, ExecutionState executionState, Throwable error) {
 		if (jobID == null || executionId == null || executionState == null) {
 			throw new NullPointerException();
 		}
@@ -63,7 +71,7 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 		this.jobID = jobID;
 		this.executionId = executionId;
 		this.executionState = executionState;
-		this.description = description;
+		this.error = error;
 	}
 
 	/**
@@ -76,13 +84,8 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 
 	// --------------------------------------------------------------------------------------------
 	
-	/**
-	 * Returns the description of this task execution state.
-	 * 
-	 * @return the description of this task execution state or <code>null</code> if there is no description available
-	 */
-	public String getDescription() {
-		return this.description;
+	public Throwable getError() {
+		return this.error;
 	}
 
 	/**
@@ -99,7 +102,7 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 	 * 
 	 * @return the new execution state of the task
 	 */
-	public ExecutionState2 getExecutionState() {
+	public ExecutionState getExecutionState() {
 		return this.executionState;
 	}
 
@@ -118,12 +121,24 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 	public void read(DataInputView in) throws IOException {
 		this.jobID.read(in);
 		this.executionId.read(in);
-		this.executionState = ExecutionState2.values()[in.readInt()];
-
-		if (in.readBoolean()) {
-			this.description = StringValue.readString(in);
-		} else {
-			this.description = null;
+		this.executionState = ExecutionState.values()[in.readInt()];
+
+		// read the exception
+		int errorDataLen = in.readInt();
+		if (errorDataLen > 0) {
+			byte[] data = new byte[errorDataLen];
+			in.readFully(data);
+			try {
+				ByteArrayInputStream bis = new ByteArrayInputStream(data);
+				ObjectInputStream ois = new ObjectInputStream(bis);
+				this.error = (Throwable) ois.readObject();
+				ois.close();
+			} catch (Throwable t) {
+				this.error = new Exception("An error occurred, but the exception could not be transfered through the RPC");
+			}
+		}
+		else {
+			this.error = null;
 		}
 	}
 
@@ -133,11 +148,20 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 		this.executionId.write(out);
 		out.writeInt(this.executionState.ordinal());
 
-		if (description != null) {
-			out.writeBoolean(true);
-			StringValue.writeString(description, out);
-		} else {
-			out.writeBoolean(false);
+		// transfer the exception
+		if (this.error != null) {
+			ByteArrayOutputStream baos = new ByteArrayOutputStream();
+			ObjectOutputStream oos = new ObjectOutputStream(baos);
+			oos.writeObject(error);
+			oos.flush();
+			oos.close();
+			
+			byte[] data = baos.toByteArray();
+			out.writeInt(data.length);
+			out.write(data);
+		}
+		else {
+			out.writeInt(0);
 		}
 	}
 	
@@ -150,8 +174,10 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 			return other.jobID.equals(this.jobID) &&
 					other.executionId.equals(this.executionId) &&
 					other.executionState == this.executionState &&
-					(other.description == null ? this.description == null :
-						(this.description != null && other.description.equals(this.description)));
+					(other.error == null ? this.error == null :
+						(this.error != null && other.error.getClass() == this.error.getClass()));
+			
+			// NOTE: exception equality does not work, so we can only check for same error class
 		}
 		else {
 			return false;
@@ -165,7 +191,7 @@ public class TaskExecutionState implements IOReadableWritable , java.io.Serializ
 	
 	@Override
 	public String toString() {
-		return String.format("TaskState jobId=%s, executionId=%s, state=%s, description=%s", 
-				jobID, executionId, executionState, description == null ? "(null)" : description);
+		return String.format("TaskState jobId=%s, executionId=%s, state=%s, error=%s", 
+				jobID, executionId, executionState, error == null ? "(null)" : error.getClass().getName() + ": " + error.getMessage());
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
index f2259c4..fd9f10d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
@@ -61,7 +61,7 @@ import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.protocols.VersionedProtocol;
 import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileRequest;
@@ -501,17 +501,21 @@ public class TaskManager implements TaskOperationProtocol {
 		return Collections.unmodifiableMap(this.runningTasks);
 	}
 	
+	public ChannelManager getChannelManager() {
+		return channelManager;
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	//  Task Operation
 	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public TaskOperationResult cancelTask(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId) throws IOException {
+	public TaskOperationResult cancelTask(ExecutionAttemptID executionId) throws IOException {
 
 		final Task task = this.runningTasks.get(executionId);
 
 		if (task == null) {
-			return new TaskOperationResult(vertexId, subtaskIndex, executionId, false, "No task with that execution ID was found.");
+			return new TaskOperationResult(executionId, false, "No task with that execution ID was found.");
 		}
 
 		// Pass call to executor service so IPC thread can return immediately
@@ -524,7 +528,7 @@ public class TaskManager implements TaskOperationProtocol {
 		this.executorService.execute(r);
 
 		// return success
-		return new TaskOperationResult(vertexId, subtaskIndex, executionId, true);
+		return new TaskOperationResult(executionId, true);
 	}
 
 
@@ -580,7 +584,7 @@ public class TaskManager implements TaskOperationProtocol {
 				}
 			
 				success = true;
-				return new TaskOperationResult(vertexId, taskIndex, executionId, true);
+				return new TaskOperationResult(executionId, true);
 			}
 			finally {
 				if (!success) {
@@ -604,7 +608,7 @@ public class TaskManager implements TaskOperationProtocol {
 				}
 			}
 			
-			return new TaskOperationResult(vertexId, taskIndex, executionId, false, ExceptionUtils.stringifyException(t));
+			return new TaskOperationResult(executionId, false, ExceptionUtils.stringifyException(t));
 		}
 	}
 
@@ -647,12 +651,12 @@ public class TaskManager implements TaskOperationProtocol {
 		}
 	}
 
-	public void notifyExecutionStateChange(JobID jobID, ExecutionAttemptID executionId, ExecutionState2 newExecutionState, String optionalDescription) {
+	public void notifyExecutionStateChange(JobID jobID, ExecutionAttemptID executionId, ExecutionState newExecutionState, Throwable optionalError) {
 		
 		// Get lock on the jobManager object and propagate the state change
 		boolean success = false;
 		try {
-			this.jobManager.updateTaskExecutionState(new TaskExecutionState(jobID, executionId, newExecutionState, optionalDescription));
+			success = this.jobManager.updateTaskExecutionState(new TaskExecutionState(jobID, executionId, newExecutionState, optionalError));
 		}
 		catch (Throwable t) {
 			String msg = "Error sending task state update to JobManager.";
@@ -662,10 +666,9 @@ public class TaskManager implements TaskOperationProtocol {
 		finally {
 			// in case of a failure, or when the tasks is in a finished state, then unregister the
 			// task (free all buffers, remove all channels, task-specific class loaders, etc...)
-			if (!success || newExecutionState == ExecutionState2.FINISHED || newExecutionState == ExecutionState2.CANCELED
-					|| newExecutionState == ExecutionState2.FAILED)
+			if (!success || newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED
+					|| newExecutionState == ExecutionState.FAILED)
 			{
-				
 				unregisterTask(executionId);
 			}
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java
index f0f00a7..f1846a8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java
@@ -24,17 +24,15 @@ import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.types.StringValue;
+import org.apache.flink.util.StringUtils;
 
 import com.google.common.base.Preconditions;
 
 
-public class TaskOperationResult implements IOReadableWritable {
-
-	private JobVertexID vertexId;
+public class TaskOperationResult implements IOReadableWritable, java.io.Serializable {
 	
-	private int subtaskIndex;
+	private static final long serialVersionUID = -3852292420229699888L;
+
 	
 	private ExecutionAttemptID executionId;
 	
@@ -44,32 +42,21 @@ public class TaskOperationResult implements IOReadableWritable {
 
 
 	public TaskOperationResult() {
-		this(new JobVertexID(), -1, new ExecutionAttemptID(), false);
+		this(new ExecutionAttemptID(), false);
 	}
 	
-	public TaskOperationResult(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, boolean success) {
-		this(vertexId, subtaskIndex, executionId, success, null);
+	public TaskOperationResult(ExecutionAttemptID executionId, boolean success) {
+		this(executionId, success, null);
 	}
 	
-	public TaskOperationResult(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, boolean success, String description) {
-		Preconditions.checkNotNull(vertexId);
+	public TaskOperationResult(ExecutionAttemptID executionId, boolean success, String description) {
 		Preconditions.checkNotNull(executionId);
 		
-		this.vertexId = vertexId;
-		this.subtaskIndex = subtaskIndex;
 		this.executionId = executionId;
 		this.success = success;
 		this.description = description;
 	}
-
-
-	public JobVertexID getVertexId() {
-		return vertexId;
-	}
 	
-	public int getSubtaskIndex() {
-		return subtaskIndex;
-	}
 	
 	public ExecutionAttemptID getExecutionId() {
 		return executionId;
@@ -82,29 +69,32 @@ public class TaskOperationResult implements IOReadableWritable {
 	public String getDescription() {
 		return description;
 	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Serialization
+	// --------------------------------------------------------------------------------------------
 
 	@Override
 	public void read(DataInputView in) throws IOException {
-		this.vertexId.read(in);
-		this.subtaskIndex = in.readInt();
+		this.executionId.read(in);
 		this.success = in.readBoolean();
-		
-		if (in.readBoolean()) {
-			this.description = StringValue.readString(in);
-		}
+		this.description = StringUtils.readNullableString(in);
 	}
 
 	@Override
 	public void write(DataOutputView out) throws IOException {
-		this.vertexId.write(out);
-		out.writeInt(subtaskIndex);
+		this.executionId.write(out);
 		out.writeBoolean(success);
-		
-		if (description != null) {
-			out.writeBoolean(true);
-			StringValue.writeString(description, out);
-		} else {
-			out.writeBoolean(false);
-		}
+		StringUtils.writeNullableString(description, out);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Utilities
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return String.format("TaskOperationResult %s [%s]%s", executionId, 
+				success ? "SUCCESS" : "FAILED", description == null ? "" : " - " + description);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
index 9ae6e2b..8ba0576 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
@@ -30,7 +30,6 @@ import org.apache.flink.runtime.event.job.JobEvent;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.apache.flink.runtime.util.SerializableArrayList;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java
index ff6732b..2d7ef37 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java
@@ -23,7 +23,7 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
@@ -43,8 +43,8 @@ public class EventsTest {
 				JobVertexID jid = new JobVertexID();
 				ExecutionAttemptID eid = new ExecutionAttemptID();
 				
-				ExecutionStateChangeEvent e1 = new ExecutionStateChangeEvent(429345231796596L, jid, 17, eid, ExecutionState2.CANCELING);
-				ExecutionStateChangeEvent e2 = new ExecutionStateChangeEvent(429345231796596L, jid, 17, eid, ExecutionState2.CANCELING);
+				ExecutionStateChangeEvent e1 = new ExecutionStateChangeEvent(429345231796596L, jid, 17, eid, ExecutionState.CANCELING);
+				ExecutionStateChangeEvent e2 = new ExecutionStateChangeEvent(429345231796596L, jid, 17, eid, ExecutionState.CANCELING);
 				
 				assertTrue(e1.equals(e2));
 				assertEquals(e1.hashCode(), e2.hashCode());
@@ -89,11 +89,11 @@ public class EventsTest {
 				JobVertexID jid = new JobVertexID();
 				ExecutionAttemptID eid = new ExecutionAttemptID();
 				
-				VertexEvent e1 = new VertexEvent(64619276017401234L, jid, "peter", 44, 13, eid, ExecutionState2.DEPLOYING, "foo");
-				VertexEvent e2 = new VertexEvent(64619276017401234L, jid, "peter", 44, 13, eid, ExecutionState2.DEPLOYING, "foo");
+				VertexEvent e1 = new VertexEvent(64619276017401234L, jid, "peter", 44, 13, eid, ExecutionState.DEPLOYING, "foo");
+				VertexEvent e2 = new VertexEvent(64619276017401234L, jid, "peter", 44, 13, eid, ExecutionState.DEPLOYING, "foo");
 				
-				VertexEvent e3 = new VertexEvent(64619276017401234L, jid, null, 44, 13, eid, ExecutionState2.DEPLOYING, null);
-				VertexEvent e4 = new VertexEvent(64619276017401234L, jid, null, 44, 13, eid, ExecutionState2.DEPLOYING, null);
+				VertexEvent e3 = new VertexEvent(64619276017401234L, jid, null, 44, 13, eid, ExecutionState.DEPLOYING, null);
+				VertexEvent e4 = new VertexEvent(64619276017401234L, jid, null, 44, 13, eid, ExecutionState.DEPLOYING, null);
 				
 				assertTrue(e1.equals(e2));
 				assertTrue(e3.equals(e4));
@@ -119,7 +119,7 @@ public class EventsTest {
 			JobVertexID vid = new JobVertexID();
 			ExecutionAttemptID eid = new ExecutionAttemptID();
 			
-			ExecutionStateChangeEvent esce = new ExecutionStateChangeEvent(429345231796596L, vid, 17, eid, ExecutionState2.CANCELING);
+			ExecutionStateChangeEvent esce = new ExecutionStateChangeEvent(429345231796596L, vid, 17, eid, ExecutionState.CANCELING);
 
 			JobEvent je1 = new JobEvent(429345231796596L, JobStatus.CANCELED, "testmessage");
 			JobEvent je2 = new JobEvent(237217579123412431L, JobStatus.RUNNING, null);
@@ -127,8 +127,8 @@ public class EventsTest {
 			RecentJobEvent rce1 = new RecentJobEvent(jid, "some name", JobStatus.FAILED, false, 634563546, 546734734672572457L);
 			RecentJobEvent rce2 = new RecentJobEvent(jid, null, JobStatus.RUNNING, false, 42364716239476L, 127843618273607L);
 
-			VertexEvent ve1 = new VertexEvent(64619276017401234L, vid, "peter", 44, 13, eid, ExecutionState2.DEPLOYING, "foo");
-			VertexEvent ve2 = new VertexEvent(64619276017401234L, vid, null, 44, 13, eid, ExecutionState2.DEPLOYING, null);
+			VertexEvent ve1 = new VertexEvent(64619276017401234L, vid, "peter", 44, 13, eid, ExecutionState.DEPLOYING, "foo");
+			VertexEvent ve2 = new VertexEvent(64619276017401234L, vid, null, 44, 13, eid, ExecutionState.DEPLOYING, null);
 			
 			assertEquals(esce, CommonTestUtils.createCopyWritable(esce));
 			assertEquals(je1, CommonTestUtils.createCopyWritable(je1));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java
index 3dd15f6..2095365 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/task/TaskEventTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.task;
 
 import static org.junit.Assert.assertEquals;
@@ -27,7 +26,6 @@ import java.io.IOException;
 import java.util.Iterator;
 
 import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
index b6f532e..642af31 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
@@ -20,14 +20,12 @@ package org.apache.flink.runtime.executiongraph;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
-
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
-import org.junit.BeforeClass;
 import org.junit.Test;
-
 import org.mockito.Matchers;
 
 import java.util.ArrayList;
@@ -40,6 +38,7 @@ import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
 import org.apache.flink.core.io.InputSplitSource;
 import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
@@ -47,19 +46,12 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.util.LogUtils;
 
 /**
  * This class contains test concerning the correct conversion from {@link JobGraph} to {@link ExecutionGraph} objects.
  */
 public class ExecutionGraphConstructionTest {
 	
-	@BeforeClass
-	public static void setLogLevel() {
-		LogUtils.initializeDefaultTestConsoleLogger();
-	}
-
-	
 	/**
 	 * Creates a JobGraph of the following form:
 	 * 
@@ -260,7 +252,7 @@ public class ExecutionGraphConstructionTest {
 			assertEquals(v1.getParallelism(), e1.getTaskVertices().length);
 			
 			int num = 0;
-			for (ExecutionVertex2 ev : e1.getTaskVertices()) {
+			for (ExecutionVertex ev : e1.getTaskVertices()) {
 				assertEquals(jobId, ev.getJobId());
 				assertEquals(v1.getID(), ev.getJobvertexId());
 				
@@ -268,6 +260,8 @@ public class ExecutionGraphConstructionTest {
 				assertEquals(num++, ev.getParallelSubtaskIndex());
 				
 				assertEquals(0, ev.getNumberOfInputs());
+				
+				assertTrue(ev.getStateTimestamp(ExecutionState.CREATED) > 0);
 			}
 		}
 		
@@ -285,7 +279,7 @@ public class ExecutionGraphConstructionTest {
 			assertEquals(v2.getParallelism(), e2.getTaskVertices().length);
 			
 			int num = 0;
-			for (ExecutionVertex2 ev : e2.getTaskVertices()) {
+			for (ExecutionVertex ev : e2.getTaskVertices()) {
 				assertEquals(jobId, ev.getJobId());
 				assertEquals(v2.getID(), ev.getJobvertexId());
 				
@@ -293,11 +287,11 @@ public class ExecutionGraphConstructionTest {
 				assertEquals(num++, ev.getParallelSubtaskIndex());
 				
 				assertEquals(1, ev.getNumberOfInputs());
-				ExecutionEdge2[] inputs = ev.getInputEdges(0);
+				ExecutionEdge[] inputs = ev.getInputEdges(0);
 				assertEquals(v1.getParallelism(), inputs.length);
 				
 				int sumOfPartitions = 0;
-				for (ExecutionEdge2 inEdge : inputs) {
+				for (ExecutionEdge inEdge : inputs) {
 					assertEquals(0,inEdge.getInputNum());
 					sumOfPartitions += inEdge.getSource().getPartition();
 				}
@@ -322,7 +316,7 @@ public class ExecutionGraphConstructionTest {
 			assertEquals(v3.getParallelism(), e3.getTaskVertices().length);
 			
 			int num = 0;
-			for (ExecutionVertex2 ev : e3.getTaskVertices()) {
+			for (ExecutionVertex ev : e3.getTaskVertices()) {
 				assertEquals(jobId, ev.getJobId());
 				assertEquals(v3.getID(), ev.getJobvertexId());
 				
@@ -346,7 +340,7 @@ public class ExecutionGraphConstructionTest {
 			assertEquals(v4.getParallelism(), e4.getTaskVertices().length);
 			
 			int num = 0;
-			for (ExecutionVertex2 ev : e4.getTaskVertices()) {
+			for (ExecutionVertex ev : e4.getTaskVertices()) {
 				assertEquals(jobId, ev.getJobId());
 				assertEquals(v4.getID(), ev.getJobvertexId());
 				
@@ -356,11 +350,11 @@ public class ExecutionGraphConstructionTest {
 				assertEquals(2, ev.getNumberOfInputs());
 				// first input
 				{
-					ExecutionEdge2[] inputs = ev.getInputEdges(0);
+					ExecutionEdge[] inputs = ev.getInputEdges(0);
 					assertEquals(v2.getParallelism(), inputs.length);
 					
 					int sumOfPartitions = 0;
-					for (ExecutionEdge2 inEdge : inputs) {
+					for (ExecutionEdge inEdge : inputs) {
 						assertEquals(0, inEdge.getInputNum());
 						sumOfPartitions += inEdge.getSource().getPartition();
 					}
@@ -369,11 +363,11 @@ public class ExecutionGraphConstructionTest {
 				}
 				// second input
 				{
-					ExecutionEdge2[] inputs = ev.getInputEdges(1);
+					ExecutionEdge[] inputs = ev.getInputEdges(1);
 					assertEquals(v3.getParallelism(), inputs.length);
 					
 					int sumOfPartitions = 0;
-					for (ExecutionEdge2 inEdge : inputs) {
+					for (ExecutionEdge inEdge : inputs) {
 						assertEquals(1, inEdge.getInputNum());
 						sumOfPartitions += inEdge.getSource().getPartition();
 					}
@@ -395,7 +389,7 @@ public class ExecutionGraphConstructionTest {
 			assertEquals(v5.getParallelism(), e5.getTaskVertices().length);
 			
 			int num = 0;
-			for (ExecutionVertex2 ev : e5.getTaskVertices()) {
+			for (ExecutionVertex ev : e5.getTaskVertices()) {
 				assertEquals(jobId, ev.getJobId());
 				assertEquals(v5.getID(), ev.getJobvertexId());
 				
@@ -405,11 +399,11 @@ public class ExecutionGraphConstructionTest {
 				assertEquals(2, ev.getNumberOfInputs());
 				// first input
 				{
-					ExecutionEdge2[] inputs = ev.getInputEdges(0);
+					ExecutionEdge[] inputs = ev.getInputEdges(0);
 					assertEquals(v4.getParallelism(), inputs.length);
 					
 					int sumOfPartitions = 0;
-					for (ExecutionEdge2 inEdge : inputs) {
+					for (ExecutionEdge inEdge : inputs) {
 						assertEquals(0, inEdge.getInputNum());
 						sumOfPartitions += inEdge.getSource().getPartition();
 					}
@@ -418,11 +412,11 @@ public class ExecutionGraphConstructionTest {
 				}
 				// second input
 				{
-					ExecutionEdge2[] inputs = ev.getInputEdges(1);
+					ExecutionEdge[] inputs = ev.getInputEdges(1);
 					assertEquals(v3.getParallelism(), inputs.length);
 					
 					int sumOfPartitions = 0;
-					for (ExecutionEdge2 inEdge : inputs) {
+					for (ExecutionEdge inEdge : inputs) {
 						assertEquals(1, inEdge.getInputNum());
 						sumOfPartitions += inEdge.getSource().getPartition();
 					}
@@ -567,4 +561,41 @@ public class ExecutionGraphConstructionTest {
 			fail(e.getMessage());
 		}
 	}
+	
+	@Test
+	public void testMoreThanOneConsumerForIntermediateResult() {
+		try {
+			final JobID jobId = new JobID();
+			final String jobName = "Test Job Sample Name";
+			final Configuration cfg = new Configuration();
+			
+			AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+			AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+			AbstractJobVertex v3 = new AbstractJobVertex("vertex3");
+			
+			v1.setParallelism(5);
+			v2.setParallelism(7);
+			v3.setParallelism(2);
+
+			IntermediateDataSet result = v1.createAndAddResultDataSet();
+			v2.connectDataSetAsInput(result, DistributionPattern.BIPARTITE);
+			v3.connectDataSetAsInput(result, DistributionPattern.BIPARTITE);
+			
+			List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3));
+
+			ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+
+			try {
+				eg.attachJobGraph(ordered);
+				fail("Should not be possible");
+			}
+			catch (RuntimeException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index 9705dcd..ac76623 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -19,11 +19,9 @@
 package org.apache.flink.runtime.executiongraph;
 
 import static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance;
-
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
-
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.spy;
@@ -31,27 +29,32 @@ import static org.mockito.Mockito.doAnswer;
 
 import java.util.Arrays;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
 import org.apache.flink.runtime.operators.RegularPactTask;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
+
 import org.junit.Test;
+
 import org.mockito.Matchers;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
 public class ExecutionGraphDeploymentTest {
-
+	
 	@Test
 	public void testBuildDeploymentDescriptor() {
 		try {
@@ -97,7 +100,7 @@ public class ExecutionGraphDeploymentTest {
 			eg.attachJobGraph(ordered);
 			
 			ExecutionJobVertex ejv = eg.getAllVertices().get(jid2);
-			ExecutionVertex2 vertex = ejv.getTaskVertices()[3];
+			ExecutionVertex vertex = ejv.getTaskVertices()[3];
 			
 			// just some reference (needs not be atomic)
 			final AtomicReference<TaskDeploymentDescriptor> reference = new AtomicReference<TaskDeploymentDescriptor>();
@@ -107,18 +110,21 @@ public class ExecutionGraphDeploymentTest {
 			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenAnswer(new Answer<TaskOperationResult>() {
 				@Override
 				public TaskOperationResult answer(InvocationOnMock invocation) {
-					final TaskDeploymentDescriptor parameter = (TaskDeploymentDescriptor) invocation.getArguments()[0];
-					reference.set(parameter);
-					return new TaskOperationResult(jid2, 0, true);
+					final TaskDeploymentDescriptor tdd = (TaskDeploymentDescriptor) invocation.getArguments()[0];
+					reference.set(tdd);
+					return new TaskOperationResult(tdd.getExecutionId(), true);
 				}
 			});
 			
 			final Instance instance = getInstance(taskManager);
 			final AllocatedSlot slot = instance.allocateSlot(jobId);
 			
-			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
+			
+			LibraryCacheManager.register(jobId, new String[0]);
 			vertex.deployToSlot(slot);
-			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
 			
 			TaskDeploymentDescriptor descr = reference.get();
 			assertNotNull(descr);
@@ -142,4 +148,128 @@ public class ExecutionGraphDeploymentTest {
 			fail(e.getMessage());
 		}
 	}
+	
+	@Test
+	public void testRegistrationOfExecutionsFinishing() {
+		try {
+			Map<ExecutionAttemptID, Execution> executions = setupExecution(7650, 2350);
+			
+			for (Execution e : executions.values()) {
+				e.markFinished();
+			}
+			
+			assertEquals(0, executions.size());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testRegistrationOfExecutionsFailing() {
+		try {
+			Map<ExecutionAttemptID, Execution> executions = setupExecution(7, 6);
+			
+			for (Execution e : executions.values()) {
+				e.markFailed(null);
+			}
+			
+			assertEquals(0, executions.size());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testRegistrationOfExecutionsFailedExternally() {
+		try {
+			Map<ExecutionAttemptID, Execution> executions = setupExecution(7, 6);
+			
+			for (Execution e : executions.values()) {
+				e.fail(null);
+			}
+			
+			assertEquals(0, executions.size());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testRegistrationOfExecutionsCanceled() {
+		try {
+			Map<ExecutionAttemptID, Execution> executions = setupExecution(19, 37);
+			
+			for (Execution e : executions.values()) {
+				e.cancel();
+				e.cancelingComplete();
+			}
+			
+			assertEquals(0, executions.size());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private Map<ExecutionAttemptID, Execution> setupExecution(int dop1, int dop2) throws Exception {
+		final JobID jobId = new JobID();
+		
+		final JobVertexID jid1 = new JobVertexID();
+		final JobVertexID jid2 = new JobVertexID();
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("v1", jid1);
+		AbstractJobVertex v2 = new AbstractJobVertex("v2", jid2);
+		
+		v1.setParallelism(dop1);
+		v2.setParallelism(dop2);
+		
+		v1.setInvokableClass(RegularPactTask.class);
+		v2.setInvokableClass(RegularPactTask.class);
+		
+		// execution graph that executes actions synchronously
+		ExecutionGraph eg = new ExecutionGraph(jobId, "some job", new Configuration());
+		eg.setQueuedSchedulingAllowed(false);
+		
+		List<AbstractJobVertex> ordered = Arrays.asList(v1, v2);
+		eg.attachJobGraph(ordered);
+		
+		// create a mock taskmanager that accepts deployment calls
+		TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+		when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenAnswer(new Answer<TaskOperationResult>() {
+			@Override
+			public TaskOperationResult answer(InvocationOnMock invocation) {
+				final TaskDeploymentDescriptor tdd = (TaskDeploymentDescriptor) invocation.getArguments()[0];
+				return new TaskOperationResult(tdd.getExecutionId(), true);
+			}
+		});
+		when(taskManager.cancelTask(Matchers.any(ExecutionAttemptID.class))).thenAnswer(new Answer<TaskOperationResult>() {
+			@Override
+			public TaskOperationResult answer(InvocationOnMock invocation) {
+				final ExecutionAttemptID id = (ExecutionAttemptID) invocation.getArguments()[0];
+				return new TaskOperationResult(id, true);
+			}
+		});
+		
+		DefaultScheduler scheduler = new DefaultScheduler();
+		for (int i = 0; i < dop1 + dop2; i++) {
+			scheduler.newInstanceAvailable(getInstance(taskManager));
+		}
+		assertEquals(dop1 + dop2, scheduler.getNumberOfAvailableSlots());
+		
+		// schedule, this triggers mock deployment
+		LibraryCacheManager.register(jobId, new String[0]);
+		eg.scheduleForExecution(scheduler);
+		
+		Map<ExecutionAttemptID, Execution> executions = eg.getRegisteredExecutions();
+		assertEquals(dop1 + dop2, executions.size());
+		
+		return executions;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
index 2207475..39c5678 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
@@ -21,6 +21,8 @@ package org.apache.flink.runtime.executiongraph;
 import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+import static org.mockito.Matchers.any;
 
 import java.lang.reflect.Field;
 import java.net.InetAddress;
@@ -28,7 +30,8 @@ import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.JobException;
-import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.Instance;
@@ -36,9 +39,11 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
 import org.mockito.Matchers;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
@@ -49,28 +54,43 @@ public class ExecutionGraphTestUtils {
 	//  state modifications
 	// --------------------------------------------------------------------------------------------
 	
-	public static void setVertexState(ExecutionVertex2 vertex, ExecutionState2 state) {
+	public static void setVertexState(ExecutionVertex vertex, ExecutionState state) {
 		try {
-			Field f = ExecutionVertex2.class.getDeclaredField("state");
+			Execution exec = vertex.getCurrentExecutionAttempt();
+			
+			Field f = Execution.class.getDeclaredField("state");
 			f.setAccessible(true);
-			f.set(vertex, state);
+			f.set(exec, state);
 		}
 		catch (Exception e) {
 			throw new RuntimeException("Modifying the state failed", e);
 		}
 	}
 	
-	public static void setVertexResource(ExecutionVertex2 vertex, AllocatedSlot slot) {
+	public static void setVertexResource(ExecutionVertex vertex, AllocatedSlot slot) {
 		try {
-			Field f = ExecutionVertex2.class.getDeclaredField("assignedSlot");
+			Execution exec = vertex.getCurrentExecutionAttempt();
+			
+			Field f = Execution.class.getDeclaredField("assignedResource");
 			f.setAccessible(true);
-			f.set(vertex, slot);
+			f.set(exec, slot);
 		}
 		catch (Exception e) {
 			throw new RuntimeException("Modifying the slot failed", e);
 		}
 	}
 	
+	public static void setGraphStatus(ExecutionGraph graph, JobStatus status) {
+		try {
+			Field f = ExecutionGraph.class.getDeclaredField("state");
+			f.setAccessible(true);
+			f.set(graph, status);
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Modifying the status failed", e);
+		}
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	//  utility mocking methods
 	// --------------------------------------------------------------------------------------------
@@ -88,6 +108,28 @@ public class ExecutionGraphTestUtils {
 		};
 	}
 	
+	public static TaskOperationProtocol getSimpleAcknowledgingTaskmanager() throws Exception {
+		TaskOperationProtocol top = mock(TaskOperationProtocol.class);
+		
+		when(top.submitTask(any(TaskDeploymentDescriptor.class))).thenAnswer(new Answer<TaskOperationResult>() {
+			@Override
+			public TaskOperationResult answer(InvocationOnMock invocation) {
+				final TaskDeploymentDescriptor tdd = (TaskDeploymentDescriptor) invocation.getArguments()[0];
+				return new TaskOperationResult(tdd.getExecutionId(), true);
+			}
+		});
+		
+		when(top.cancelTask(Matchers.any(ExecutionAttemptID.class))).thenAnswer(new Answer<TaskOperationResult>() {
+			@Override
+			public TaskOperationResult answer(InvocationOnMock invocation) {
+				final ExecutionAttemptID id = (ExecutionAttemptID) invocation.getArguments()[0];
+				return new TaskOperationResult(id, true);
+			}
+		});
+		
+		return top;
+	}
+	
 	public static ExecutionJobVertex getJobVertexNotExecuting(JobVertexID id) throws JobException {
 		ExecutionJobVertex ejv = getJobVertexBase(id);
 		
@@ -153,7 +195,20 @@ public class ExecutionGraphTestUtils {
 		
 		ExecutionGraph graph = new ExecutionGraph(new JobID(), "test job", new Configuration());
 		
-		return spy(new ExecutionJobVertex(graph, ajv, 1));
+		ExecutionJobVertex ejv = spy(new ExecutionJobVertex(graph, ajv, 1));
+		
+		Answer<Void> noop = new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				return null;
+			}
+		};
+		
+		doAnswer(noop).when(ejv).vertexCancelled(Matchers.anyInt());
+		doAnswer(noop).when(ejv).vertexFailed(Matchers.anyInt(), Matchers.any(Throwable.class));
+		doAnswer(noop).when(ejv).vertexFinished(Matchers.anyInt());
+		
+		return ejv;
 	}
 	
 	// --------------------------------------------------------------------------------------------


[22/63] [abbrv] git commit: Unify all job vertices to one type (rather than dedicated input/output types)

Posted by se...@apache.org.
Unify all job vertices to one type (rather than dedicated input/output types)


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/cb7039e3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/cb7039e3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/cb7039e3

Branch: refs/heads/master
Commit: cb7039e3e171474a7635a73dda9c086c84966dd0
Parents: e6aadfc
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Jun 22 19:05:02 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../plantranslate/NepheleJobGraphGenerator.java |  25 ++-
 .../jobgraph/AbstractJobOutputVertex.java       |   2 -
 .../runtime/jobgraph/AbstractJobVertex.java     |  38 +----
 .../jobgraph/InputFormatInputVertex.java        | 103 ++++++++++++
 .../apache/flink/runtime/jobgraph/JobEdge.java  |   2 -
 .../apache/flink/runtime/jobgraph/JobGraph.java | 162 +++----------------
 .../flink/runtime/jobgraph/JobInputVertex.java  | 103 ------------
 .../flink/runtime/jobgraph/JobOutputVertex.java |  84 ----------
 .../jobgraph/OutputFormatOutputVertex.java      |  83 ++++++++++
 .../runtime/jobgraph/SimpleInputVertex.java     |  61 +++++++
 .../runtime/jobgraph/SimpleOutputVertex.java    |  53 ++++++
 .../jobgraph/tasks/AbstractInvokable.java       |  12 +-
 .../executiongraph/SelfCrossForwardTask.java    |   2 -
 .../BroadcastVarsNepheleITCase.java             |  22 +--
 .../KMeansIterativeNepheleITCase.java           |  31 ++--
 .../ConnectedComponentsNepheleITCase.java       |  67 ++++----
 .../IterationWithChainingNepheleITCase.java     |  13 +-
 .../test/iterative/nephele/JobGraphUtils.java   |  23 +--
 .../CustomCompensatableDanglingPageRank.java    |  16 +-
 ...mpensatableDanglingPageRankWithCombiner.java |  17 +-
 .../CompensatableDanglingPageRank.java          |  15 +-
 .../test/runtime/NetworkStackThroughput.java    |   8 +-
 22 files changed, 449 insertions(+), 493 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index 043a0a7..cb912de 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -66,11 +66,12 @@ import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.AbstractJobOutputVertex;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetFirstDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetSecondDriver;
@@ -808,8 +809,8 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		return vertex;
 	}
 
-	private JobInputVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException {
-		final JobInputVertex vertex = new JobInputVertex(node.getNodeName(), this.jobGraph);
+	private AbstractJobVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException {
+		final InputFormatInputVertex vertex = new InputFormatInputVertex(node.getNodeName(), this.jobGraph);
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
 		vertex.setInvokableClass(DataSourceTask.class);
@@ -823,7 +824,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	}
 
 	private AbstractJobOutputVertex createDataSinkVertex(SinkPlanNode node) throws CompilerException {
-		final JobOutputVertex vertex = new JobOutputVertex(node.getNodeName(), this.jobGraph);
+		final OutputFormatOutputVertex vertex = new OutputFormatOutputVertex(node.getNodeName(), this.jobGraph);
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
 		vertex.setInvokableClass(DataSinkTask.class);
@@ -1138,8 +1139,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		headConfig.setRelativeBackChannelMemory(relativeMemForBackChannel);
 		
 		// --------------------------- create the sync task ---------------------------
-		final JobOutputVertex sync = new JobOutputVertex("Sync(" +
-					bulkNode.getNodeName() + ")", this.jobGraph);
+		final SimpleOutputVertex sync = new SimpleOutputVertex("Sync(" + bulkNode.getNodeName() + ")", this.jobGraph);
 		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
 		sync.setNumberOfSubtasks(1);
 		this.auxVertices.add(sync);
@@ -1194,7 +1194,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			tailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 			
 			// create the fake output task
-			JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
+			SimpleOutputVertex fakeTail = new SimpleOutputVertex("Fake Tail", this.jobGraph);
 			fakeTail.setInvokableClass(FakeOutputTask.class);
 			fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 			this.auxVertices.add(fakeTail);
@@ -1236,7 +1236,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			tailConfigOfTerminationCriterion.setOutputSerializer(bulkNode.getSerializerForIterationChannel());
 			tailConfigOfTerminationCriterion.addOutputShipStrategy(ShipStrategyType.FORWARD);
 			
-			JobOutputVertex fakeTailTerminationCriterion = new JobOutputVertex("Fake Tail for Termination Criterion", this.jobGraph);
+			SimpleOutputVertex fakeTailTerminationCriterion = new SimpleOutputVertex("Fake Tail for Termination Criterion", this.jobGraph);
 			fakeTailTerminationCriterion.setInvokableClass(FakeOutputTask.class);
 			fakeTailTerminationCriterion.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 			this.auxVertices.add(fakeTailTerminationCriterion);
@@ -1303,8 +1303,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		// --------------------------- create the sync task ---------------------------
 		final TaskConfig syncConfig;
 		{
-			final JobOutputVertex sync = new JobOutputVertex("Sync (" +
-						iterNode.getNodeName() + ")", this.jobGraph);
+			final SimpleOutputVertex sync = new SimpleOutputVertex("Sync (" + iterNode.getNodeName() + ")", this.jobGraph);
 			sync.setInvokableClass(IterationSynchronizationSinkTask.class);
 			sync.setNumberOfSubtasks(1);
 			this.auxVertices.add(sync);
@@ -1369,7 +1368,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					worksetTailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 					
 					// create the fake output task
-					JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
+					SimpleOutputVertex fakeTail = new SimpleOutputVertex("Fake Tail", this.jobGraph);
 					fakeTail.setInvokableClass(FakeOutputTask.class);
 					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 					this.auxVertices.add(fakeTail);
@@ -1407,7 +1406,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					solutionDeltaConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 	
 					// create the fake output task
-					JobOutputVertex fakeTail = new JobOutputVertex("Fake Tail", this.jobGraph);
+					SimpleOutputVertex fakeTail = new SimpleOutputVertex("Fake Tail", this.jobGraph);
 					fakeTail.setInvokableClass(FakeOutputTask.class);
 					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
 					this.auxVertices.add(fakeTail);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java
index c1f0ec5..edb8d0e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobOutputVertex.java
@@ -16,12 +16,10 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph;
 
 /**
  * An abstract base class for output vertices in Nephele.
- * 
  */
 public abstract class AbstractJobOutputVertex extends AbstractJobVertex {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index 08a9567..7df76c3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -72,14 +72,9 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	private int numberOfSubtasks = -1;
 
 	/**
-	 * Number of retries in case of an error before the task represented by this vertex is considered as failed.
-	 */
-	private int numberOfExecutionRetries = -1;
-
-	/**
 	 * Other task to share a (set of) of instances with at runtime.
 	 */
-	private AbstractJobVertex vertexToShareInstancesWith = null;
+	private AbstractJobVertex vertexToShareInstancesWith;
 
 	/**
 	 * Custom configuration passed to the assigned task at runtime.
@@ -89,7 +84,7 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	/**
 	 * The class of the invokable.
 	 */
-	protected Class<? extends AbstractInvokable> invokableClass = null;
+	protected Class<? extends AbstractInvokable> invokableClass;
 
 	
 	/**
@@ -388,9 +383,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 		// Read number of subtasks
 		this.numberOfSubtasks = in.readInt();
 
-		// Number of execution retries
-		this.numberOfExecutionRetries = in.readInt();
-
 		// Read vertex to share instances with
 		if (in.readBoolean()) {
 			final JobVertexID id = new JobVertexID();
@@ -464,9 +456,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 		// Number of subtasks
 		out.writeInt(this.numberOfSubtasks);
 
-		// Number of execution retries
-		out.writeInt(this.numberOfExecutionRetries);
-
 		// Vertex to share instance with
 		if (this.vertexToShareInstancesWith != null) {
 			out.writeBoolean(true);
@@ -538,29 +527,6 @@ public abstract class AbstractJobVertex implements IOReadableWritable {
 	}
 
 	/**
-	 * Sets the number of retries in case of an error before the task represented by this vertex is considered as
-	 * failed.
-	 * 
-	 * @param numberOfExecutionRetries
-	 *        the number of retries in case of an error before the task represented by this vertex is considered as
-	 *        failed
-	 */
-	public void setNumberOfExecutionRetries(final int numberOfExecutionRetries) {
-		this.numberOfExecutionRetries = numberOfExecutionRetries;
-	}
-
-	/**
-	 * Returns the number of retries in case of an error before the task represented by this vertex is considered as
-	 * failed.
-	 * 
-	 * @return the number of retries in case of an error before the task represented by this vertex is considered as
-	 *         failed or <code>-1</code> if unspecified
-	 */
-	public int getNumberOfExecutionRetries() {
-		return this.numberOfExecutionRetries;
-	}
-
-	/**
 	 * Sets the vertex this vertex should share its instances with at runtime.
 	 * 
 	 * @param vertex

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java
new file mode 100644
index 0000000..f79264a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/InputFormatInputVertex.java
@@ -0,0 +1,103 @@
+/**
+ * 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.jobgraph;
+
+import java.io.IOException;
+
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.operators.util.UserCodeWrapper;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+
+
+public class InputFormatInputVertex extends AbstractJobInputVertex {
+
+	private InputFormat<?, ?> inputFormat;
+	
+	public InputFormatInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
+		super(name, id, jobGraph);
+	}
+	
+	/**
+	 * Creates a new job file input vertex with the specified name.
+	 * 
+	 * @param name
+	 *        The name of the new job file input vertex.
+	 * @param jobGraph
+	 *        The job graph this vertex belongs to.
+	 */
+	public InputFormatInputVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
+	}
+
+	/**
+	 * Creates a new job file input vertex.
+	 * 
+	 * @param jobGraph
+	 *        The job graph this vertex belongs to.
+	 */
+	public InputFormatInputVertex(JobGraph jobGraph) {
+		this(null, jobGraph);
+	}
+	
+	public void setInputFormat(InputFormat<?, ?> format) {
+		this.inputFormat = format;
+	}
+	
+	public void initializeInputFormatFromTaskConfig(ClassLoader cl) {
+		TaskConfig cfg = new TaskConfig(getConfiguration());
+		
+		UserCodeWrapper<InputFormat<?, ?>> wrapper = cfg.<InputFormat<?, ?>>getStubWrapper(cl);
+		
+		if (wrapper != null) {
+			this.inputFormat = wrapper.getUserCodeObject(InputFormat.class, cl);
+			this.inputFormat.configure(cfg.getStubParameters());
+		}
+	}
+
+	/**
+	 * Gets the input split type class
+	 *
+	 * @return Input split type class
+	 */
+	@Override
+	public Class<? extends InputSplit> getInputSplitType() {
+		if (inputFormat == null){
+			return InputSplit.class;
+		}
+
+		return inputFormat.getInputSplitType();
+	}
+
+	/**
+	 * Gets the input splits from the input format.
+	 *
+	 * @param minNumSplits Number of minimal input splits
+	 * @return Array of input splits
+	 * @throws IOException
+	 */
+	@Override
+	public InputSplit[] getInputSplits(int minNumSplits) throws IOException {
+		if (inputFormat == null){
+			return null;
+		}
+
+		return inputFormat.createInputSplits(minNumSplits);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
index 0a5df3a..33b6576 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobEdge.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph;
 
 import org.apache.flink.runtime.io.network.channels.ChannelType;
@@ -24,7 +23,6 @@ import org.apache.flink.runtime.io.network.channels.ChannelType;
 /**
  * Objects of this class represent edges in the user's job graph.
  * The edges can be annotated by a specific channel and compression level.
- * 
  */
 public class JobEdge {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 2040c8e..48d858a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -27,6 +27,7 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -46,24 +47,13 @@ import org.apache.flink.util.ClassUtils;
 /**
  * A job graph represents an entire job in Nephele. A job graph must consists at least of one job vertex
  * and must be acyclic.
- * 
  */
 public class JobGraph implements IOReadableWritable {
 
 	/**
-	 * List of input vertices included in this job graph.
-	 */
-	private Map<JobVertexID, AbstractJobInputVertex> inputVertices = new HashMap<JobVertexID, AbstractJobInputVertex>();
-
-	/**
-	 * List of output vertices included in this job graph.
-	 */
-	private Map<JobVertexID, AbstractJobOutputVertex> outputVertices = new HashMap<JobVertexID, AbstractJobOutputVertex>();
-
-	/**
 	 * List of task vertices included in this job graph.
 	 */
-	private Map<JobVertexID, JobTaskVertex> taskVertices = new HashMap<JobVertexID, JobTaskVertex>();
+	private Map<JobVertexID, AbstractJobVertex> taskVertices = new LinkedHashMap<JobVertexID, AbstractJobVertex>();
 
 	/**
 	 * ID of this job.
@@ -90,11 +80,8 @@ public class JobGraph implements IOReadableWritable {
 	 */
 	private static final int BUFFERSIZE = 8192;
 
-	/**
-	 * Buffer for array of reachable job vertices
-	 */
-	private volatile AbstractJobVertex[] bufferedAllReachableJobVertices = null;
-
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Constructs a new job graph with a random job ID.
 	 */
@@ -108,7 +95,7 @@ public class JobGraph implements IOReadableWritable {
 	 * @param jobName
 	 *        the name for this job graph
 	 */
-	public JobGraph(final String jobName) {
+	public JobGraph(String jobName) {
 		this();
 		this.jobName = jobName;
 	}
@@ -128,65 +115,27 @@ public class JobGraph implements IOReadableWritable {
 	 * @return the configuration object for this job, or <code>null</code> if it is not set
 	 */
 	public Configuration getJobConfiguration() {
-
 		return this.jobConfiguration;
 	}
 
 	/**
-	 * Adds a new input vertex to the job graph if it is not already included.
-	 * 
-	 * @param inputVertex
-	 *        the new input vertex to be added
-	 */
-	public void addVertex(AbstractJobInputVertex inputVertex) {
-		if (!inputVertices.containsKey(inputVertex.getID())) {
-			inputVertices.put(inputVertex.getID(), inputVertex);
-		}
-	}
-
-	/**
 	 * Adds a new task vertex to the job graph if it is not already included.
 	 * 
 	 * @param taskVertex
 	 *        the new task vertex to be added
 	 */
-	public void addVertex(JobTaskVertex taskVertex) {
-		if (!taskVertices.containsKey(taskVertex.getID())) {
-			taskVertices.put(taskVertex.getID(), taskVertex);
-		}
-	}
-
-	/**
-	 * Adds a new output vertex to the job graph if it is not already included.
-	 * 
-	 * @param outputVertex
-	 *        the new output vertex to be added
-	 */
-	public void addVertex(AbstractJobOutputVertex outputVertex) {
-		if (!outputVertices.containsKey(outputVertex.getID())) {
-			outputVertices.put(outputVertex.getID(), outputVertex);
+	public void addVertex(AbstractJobVertex vertex) {
+		final JobVertexID id = vertex.getID();
+		AbstractJobVertex previous = taskVertices.put(id, vertex);
+		
+		// if we had a prior association, restore and throw an exception
+		if (previous != null) {
+			taskVertices.put(id, vertex);
+			throw new IllegalArgumentException("The JobGraph already contains a vertex with that id.");
 		}
 	}
 
 	/**
-	 * Returns the number of input vertices registered with the job graph.
-	 * 
-	 * @return the number of input vertices registered with the job graph
-	 */
-	public int getNumberOfInputVertices() {
-		return this.inputVertices.size();
-	}
-
-	/**
-	 * Returns the number of output vertices registered with the job graph.
-	 * 
-	 * @return the number of output vertices registered with the job graph
-	 */
-	public int getNumberOfOutputVertices() {
-		return this.outputVertices.size();
-	}
-
-	/**
 	 * Returns the number of task vertices registered with the job graph.
 	 * 
 	 * @return the number of task vertices registered with the job graph
@@ -196,39 +145,12 @@ public class JobGraph implements IOReadableWritable {
 	}
 
 	/**
-	 * Returns an iterator to iterate all input vertices registered with the job graph.
-	 * 
-	 * @return an iterator to iterate all input vertices registered with the job graph
-	 */
-	public Iterator<AbstractJobInputVertex> getInputVertices() {
-
-		final Collection<AbstractJobInputVertex> coll = this.inputVertices.values();
-
-		return coll.iterator();
-	}
-
-	/**
-	 * Returns an iterator to iterate all output vertices registered with the job graph.
-	 * 
-	 * @return an iterator to iterate all output vertices registered with the job graph
-	 */
-	public Iterator<AbstractJobOutputVertex> getOutputVertices() {
-
-		final Collection<AbstractJobOutputVertex> coll = this.outputVertices.values();
-
-		return coll.iterator();
-	}
-
-	/**
-	 * Returns an iterator to iterate all task vertices registered with the job graph.
+	 * Returns an Iterable to iterate all vertices registered with the job graph.
 	 * 
-	 * @return an iterator to iterate all task vertices registered with the job graph
+	 * @return an Iterable to iterate all vertices registered with the job graph
 	 */
-	public Iterator<JobTaskVertex> getTaskVertices() {
-
-		final Collection<JobTaskVertex> coll = this.taskVertices.values();
-
-		return coll.iterator();
+	public Iterable<AbstractJobVertex> getTaskVertices() {
+		return this.taskVertices.values();
 	}
 
 	/**
@@ -237,35 +159,7 @@ public class JobGraph implements IOReadableWritable {
 	 * @return the number of all job vertices registered with this job graph
 	 */
 	public int getNumberOfVertices() {
-
-		return this.inputVertices.size() + this.outputVertices.size() + this.taskVertices.size();
-	}
-
-	/**
-	 * Returns an array of all job vertices than can be reached when traversing the job graph from the input vertices.
-	 * Each job vertex is contained only one time.
-	 * 
-	 * @return an array of all job vertices than can be reached when traversing the job graph from the input vertices
-	 */
-	public AbstractJobVertex[] getAllReachableJobVertices() {
-		if(bufferedAllReachableJobVertices == null){
-			final List<AbstractJobVertex> collector = new ArrayList<AbstractJobVertex>();
-			final HashSet<JobVertexID> visited = new HashSet<JobVertexID>();
-
-			final Iterator<AbstractJobInputVertex> inputs = getInputVertices();
-
-			while(inputs.hasNext()){
-				AbstractJobVertex vertex = inputs.next();
-
-				if(!visited.contains(vertex.getID())){
-					collectVertices(vertex, visited, collector);
-				}
-			}
-
-			bufferedAllReachableJobVertices = collector.toArray(new AbstractJobVertex[0]);
-		}
-
-		return bufferedAllReachableJobVertices;
+		return this.taskVertices.size();
 	}
 
 	/**
@@ -297,7 +191,8 @@ public class JobGraph implements IOReadableWritable {
 	 * @return an array of all job vertices that are registered with the job graph
 	 */
 	public AbstractJobVertex[] getAllJobVertices() {
-
+		return this.taskVertices.values().toArray(new AbstractJobVertex[this.taskVertices.size()]);
+		
 		int i = 0;
 		final AbstractJobVertex[] vertices = new AbstractJobVertex[inputVertices.size() + outputVertices.size()
 			+ taskVertices.size()];
@@ -337,21 +232,8 @@ public class JobGraph implements IOReadableWritable {
 	 *        the ID of the vertex to search for
 	 * @return the vertex with the matching ID or <code>null</code> if no vertex with such ID could be found
 	 */
-	public AbstractJobVertex findVertexByID(final JobVertexID id) {
-
-		if (this.inputVertices.containsKey(id)) {
-			return this.inputVertices.get(id);
-		}
-
-		if (this.outputVertices.containsKey(id)) {
-			return this.outputVertices.get(id);
-		}
-
-		if (this.taskVertices.containsKey(id)) {
-			return this.taskVertices.get(id);
-		}
-
-		return null;
+	public AbstractJobVertex findVertexByID(JobVertexID id) {
+		return this.taskVertices.get(id);
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobInputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobInputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobInputVertex.java
deleted file mode 100644
index bffb182..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobInputVertex.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/**
- * 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.jobgraph;
-
-import java.io.IOException;
-
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.operators.util.UserCodeWrapper;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-
-public class JobInputVertex extends AbstractJobInputVertex {
-
-	private InputFormat<?, ?> inputFormat;
-	
-	public JobInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-	
-	/**
-	 * Creates a new job file input vertex with the specified name.
-	 * 
-	 * @param name
-	 *        The name of the new job file input vertex.
-	 * @param jobGraph
-	 *        The job graph this vertex belongs to.
-	 */
-	public JobInputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        The job graph this vertex belongs to.
-	 */
-	public JobInputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-	
-	public void setInputFormat(InputFormat<?, ?> format) {
-		this.inputFormat = format;
-	}
-	
-	public void initializeInputFormatFromTaskConfig(ClassLoader cl) {
-		TaskConfig cfg = new TaskConfig(getConfiguration());
-		
-		UserCodeWrapper<InputFormat<?, ?>> wrapper = cfg.<InputFormat<?, ?>>getStubWrapper(cl);
-		
-		if (wrapper != null) {
-			this.inputFormat = wrapper.getUserCodeObject(InputFormat.class, cl);
-			this.inputFormat.configure(cfg.getStubParameters());
-		}
-	}
-
-	/**
-	 * Gets the input split type class
-	 *
-	 * @return Input split type class
-	 */
-	@Override
-	public Class<? extends InputSplit> getInputSplitType() {
-		if (inputFormat == null){
-			return InputSplit.class;
-		}
-
-		return inputFormat.getInputSplitType();
-	}
-
-	/**
-	 * Gets the input splits from the input format.
-	 *
-	 * @param minNumSplits Number of minimal input splits
-	 * @return Array of input splits
-	 * @throws IOException
-	 */
-	@Override
-	public InputSplit[] getInputSplits(int minNumSplits) throws IOException {
-		if (inputFormat == null){
-			return null;
-		}
-
-		return inputFormat.createInputSplits(minNumSplits);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobOutputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobOutputVertex.java
deleted file mode 100644
index 352d9b3..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobOutputVertex.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * 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.jobgraph;
-
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.api.common.operators.util.UserCodeWrapper;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-
-/**
- * A JobOutputVertex is a specific sub-type of a {@link AbstractJobOutputVertex} and is designed
- * for Nephele tasks which sink data in a not further specified way. As every job output vertex,
- * a JobOutputVertex must not have any further output.
- */
-public class JobOutputVertex extends AbstractJobOutputVertex {
-	/**
-	 * Contains the output format associated to this output vertex. It can be <pre>null</pre>.
-	 */
-	private OutputFormat<?> outputFormat;
-
-
-	/**
-	 * Creates a new job file output vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file output vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobOutputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-	
-	public JobOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobOutputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-	
-	public void setOutputFormat(OutputFormat<?> format) {
-		this.outputFormat = format;
-	}
-	
-	public void initializeOutputFormatFromTaskConfig(ClassLoader cl) {
-		TaskConfig cfg = new TaskConfig(getConfiguration());
-		UserCodeWrapper<OutputFormat<?>> wrapper = cfg.<OutputFormat<?>>getStubWrapper(cl);
-		
-		if (wrapper != null) {
-			this.outputFormat = wrapper.getUserCodeObject(OutputFormat.class, cl);
-			this.outputFormat.configure(cfg.getStubParameters());
-		}
-	}
-
-	/**
-	 * Returns the output format. It can also be <pre>null</pre>.
-	 *
-	 * @return output format or <pre>null</pre>
-	 */
-	public OutputFormat<?> getOutputFormat() { return outputFormat; }
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java
new file mode 100644
index 0000000..08a03bc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/OutputFormatOutputVertex.java
@@ -0,0 +1,83 @@
+/**
+ * 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.jobgraph;
+
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.api.common.operators.util.UserCodeWrapper;
+import org.apache.flink.runtime.operators.util.TaskConfig;
+
+/**
+ * A JobOutputVertex is a specific sub-type of a {@link AbstractJobOutputVertex} and is designed
+ * for Nephele tasks which sink data in a not further specified way. As every job output vertex,
+ * a JobOutputVertex must not have any further output.
+ */
+public class OutputFormatOutputVertex extends AbstractJobOutputVertex {
+	/**
+	 * Contains the output format associated to this output vertex. It can be <pre>null</pre>.
+	 */
+	private OutputFormat<?> outputFormat;
+
+
+	/**
+	 * Creates a new job file output vertex with the specified name.
+	 * 
+	 * @param name
+	 *        the name of the new job file output vertex
+	 * @param jobGraph
+	 *        the job graph this vertex belongs to
+	 */
+	public OutputFormatOutputVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
+	}
+	
+	public OutputFormatOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
+		super(name, id, jobGraph);
+	}
+
+	/**
+	 * Creates a new job file input vertex.
+	 * 
+	 * @param jobGraph
+	 *        the job graph this vertex belongs to
+	 */
+	public OutputFormatOutputVertex(JobGraph jobGraph) {
+		this(null, jobGraph);
+	}
+	
+	public void setOutputFormat(OutputFormat<?> format) {
+		this.outputFormat = format;
+	}
+	
+	public void initializeOutputFormatFromTaskConfig(ClassLoader cl) {
+		TaskConfig cfg = new TaskConfig(getConfiguration());
+		UserCodeWrapper<OutputFormat<?>> wrapper = cfg.<OutputFormat<?>>getStubWrapper(cl);
+		
+		if (wrapper != null) {
+			this.outputFormat = wrapper.getUserCodeObject(OutputFormat.class, cl);
+			this.outputFormat.configure(cfg.getStubParameters());
+		}
+	}
+
+	/**
+	 * Returns the output format. It can also be <pre>null</pre>.
+	 *
+	 * @return output format or <pre>null</pre>
+	 */
+	public OutputFormat<?> getOutputFormat() { return outputFormat; }
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java
new file mode 100644
index 0000000..3699f0e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleInputVertex.java
@@ -0,0 +1,61 @@
+/**
+ * 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.jobgraph;
+
+import org.apache.flink.core.io.InputSplit;
+
+
+public class SimpleInputVertex extends AbstractJobInputVertex {
+
+	/**
+	 * Creates a new job file output vertex with the specified name.
+	 * 
+	 * @param name
+	 *        the name of the new job file output vertex
+	 * @param jobGraph
+	 *        the job graph this vertex belongs to
+	 */
+	public SimpleInputVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
+	}
+	
+	public SimpleInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
+		super(name, id, jobGraph);
+	}
+
+	/**
+	 * Creates a new job file input vertex.
+	 * 
+	 * @param jobGraph
+	 *        the job graph this vertex belongs to
+	 */
+	public SimpleInputVertex(JobGraph jobGraph) {
+		this(null, jobGraph);
+	}
+
+	@Override
+	public Class<? extends InputSplit> getInputSplitType() {
+		return null;
+	}
+
+	@Override
+	public InputSplit[] getInputSplits(int minNumSplits) throws Exception {
+		return null;
+	}	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java
new file mode 100644
index 0000000..8709a07
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SimpleOutputVertex.java
@@ -0,0 +1,53 @@
+/**
+ * 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.jobgraph;
+
+/**
+ * A JobOutputVertex is a specific sub-type of a {@link AbstractJobOutputVertex} and is designed
+ * for Nephele tasks which sink data in a not further specified way. As every job output vertex,
+ * a JobOutputVertex must not have any further output.
+ */
+public class SimpleOutputVertex extends AbstractJobOutputVertex {
+
+	/**
+	 * Creates a new job file output vertex with the specified name.
+	 * 
+	 * @param name
+	 *        the name of the new job file output vertex
+	 * @param jobGraph
+	 *        the job graph this vertex belongs to
+	 */
+	public SimpleOutputVertex(String name, JobGraph jobGraph) {
+		this(name, null, jobGraph);
+	}
+	
+	public SimpleOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
+		super(name, id, jobGraph);
+	}
+
+	/**
+	 * Creates a new job file input vertex.
+	 * 
+	 * @param jobGraph
+	 *        the job graph this vertex belongs to
+	 */
+	public SimpleOutputVertex(JobGraph jobGraph) {
+		this(null, jobGraph);
+	}	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
index 25bb027..d3ad516 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/tasks/AbstractInvokable.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph.tasks;
 
 import org.apache.flink.configuration.Configuration;
@@ -24,7 +23,6 @@ import org.apache.flink.runtime.execution.Environment;
 
 /**
  * Abstract base class for every task class in Nephele.
- * 
  */
 public abstract class AbstractInvokable {
 
@@ -42,7 +40,7 @@ public abstract class AbstractInvokable {
 	 * Must be overwritten by the concrete task. This method is called by the task manager
 	 * when the actual execution of the task starts.
 	 * 
-	 * @throws Execution
+	 * @throws Exception
 	 *         thrown if any exception occurs during the execution of the tasks
 	 */
 	public abstract void invoke() throws Exception;
@@ -89,9 +87,9 @@ public abstract class AbstractInvokable {
 	}
 
 	/**
-	 * Returns the task configuration object which was attached to the original {@link JobVertex}.
+	 * Returns the task configuration object which was attached to the original {@link org.apache.flink.runtime.jobgraph.AbstractJobVertex}.
 	 * 
-	 * @return the task configuration object which was attached to the original {@link JobVertex}
+	 * @return the task configuration object which was attached to the original {@link org.apache.flink.runtime.jobgraph.AbstractJobVertex}
 	 */
 	public final Configuration getTaskConfiguration() {
 
@@ -99,9 +97,9 @@ public abstract class AbstractInvokable {
 	}
 
 	/**
-	 * Returns the job configuration object which was attached to the original {@link JobGraph}.
+	 * Returns the job configuration object which was attached to the original {@link org.apache.flink.runtime.jobgraph.JobGraph}.
 	 * 
-	 * @return the job configuration object which was attached to the original {@link JobGraph}
+	 * @return the job configuration object which was attached to the original {@link org.apache.flink.runtime.jobgraph.JobGraph}
 	 */
 	public final Configuration getJobConfiguration() {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java
index c181b58..7fafeda 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java
@@ -16,10 +16,8 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.executiongraph;
 
-
 import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.runtime.io.network.api.RecordReader;
 import org.apache.flink.runtime.io.network.api.RecordWriter;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
index 68f6496..a409222 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
@@ -35,11 +35,11 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.RegularPactTask;
@@ -227,9 +227,9 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	// -------------------------------------------------------------------------------------------------------------
 
 	@SuppressWarnings("unchecked")
-	private static JobInputVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatInputVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		CsvInputFormat pointsInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class, LongValue.class, LongValue.class);
-		JobInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "Input[Points]", jobGraph, numSubTasks);
+		InputFormatInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "Input[Points]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
@@ -241,9 +241,9 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	}
 
 	@SuppressWarnings("unchecked")
-	private static JobInputVertex createModelsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatInputVertex createModelsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		CsvInputFormat modelsInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class, LongValue.class, LongValue.class);
-		JobInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, pointsPath, "Input[Models]", jobGraph, numSubTasks);
+		InputFormatInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, pointsPath, "Input[Models]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(modelsInput.getConfiguration());
@@ -278,8 +278,8 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 		return pointsInput;
 	}
 
-	private static JobOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
+	private static OutputFormatOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(output.getConfiguration());
@@ -308,10 +308,10 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Distance Builder");
 
 		// -- vertices ---------------------------------------------------------------------------------------------
-		JobInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
-		JobInputVertex models = createModelsInput(jobGraph, centersPath, numSubTasks, serializer);
+		InputFormatInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
+		InputFormatInputVertex models = createModelsInput(jobGraph, centersPath, numSubTasks, serializer);
 		JobTaskVertex mapper = createMapper(jobGraph, numSubTasks, serializer);
-		JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
 
 		// -- edges ------------------------------------------------------------------------------------------------
 		JobGraphUtils.connect(points, mapper, ChannelType.NETWORK, DistributionPattern.POINTWISE);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
index 77d68f8..4d46b16 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -32,11 +32,12 @@ import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.GroupReduceDriver;
@@ -95,10 +96,10 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	// Job vertex builder methods
 	// -------------------------------------------------------------------------------------------------------------
 
-	private static JobInputVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatInputVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat pointsInFormat = new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class);
-		JobInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "[Points]", jobGraph, numSubTasks);
+		InputFormatInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "[Points]", jobGraph, numSubTasks);
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
 			taskConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
@@ -116,10 +117,10 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return pointsInput;
 	}
 
-	private static JobInputVertex createCentersInput(JobGraph jobGraph, String centersPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatInputVertex createCentersInput(JobGraph jobGraph, String centersPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat modelsInFormat = new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class);
-		JobInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, centersPath, "[Models]", jobGraph, numSubTasks);
+		InputFormatInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, centersPath, "[Models]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(modelsInput.getConfiguration());
@@ -138,9 +139,9 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return modelsInput;
 	}
 
-	private static JobOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static OutputFormatOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
+		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(output.getConfiguration());
@@ -254,8 +255,8 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return tail;
 	}
 	
-	private static JobOutputVertex createSync(JobGraph jobGraph, int numIterations, int dop) {
-		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, dop);
+	private static SimpleOutputVertex createSync(JobGraph jobGraph, int numIterations, int dop) {
+		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, dop);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.setIterationId(ITERATION_ID);
@@ -276,19 +277,19 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("KMeans Iterative");
 
 		// -- vertices ---------------------------------------------------------------------------------------------
-		JobInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
-		JobInputVertex centers = createCentersInput(jobGraph, centersPath, numSubTasks, serializer);
+		InputFormatInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
+		InputFormatInputVertex centers = createCentersInput(jobGraph, centersPath, numSubTasks, serializer);
 		
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer);
 		JobTaskVertex mapper = createMapper(jobGraph, numSubTasks, serializer, serializer, serializer, int0Comparator);
 		
 		JobTaskVertex reducer = createReducer(jobGraph, numSubTasks, serializer, int0Comparator, serializer);
 		
-		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
+		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		
-		JobOutputVertex sync = createSync(jobGraph, numIterations, numSubTasks);
+		SimpleOutputVertex sync = createSync(jobGraph, numIterations, numSubTasks);
 		
-		JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
 
 		// -- edges ------------------------------------------------------------------------------------------------
 		JobGraphUtils.connect(points, mapper, ChannelType.NETWORK, DistributionPattern.POINTWISE);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
index 4fd22a3..8da4e5c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
@@ -44,11 +44,12 @@ import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
@@ -173,12 +174,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 	// Invariant vertices across all variants
 	// -----------------------------------------------------------------------------------------------------------------
 
-	private static JobInputVertex createVerticesInput(JobGraph jobGraph, String verticesPath, int numSubTasks,
+	private static InputFormatInputVertex createVerticesInput(JobGraph jobGraph, String verticesPath, int numSubTasks,
 			TypeSerializerFactory<?> serializer,
 			TypeComparatorFactory<?> comparator) {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat verticesInFormat = new CsvInputFormat(' ', LongValue.class);
-		JobInputVertex verticesInput = JobGraphUtils.createInput(verticesInFormat, verticesPath, "VerticesInput",
+		InputFormatInputVertex verticesInput = JobGraphUtils.createInput(verticesInFormat, verticesPath, "VerticesInput",
 			jobGraph, numSubTasks);
 		TaskConfig verticesInputConfig = new TaskConfig(verticesInput.getConfiguration());
 		{
@@ -204,13 +205,13 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return verticesInput;
 	}
 
-	private static JobInputVertex createEdgesInput(JobGraph jobGraph, String edgesPath, int numSubTasks,
+	private static InputFormatInputVertex createEdgesInput(JobGraph jobGraph, String edgesPath, int numSubTasks,
 			TypeSerializerFactory<?> serializer,
 			TypeComparatorFactory<?> comparator) {
 		// edges
 		@SuppressWarnings("unchecked")
 		CsvInputFormat edgesInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class);
-		JobInputVertex edgesInput = JobGraphUtils.createInput(edgesInFormat, edgesPath, "EdgesInput", jobGraph,
+		InputFormatInputVertex edgesInput = JobGraphUtils.createInput(edgesInFormat, edgesPath, "EdgesInput", jobGraph,
 			numSubTasks);
 		TaskConfig edgesInputConfig = new TaskConfig(edgesInput.getConfiguration());
 		{
@@ -326,9 +327,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return intermediate;
 	}
 
-	private static JobOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks,
+	private static OutputFormatOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks,
 			TypeSerializerFactory<?> serializer) {
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Final Output", numSubTasks);
+		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Final Output", numSubTasks);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		{
 
@@ -351,14 +352,14 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return output;
 	}
 
-	private static JobOutputVertex createFakeTail(JobGraph jobGraph, int numSubTasks) {
-		JobOutputVertex fakeTailOutput =
+	private static SimpleOutputVertex createFakeTail(JobGraph jobGraph, int numSubTasks) {
+		SimpleOutputVertex fakeTailOutput =
 			JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		return fakeTailOutput;
 	}
 
-	private static JobOutputVertex createSync(JobGraph jobGraph, int numSubTasks, int maxIterations) {
-		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
+	private static SimpleOutputVertex createSync(JobGraph jobGraph, int numSubTasks, int maxIterations) {
+		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(maxIterations);
 		syncConfig.setIterationId(ITERATION_ID);
@@ -388,16 +389,16 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Unified Tails)");
 
 		// -- invariant vertices -----------------------------------------------------------------------------------
-		JobInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		JobInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
 
 		JobTaskVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
-		JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		JobOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		SimpleOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// --------------- the tail (solution set join) ---------------
 		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
@@ -472,8 +473,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Unified Tails)");
 
 		// input
-		JobInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		JobInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
@@ -485,10 +486,10 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobOutputVertex ssFakeTail = createFakeTail(jobGraph, numSubTasks);
-		JobOutputVertex wsFakeTail = createFakeTail(jobGraph, numSubTasks);
-		JobOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		SimpleOutputVertex ssFakeTail = createFakeTail(jobGraph, numSubTasks);
+		SimpleOutputVertex wsFakeTail = createFakeTail(jobGraph, numSubTasks);
+		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ss join) ----------------------
 		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
@@ -623,8 +624,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Intermediate Workset Update, Solution Set Tail)");
 
 		// input
-		JobInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		JobInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
@@ -636,9 +637,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		JobOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		SimpleOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ws update) ----------------------
 		JobTaskVertex wsUpdateIntermediate =
@@ -749,8 +750,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Intermediate Solution Set Update, Workset Tail)");
 
 		// input
-		JobInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		JobInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
@@ -760,9 +761,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		JobOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		JobOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		JobOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		SimpleOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ss update) ----------------------
 		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
index 5a6e4f5..8246d22 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/IterationWithChainingNepheleITCase.java
@@ -33,11 +33,12 @@ import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.GroupReduceDriver;
@@ -130,7 +131,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		// --------------------------------------------------------------------------------------------------------------
 
 		// - input -----------------------------------------------------------------------------------------------------
-		JobInputVertex input = JobGraphUtils.createInput(
+		InputFormatInputVertex input = JobGraphUtils.createInput(
 			new PointInFormat(), inputPath, "Input", jobGraph, numSubTasks);
 		TaskConfig inputConfig = new TaskConfig(input.getConfiguration());
 		{
@@ -213,7 +214,7 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - output ----------------------------------------------------------------------------------------------------
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
+		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		{
 			outputConfig.addInputToGroup(0);
@@ -224,10 +225,10 @@ public class IterationWithChainingNepheleITCase extends RecordAPITestBase {
 		}
 
 		// - fake tail -------------------------------------------------------------------------------------------------
-		JobOutputVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks);
+		SimpleOutputVertex fakeTail = JobGraphUtils.createFakeOutput(jobGraph, "Fake Tail", numSubTasks);
 
 		// - sync ------------------------------------------------------------------------------------------------------
-		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
+		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(maxIterations);
 		syncConfig.setIterationId(ITERATION_ID);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
index 4370111..052c7ea 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
@@ -33,11 +33,12 @@ import org.apache.flink.runtime.iterative.io.FakeOutputTask;
 import org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.DataSinkTask;
 import org.apache.flink.runtime.operators.DataSourceTask;
 import org.apache.flink.runtime.operators.RegularPactTask;
@@ -55,17 +56,17 @@ public class JobGraphUtils {
 		client.submitJobAndWait();
 	}
 	
-	public static <T extends FileInputFormat<?>> JobInputVertex createInput(T stub, String path, String name, JobGraph graph,
+	public static <T extends FileInputFormat<?>> InputFormatInputVertex createInput(T stub, String path, String name, JobGraph graph,
 			int degreeOfParallelism)
 	{
 		stub.setFilePath(path);
 		return createInput(new UserCodeObjectWrapper<T>(stub), name, graph, degreeOfParallelism);
 	}
 
-	private static <T extends InputFormat<?,?>> JobInputVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
+	private static <T extends InputFormat<?,?>> InputFormatInputVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
 			int degreeOfParallelism)
 	{
-		JobInputVertex inputVertex = new JobInputVertex(name, graph);
+		InputFormatInputVertex inputVertex = new InputFormatInputVertex(name, graph);
 		
 		inputVertex.setInvokableClass(DataSourceTask.class);
 		
@@ -99,8 +100,8 @@ public class JobGraphUtils {
 		return taskVertex;
 	}
 
-	public static JobOutputVertex createSync(JobGraph jobGraph, int degreeOfParallelism) {
-		JobOutputVertex sync = new JobOutputVertex("BulkIterationSync", jobGraph);
+	public static SimpleOutputVertex createSync(JobGraph jobGraph, int degreeOfParallelism) {
+		SimpleOutputVertex sync = new SimpleOutputVertex("BulkIterationSync", jobGraph);
 		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
 		sync.setNumberOfSubtasks(1);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
@@ -108,17 +109,17 @@ public class JobGraphUtils {
 		return sync;
 	}
 
-	public static JobOutputVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
+	public static SimpleOutputVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
-		JobOutputVertex outputVertex = new JobOutputVertex(name, jobGraph);
+		SimpleOutputVertex outputVertex = new SimpleOutputVertex(name, jobGraph);
 		outputVertex.setInvokableClass(FakeOutputTask.class);
 		outputVertex.setNumberOfSubtasks(degreeOfParallelism);
 		return outputVertex;
 	}
 
-	public static JobOutputVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
+	public static OutputFormatOutputVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
 	{
-		JobOutputVertex sinkVertex = new JobOutputVertex(name, jobGraph);
+		OutputFormatOutputVertex sinkVertex = new OutputFormatOutputVertex(name, jobGraph);
 		sinkVertex.setInvokableClass(DataSinkTask.class);
 		sinkVertex.setNumberOfSubtasks(degreeOfParallelism);
 		return sinkVertex;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
index 85cedba..aea2c2c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRank.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.iterative.nephele.customdanglingpagerank;
 
 import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
@@ -31,10 +30,11 @@ import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
@@ -138,7 +138,7 @@ public class CustomCompensatableDanglingPageRank {
 		// --------------- the inputs ---------------------
 
 		// page rank input
-		JobInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
+		InputFormatInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
 			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -147,7 +147,7 @@ public class CustomCompensatableDanglingPageRank {
 		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
 
 		// edges as adjacency list
-		JobInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
+		InputFormatInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
 			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -267,7 +267,7 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -276,10 +276,10 @@ public class CustomCompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
+		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
 			degreeOfParallelism);
 
-		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
index c60f905..a740cf3 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/customdanglingpagerank/CustomCompensatableDanglingPageRankWithCombiner.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.test.iterative.nephele.customdanglingpagerank;
 
 import org.apache.flink.api.common.operators.util.UserCodeClassWrapper;
@@ -31,10 +30,11 @@ import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
@@ -138,7 +138,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		// --------------- the inputs ---------------------
 
 		// page rank input
-		JobInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
+		InputFormatInputVertex pageWithRankInput = JobGraphUtils.createInput(new CustomImprovedDanglingPageRankInputFormat(),
 			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -147,7 +147,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
 
 		// edges as adjacency list
-		JobInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
+		InputFormatInputVertex adjacencyListInput = JobGraphUtils.createInput(new CustomImprovedAdjacencyListInputFormat(),
 			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -279,7 +279,7 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the output ---------------------
 
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(vertexWithRankAndDanglingSerializer, 0);
@@ -288,10 +288,9 @@ public class CustomCompensatableDanglingPageRankWithCombiner {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
-			degreeOfParallelism);
+		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", degreeOfParallelism);
 
-		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CustomCompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());


[20/63] [abbrv] git commit: Introduce execution attempts at execution vertex. Add tests for job event classes

Posted by se...@apache.org.
Introduce execution attempts at execution vertex.
Add tests for job event classes


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/43e7d0fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/43e7d0fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/43e7d0fa

Branch: refs/heads/master
Commit: 43e7d0fa0a2b71ed615039bcf9e708523b015cf3
Parents: 2ac08a6
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 9 21:12:20 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../executiongraph/ExecutionAttempt.java        | 111 +++++++++++++
 .../runtime/executiongraph/ExecutionGate.java   |  96 ------------
 .../executiongraph/ExecutionVertex2.java        |  57 +++++--
 .../flink/runtime/event/job/EventsTest.java     | 154 +++++++++++++++++++
 4 files changed, 307 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/43e7d0fa/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java
new file mode 100644
index 0000000..b623d6f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttempt.java
@@ -0,0 +1,111 @@
+/**
+ * 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 java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+
+public class ExecutionAttempt {
+
+	private final AtomicBoolean finished = new AtomicBoolean();
+	
+	private final ExecutionAttemptID attemptId;
+	
+	private final AllocatedSlot assignedResource;
+	
+	private final int attemptNumber;
+	
+	private final long startTimestamp;
+	
+	private volatile long endTimestamp;
+	
+	private volatile Throwable failureCause;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public ExecutionAttemptID getAttemptId() {
+		return attemptId;
+	}
+	
+	public ExecutionAttempt(ExecutionAttemptID attemptId, AllocatedSlot assignedResource, int attemptNumber, long startTimestamp) {
+		this.attemptId = attemptId;
+		this.assignedResource = assignedResource;
+		this.attemptNumber = attemptNumber;
+		this.startTimestamp = startTimestamp;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+
+	public AllocatedSlot getAssignedResource() {
+		return assignedResource;
+	}
+	
+	public int getAttemptNumber() {
+		return attemptNumber;
+	}
+	
+	public long getStartTimestamp() {
+		return startTimestamp;
+	}
+	
+	public long getEndTimestamp() {
+		return endTimestamp;
+	}
+	
+	public Throwable getFailureCause() {
+		return failureCause;
+	}
+	
+	public boolean isFinished() {
+		return finished.get();
+	}
+	
+	public boolean isFailed() {
+		return finished.get() && failureCause != null;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public boolean finish() {
+		if (finished.compareAndSet(false, true)) {
+			endTimestamp = System.currentTimeMillis();
+			return true;
+		} else {
+			return false;
+		}
+	}
+	
+	public boolean fail(Throwable error) {
+		if (finished.compareAndSet(false, true)) {
+			failureCause = error;
+			return true;
+		} else {
+			return false;
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return String.format("Attempt #%d (%s) @ %s - started %d %s", attemptNumber, attemptId,
+				assignedResource.toString(), startTimestamp, isFinished() ? "finished " + endTimestamp : "[RUNNING]");
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/43e7d0fa/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGate.java
deleted file mode 100644
index 1bfb923..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGate.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/**
- * 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 java.util.Collection;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-import org.apache.flink.runtime.io.network.gates.GateID;
-
-/**
- * Objects of this class represent either an {@link InputGate} or {@link OutputGate} within an {@link ExecutionGraph},
- * Nephele's internal scheduling representation for jobs.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class ExecutionGate {
-
-	private final GateID gateID;
-
-	private volatile ExecutionVertex vertex;
-
-	private final ExecutionGroupEdge groupEdge;
-
-	private final boolean isInputGate;
-
-	private final CopyOnWriteArrayList<ExecutionEdge> edges = new CopyOnWriteArrayList<ExecutionEdge>();
-
-	ExecutionGate(final GateID gateID, final ExecutionVertex vertex, final ExecutionGroupEdge groupEdge,
-			final boolean isInputGate) {
-
-		this.gateID = gateID;
-		this.vertex = vertex;
-		this.groupEdge = groupEdge;
-		this.isInputGate = isInputGate;
-	}
-
-	public GateID getGateID() {
-
-		return this.gateID;
-	}
-
-	public ExecutionVertex getVertex() {
-
-		return this.vertex;
-	}
-
-	public boolean isInputGate() {
-
-		return this.isInputGate;
-	}
-
-	public int getNumberOfEdges() {
-
-		return this.edges.size();
-	}
-
-	public ExecutionEdge getEdge(final int index) {
-
-		return this.edges.get(index);
-	}
-
-	void replaceAllEdges(final Collection<ExecutionEdge> newEdges) {
-		
-		this.edges.clear();
-		this.edges.addAll(newEdges);
-	}
-
-	public ChannelType getChannelType() {
-
-		return this.groupEdge.getChannelType();
-	}
-
-	ExecutionGroupEdge getGroupEdge() {
-
-		return this.groupEdge;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/43e7d0fa/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
index 4a4d0e7..787605d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.executiongraph;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import org.apache.commons.logging.Log;
@@ -65,8 +66,8 @@ public class ExecutionVertex2 {
 	private static final AtomicReferenceFieldUpdater<ExecutionVertex2, ExecutionState2> STATE_UPDATER =
 			AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, ExecutionState2.class, "state");
 	
-	private static final AtomicReferenceFieldUpdater<ExecutionVertex2, AllocatedSlot> ASSIGNED_SLOT_UPDATER =
-			AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, AllocatedSlot.class, "assignedSlot");
+	private static final AtomicReferenceFieldUpdater<ExecutionVertex2, ExecutionAttempt> ATTEMPT_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, ExecutionAttempt.class, "currentOrLastAttempt");
 
 	private static final Logger LOG = ExecutionGraph.LOG;
 	
@@ -82,15 +83,13 @@ public class ExecutionVertex2 {
 	
 	private final int subTaskIndex;
 	
-	
 	private final long[] stateTimestamps;
 	
+	private final List<ExecutionAttempt> priorAttempts;
 	
-	private volatile ExecutionState2 state = CREATED;
-	
-	private volatile AllocatedSlot assignedSlot;
+	private volatile ExecutionAttempt currentOrLastAttempt;
 	
-	private volatile Throwable failureCause;
+	private volatile ExecutionState2 state = CREATED;
 	
 	
 	public ExecutionVertex2(ExecutionJobVertex jobVertex, int subTaskIndex, IntermediateResult[] producedDataSets) {
@@ -107,6 +106,7 @@ public class ExecutionVertex2 {
 		this.inputEdges = new ExecutionEdge2[jobVertex.getJobVertex().getInputs().size()][];
 		
 		this.stateTimestamps = new long[ExecutionState2.values().length];
+		this.priorAttempts = new CopyOnWriteArrayList<ExecutionAttempt>();
 	}
 	
 	
@@ -149,21 +149,48 @@ public class ExecutionVertex2 {
 		return state;
 	}
 	
-	public Throwable getFailureCause() {
-		return failureCause;
+	public long getStateTimestamp(ExecutionState2 state) {
+		return this.stateTimestamps[state.ordinal()];
 	}
 	
-	public AllocatedSlot getAssignedResource() {
-		return assignedSlot;
+	private ExecutionGraph getExecutionGraph() {
+		return this.jobVertex.getGraph();
 	}
 	
-	public long getStateTimestamp(ExecutionState2 state) {
-		return this.stateTimestamps[state.ordinal()];
+	public Throwable getLastFailureCause() {
+		// copy reference to the stack
+		ExecutionAttempt attempt = this.currentOrLastAttempt;
+		if (attempt != null) {
+			return attempt.getFailureCause();
+		}
+		else if (priorAttempts.size() > 0) {
+			// since the list is append-only, this always works in the presence of concurrent modifications
+			return priorAttempts.get(priorAttempts.size() - 1).getFailureCause();
+		}
+		else {
+			return null;
+		}
 	}
 	
+	public AllocatedSlot getCurrentAssignedResource() {
+		// copy reference to the stack
+		ExecutionAttempt attempt = this.currentOrLastAttempt;
+		return attempt == null ? null : attempt.getAssignedResource();
+	}
 	
-	private ExecutionGraph getExecutionGraph() {
-		return this.jobVertex.getGraph();
+	public AllocatedSlot getLastAssignedResource() {
+		// copy reference to the stack
+		ExecutionAttempt attempt = this.currentOrLastAttempt;
+		if (attempt != null) {
+			return attempt.getAssignedResource();
+		}
+		else if (priorAttempts.size() > 0) {
+			// since the list is append-only, this always works in the presence of concurrent modifications
+			return priorAttempts.get(priorAttempts.size() - 1).getAssignedResource();
+		}
+		else {
+			return null;
+		}
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/43e7d0fa/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java
new file mode 100644
index 0000000..ff6732b
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/EventsTest.java
@@ -0,0 +1,154 @@
+/**
+ * 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.event.job;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.junit.Test;
+
+
+public class EventsTest {
+
+	@Test
+	public void testEqualsHashCodeToString() {
+		try {
+			
+			// ExecutionStateChangeEvent
+			{
+				JobVertexID jid = new JobVertexID();
+				ExecutionAttemptID eid = new ExecutionAttemptID();
+				
+				ExecutionStateChangeEvent e1 = new ExecutionStateChangeEvent(429345231796596L, jid, 17, eid, ExecutionState2.CANCELING);
+				ExecutionStateChangeEvent e2 = new ExecutionStateChangeEvent(429345231796596L, jid, 17, eid, ExecutionState2.CANCELING);
+				
+				assertTrue(e1.equals(e2));
+				assertEquals(e1.hashCode(), e2.hashCode());
+				e1.toString();
+			}
+			
+			// JobEvent
+			{
+				JobEvent e1 = new JobEvent(429345231796596L, JobStatus.CANCELED, "testmessage");
+				JobEvent e2 = new JobEvent(429345231796596L, JobStatus.CANCELED, "testmessage");
+				JobEvent e3 = new JobEvent(237217579123412431L, JobStatus.RUNNING, null);
+				JobEvent e4 = new JobEvent(237217579123412431L, JobStatus.RUNNING, null);
+				
+				assertTrue(e1.equals(e2));
+				assertTrue(e3.equals(e4));
+				
+				assertEquals(e1.hashCode(), e2.hashCode());
+				assertEquals(e3.hashCode(), e4.hashCode());
+				e1.toString();
+				e3.toString();
+			}
+			
+			// RecentJobEvent
+			{
+				JobID jid = new JobID();
+				RecentJobEvent e1 = new RecentJobEvent(jid, "some name", JobStatus.FAILED, false, 634563546, 546734734672572457L);
+				RecentJobEvent e2 = new RecentJobEvent(jid, "some name", JobStatus.FAILED, false, 634563546, 546734734672572457L);
+				RecentJobEvent e3 = new RecentJobEvent(jid, null, JobStatus.RUNNING, false, 42364716239476L, 127843618273607L);
+				RecentJobEvent e4 = new RecentJobEvent(jid, null, JobStatus.RUNNING, false, 42364716239476L, 127843618273607L);
+				
+				assertTrue(e1.equals(e2));
+				assertTrue(e3.equals(e4));
+				
+				assertEquals(e1.hashCode(), e2.hashCode());
+				assertEquals(e3.hashCode(), e4.hashCode());
+				e1.toString();
+				e3.toString();
+			}
+			
+			// VertexEvent
+			{
+				JobVertexID jid = new JobVertexID();
+				ExecutionAttemptID eid = new ExecutionAttemptID();
+				
+				VertexEvent e1 = new VertexEvent(64619276017401234L, jid, "peter", 44, 13, eid, ExecutionState2.DEPLOYING, "foo");
+				VertexEvent e2 = new VertexEvent(64619276017401234L, jid, "peter", 44, 13, eid, ExecutionState2.DEPLOYING, "foo");
+				
+				VertexEvent e3 = new VertexEvent(64619276017401234L, jid, null, 44, 13, eid, ExecutionState2.DEPLOYING, null);
+				VertexEvent e4 = new VertexEvent(64619276017401234L, jid, null, 44, 13, eid, ExecutionState2.DEPLOYING, null);
+				
+				assertTrue(e1.equals(e2));
+				assertTrue(e3.equals(e4));
+				
+				assertFalse(e1.equals(e3));
+				
+				assertEquals(e1.hashCode(), e2.hashCode());
+				assertEquals(e3.hashCode(), e4.hashCode());
+				e1.toString();
+				e3.toString();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialization() {
+		try {
+			JobID jid = new JobID();
+			JobVertexID vid = new JobVertexID();
+			ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			ExecutionStateChangeEvent esce = new ExecutionStateChangeEvent(429345231796596L, vid, 17, eid, ExecutionState2.CANCELING);
+
+			JobEvent je1 = new JobEvent(429345231796596L, JobStatus.CANCELED, "testmessage");
+			JobEvent je2 = new JobEvent(237217579123412431L, JobStatus.RUNNING, null);
+
+			RecentJobEvent rce1 = new RecentJobEvent(jid, "some name", JobStatus.FAILED, false, 634563546, 546734734672572457L);
+			RecentJobEvent rce2 = new RecentJobEvent(jid, null, JobStatus.RUNNING, false, 42364716239476L, 127843618273607L);
+
+			VertexEvent ve1 = new VertexEvent(64619276017401234L, vid, "peter", 44, 13, eid, ExecutionState2.DEPLOYING, "foo");
+			VertexEvent ve2 = new VertexEvent(64619276017401234L, vid, null, 44, 13, eid, ExecutionState2.DEPLOYING, null);
+			
+			assertEquals(esce, CommonTestUtils.createCopyWritable(esce));
+			assertEquals(je1, CommonTestUtils.createCopyWritable(je1));
+			assertEquals(je2, CommonTestUtils.createCopyWritable(je2));
+			assertEquals(rce1, CommonTestUtils.createCopyWritable(rce1));
+			assertEquals(rce2, CommonTestUtils.createCopyWritable(rce2));
+			assertEquals(ve1, CommonTestUtils.createCopyWritable(ve1));
+			assertEquals(ve2, CommonTestUtils.createCopyWritable(ve2));
+			
+			assertEquals(esce, CommonTestUtils.createCopySerializable(esce));
+			assertEquals(je1, CommonTestUtils.createCopySerializable(je1));
+			assertEquals(je2, CommonTestUtils.createCopySerializable(je2));
+			assertEquals(rce1, CommonTestUtils.createCopySerializable(rce1));
+			assertEquals(rce2, CommonTestUtils.createCopySerializable(rce2));
+			assertEquals(ve1, CommonTestUtils.createCopySerializable(ve1));
+			assertEquals(ve2, CommonTestUtils.createCopySerializable(ve2));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[43/63] [abbrv] git commit: Adjust ExecutionGraph state machine to TaskManager's failing model (direct transitions to canceled)

Posted by se...@apache.org.
Adjust ExecutionGraph state machine to TaskManager's failing model (direct transitions to canceled)


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/25acb6ba
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/25acb6ba
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/25acb6ba

Branch: refs/heads/master
Commit: 25acb6ba7724f40ac041a499b607ac0206eadc97
Parents: ae139f5
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Sep 11 16:31:50 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../flink/core/memory/DataInputViewStream.java  |   2 +-
 .../deployment/TaskDeploymentDescriptor.java    |   4 +
 .../runtime/execution/RuntimeEnvironment.java   |  62 +-
 .../flink/runtime/executiongraph/Execution.java | 181 +++--
 .../runtime/executiongraph/ExecutionGraph.java  | 267 ++++----
 .../executiongraph/ExecutionJobVertex.java      |   4 +-
 .../runtime/executiongraph/ExecutionVertex.java |   5 +-
 .../executiongraph/IntermediateResult.java      |   9 +
 .../IntermediateResultPartition.java            |   1 +
 .../runtime/instance/InstanceDiedException.java |  25 +-
 .../runtime/io/network/ChannelManager.java      |   2 +-
 .../apache/flink/runtime/jobgraph/JobGraph.java |  12 +
 .../flink/runtime/jobgraph/JobStatus.java       |  27 +-
 .../runtime/jobmanager/EventCollector.java      |   6 +
 .../flink/runtime/jobmanager/JobManager.java    |  20 +-
 .../jobmanager/scheduler/DefaultScheduler.java  | 413 ------------
 .../runtime/jobmanager/scheduler/Scheduler.java | 447 +++++++++++++
 .../apache/flink/runtime/taskmanager/Task.java  |  35 +-
 .../flink/runtime/taskmanager/TaskManager.java  |  39 +-
 .../runtime/util/ExecutorThreadFactory.java     |   4 +-
 .../ExecutionGraphDeploymentTest.java           |   4 +-
 .../ExecutionVertexCancelTest.java              |  45 +-
 .../ExecutionVertexSchedulingTest.java          |   8 +-
 .../runtime/jobmanager/JobManagerITCase.java    | 670 ++++++++++++++++++-
 .../scheduler/SchedulerIsolatedTasksTest.java   |  12 +-
 .../scheduler/SchedulerSlotSharingTest.java     |  14 +-
 .../runtime/taskmanager/TaskManagerTest.java    |  14 -
 .../util/EnvironmentInformationTest.java        |  25 +-
 .../runtime/util/KeyGroupedIteratorTest.java    |   1 -
 29 files changed, 1623 insertions(+), 735 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java
index 25835f5..8fe7b03 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java
@@ -36,7 +36,7 @@ public final class DataInputViewStream extends InputStream {
 		return inputView.readByte();
 	}
 	
-	public int read(byte b[], int off, int len) throws IOException {
+	public int read(byte[] b, int off, int len) throws IOException {
 		inputView.readFully(b, off, len);
 		return len;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
index 2d00f40..e1e80f9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
@@ -228,6 +228,10 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 		return inputGates;
 	}
 	
+	public String[] getRequiredJarFiles() {
+		return requiredJarFiles;
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	//  Serialization
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
index eef081c..79a4aaa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
@@ -113,7 +113,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	
 	/** The thread executing the task in the environment. */
-	private volatile Thread executingThread;
+	private Thread executingThread;
 
 	/**
 	 * The RPC proxy to report accumulators to JobManager
@@ -221,7 +221,6 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	@Override
 	public void run() {
-
 		// quick fail in case the task was cancelled while the tread was started
 		if (owner.isCanceled()) {
 			owner.cancelingDone();
@@ -238,6 +237,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			}
 		}
 		catch (Throwable t) {
+			
 			if (!this.owner.isCanceled()) {
 
 				// Perform clean up when the task failed and has been not canceled by the user
@@ -260,7 +260,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 			return;
 		}
-
+		
 		try {
 			// If there is any unclosed input gate, close it and propagate close operation to corresponding output gate
 			closeInputGates();
@@ -275,6 +275,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			waitForOutputChannelsToBeClosed();
 		}
 		catch (Throwable t) {
+			
 			// Release all resources that may currently be allocated by the individual channels
 			releaseAllChannelResources();
 
@@ -396,37 +397,38 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 		}
 		
 		final Thread executingThread = this.executingThread;
-		
-		// interrupt the running thread and wait for it to die
-		executingThread.interrupt();
-		
-		try {
-			executingThread.join(5000);
-		} catch (InterruptedException e) {}
-		
-		if (!executingThread.isAlive()) {
-			return;
-		}
-		
-		// Continuously interrupt the user thread until it changed to state CANCELED
-		while (executingThread != null && executingThread.isAlive()) {
-			LOG.warn("Task " + owner.getTaskNameWithSubtasks() + " did not react to cancelling signal. Sending repeated interrupt.");
-
-			if (LOG.isDebugEnabled()) {
-				StringBuilder bld = new StringBuilder("Task ").append(owner.getTaskNameWithSubtasks()).append(" is stuck in method:\n");
-				
-				StackTraceElement[] stack = executingThread.getStackTrace();
-				for (StackTraceElement e : stack) {
-					bld.append(e).append('\n');
-				}
-				LOG.debug(bld.toString());
-			}
-			
+		if (executingThread != null) {
+			// interrupt the running thread and wait for it to die
 			executingThread.interrupt();
 			
 			try {
-				executingThread.join(1000);
+				executingThread.join(5000);
 			} catch (InterruptedException e) {}
+			
+			if (!executingThread.isAlive()) {
+				return;
+			}
+			
+			// Continuously interrupt the user thread until it changed to state CANCELED
+			while (executingThread != null && executingThread.isAlive()) {
+				LOG.warn("Task " + owner.getTaskNameWithSubtasks() + " did not react to cancelling signal. Sending repeated interrupt.");
+	
+				if (LOG.isDebugEnabled()) {
+					StringBuilder bld = new StringBuilder("Task ").append(owner.getTaskNameWithSubtasks()).append(" is stuck in method:\n");
+					
+					StackTraceElement[] stack = executingThread.getStackTrace();
+					for (StackTraceElement e : stack) {
+						bld.append(e).append('\n');
+					}
+					LOG.debug(bld.toString());
+				}
+				
+				executingThread.interrupt();
+				
+				try {
+					executingThread.join(1000);
+				} catch (InterruptedException e) {}
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index 8cfc7fd..c290883 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -18,7 +18,14 @@
 
 package org.apache.flink.runtime.executiongraph;
 
-import static org.apache.flink.runtime.execution.ExecutionState.*;
+import static org.apache.flink.runtime.execution.ExecutionState.CANCELED;
+import static org.apache.flink.runtime.execution.ExecutionState.CANCELING;
+import static org.apache.flink.runtime.execution.ExecutionState.CREATED;
+import static org.apache.flink.runtime.execution.ExecutionState.DEPLOYING;
+import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
+import static org.apache.flink.runtime.execution.ExecutionState.FINISHED;
+import static org.apache.flink.runtime.execution.ExecutionState.RUNNING;
+import static org.apache.flink.runtime.execution.ExecutionState.SCHEDULED;
 
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
@@ -27,9 +34,9 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
@@ -124,6 +131,10 @@ public class Execution {
 		return failureCause;
 	}
 	
+	public long[] getStateTimestamps() {
+		return stateTimestamps;
+	}
+	
 	public long getStateTimestamp(ExecutionState state) {
 		return this.stateTimestamps[state.ordinal()];
 	}
@@ -146,16 +157,13 @@ public class Execution {
 	 * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only state that permits scheduling.
 	 * @throws NoResourceAvailableException Thrown is no queued scheduling is allowed and no resources are currently available.
 	 */
-	public void scheduleForExecution(DefaultScheduler scheduler, boolean queued) throws NoResourceAvailableException {
+	public void scheduleForExecution(Scheduler scheduler, boolean queued) throws NoResourceAvailableException {
 		if (scheduler == null) {
 			throw new NullPointerException();
 		}
 		
 		if (transitionState(CREATED, SCHEDULED)) {
 			
-			// record that we were scheduled
-			vertex.notifyStateTransition(attemptId, SCHEDULED, null);
-			
 			ScheduledUnit toSchedule = new ScheduledUnit(this, vertex.getJobVertex().getSlotSharingGroup());
 		
 			// IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned
@@ -221,8 +229,6 @@ public class Execution {
 				// this should actually not happen and indicates a race somewhere else
 				throw new IllegalStateException("Cannot deploy task: Concurrent deployment call race.");
 			}
-			
-			vertex.notifyStateTransition(attemptId, DEPLOYING, null);
 		}
 		else {
 			// vertex may have been cancelled, or it was already scheduled
@@ -236,9 +242,15 @@ public class Execution {
 			}
 			this.assignedResource = slot;
 			
+			// race double check, did we fail/cancel and do we need to release the slot?
+			if (this.state != DEPLOYING) {
+				slot.releaseSlot();
+				return;
+			}
+			
 			final TaskDeploymentDescriptor deployment = vertex.createDeploymentDescriptor(attemptId, slot);
 			
-			// register this execution at the execution graph, to receive callbacks
+			// register this execution at the execution graph, to receive call backs
 			vertex.getExecutionGraph().registerExecution(this);
 			
 			// we execute the actual deploy call in a concurrent action to prevent this call from blocking for long
@@ -300,7 +312,6 @@ public class Execution {
 			else if (current == RUNNING || current == DEPLOYING) {
 				// try to transition to canceling, if successful, send the cancel call
 				if (transitionState(current, CANCELING)) {
-					vertex.notifyStateTransition(attemptId, CANCELING, null);
 					sendCancelRpcCall();
 					return;
 				}
@@ -318,7 +329,7 @@ public class Execution {
 					
 					// we skip the canceling state. set the timestamp, for a consistent appearance
 					markTimestamp(CANCELING, getStateTimestamp(CANCELED));
-					vertex.notifyStateTransition(attemptId, CANCELED, null);
+					vertex.executionCanceled();
 					return;
 				}
 				// else: fall through the loop
@@ -336,11 +347,7 @@ public class Execution {
 	 * @param t The exception that caused the task to fail.
 	 */
 	public void fail(Throwable t) {
-		if (processFail(t, false)) {
-			if (LOG.isErrorEnabled()) {
-				LOG.error("Task " + getVertexWithAttempt() + " was failed.", t);
-			}
-		}
+		processFail(t, false);
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -355,15 +362,11 @@ public class Execution {
 	 * @param t The exception that caused the task to fail.
 	 */
 	void markFailed(Throwable t) {
-		// the call returns true if it actually made the state transition (was not already failed before, etc)
-		if (processFail(t, true)) {
-			if (LOG.isErrorEnabled()) {
-				LOG.error("Task " + getVertexWithAttempt() + " failed.", t);
-			}
-		}
+		processFail(t, true);
 	}
 	
 	void markFinished() {
+		
 		// this call usually comes during RUNNING, but may also come while still in deploying (very fast tasks!)
 		while (true) {
 			ExecutionState current = this.state;
@@ -372,7 +375,6 @@ public class Execution {
 			
 				if (transitionState(current, FINISHED)) {
 					try {
-						vertex.notifyStateTransition(attemptId, FINISHED, null);
 						vertex.executionFinished();
 						return;
 					}
@@ -382,41 +384,60 @@ public class Execution {
 					}
 				}
 			}
-			else {
-				if (current == CANCELED || current == CANCELING || current == FAILED) {
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Task FINISHED, but concurrently went to state " + state);
-					}
-					return;
-				}
-				else {
-					// this should not happen, we need to fail this
-					markFailed(new Exception("Vertex received FINISHED message while being in state " + state));
-					return;
+			else if (current == CANCELING) {
+				// we sent a cancel call, and the task manager finished before it arrived. We
+				// will never get a CANCELED call back from the job manager
+				cancelingComplete();
+				return;
+			}
+			else if (current == CANCELED || current == FAILED) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Task FINISHED, but concurrently went to state " + state);
 				}
+				return;
+			}
+			else {
+				// this should not happen, we need to fail this
+				markFailed(new Exception("Vertex received FINISHED message while being in state " + state));
+				return;
 			}
 		}
 	}
 	
 	void cancelingComplete() {
-		if (transitionState(CANCELING, CANCELED)) {
-			try {
-				vertex.executionCanceled();
-				vertex.notifyStateTransition(attemptId, CANCELED, null);
-			}
-			finally {
-				vertex.getExecutionGraph().deregisterExecution(this);
-				assignedResource.releaseSlot();
+		
+		// the taskmanagers can themselves cancel tasks without an external trigger, if they find that the
+		// network stack is canceled (for example by a failing / canceling receiver or sender
+		// this is an artifact of the old network runtime, but for now we need to support task transitions
+		// from running directly to canceled
+		
+		while (true) {
+			ExecutionState current = this.state;
+			
+			if (current == CANCELED) {
+				return;
 			}
-		}
-		else {
-			ExecutionState actualState = this.state;
-			// failing in the meantime may happen and is no problem.
-			// anything else is a serious problem !!!
-			if (actualState != FAILED) {
-				String message = String.format("Asynchronous race: Found state %s after successful cancel call.", state);
-				LOG.error(message);
-				vertex.getExecutionGraph().fail(new Exception(message));
+			else if (current == CANCELING || current == RUNNING) {
+				if (transitionState(current, CANCELED)) {
+					try {
+						vertex.executionCanceled();
+					}
+					finally {
+						vertex.getExecutionGraph().deregisterExecution(this);
+						assignedResource.releaseSlot();
+					}
+					return;
+				}
+			} 
+			else {
+				// failing in the meantime may happen and is no problem.
+				// anything else is a serious problem !!!
+				if (current != FAILED) {
+					String message = String.format("Asynchronous race: Found state %s after successful cancel call.", state);
+					LOG.error(message);
+					vertex.getExecutionGraph().fail(new Exception(message));
+				}
+				return;
 			}
 		}
 	}
@@ -440,17 +461,29 @@ public class Execution {
 				return false;
 			}
 			
-			if (current == CANCELED || (current == CANCELING && isCallback)) {
+			if (current == CANCELED) {
 				// we are already aborting or are already aborted
 				if (LOG.isDebugEnabled()) {
 					LOG.debug(String.format("Ignoring transition of vertex %s to %s while being %s", 
-							getVertexWithAttempt(), FAILED, current));
+							getVertexWithAttempt(), FAILED, CANCELED));
 				}
 				return false;
 			}
 			
-			if (transitionState(current, FAILED)) {
+			if (transitionState(current, FAILED, t)) {
 				// success (in a manner of speaking)
+				this.failureCause = t;
+				
+				try {
+					vertex.getExecutionGraph().deregisterExecution(this);
+					vertex.executionFailed(t);
+				}
+				finally {
+					if (assignedResource != null) {
+						assignedResource.releaseSlot();
+					}
+				}
+				
 				
 				if (!isCallback && (current == RUNNING || current == DEPLOYING)) {
 					if (LOG.isDebugEnabled()) {
@@ -467,29 +500,16 @@ public class Execution {
 					}
 				}
 				
-				try {
-					this.failureCause = t;
-					vertex.executionFailed(t);
-					vertex.notifyStateTransition(attemptId, FAILED, t);
-				}
-				finally {
-					if (assignedResource != null) {
-						assignedResource.releaseSlot();
-					}
-					vertex.getExecutionGraph().deregisterExecution(this);
-				}
-				
 				// leave the loop
 				return true;
 			}
 		}
 	}
 	
-	private void switchToRunning() {
+	private boolean switchToRunning() {
 		
-		// transition state, the common case
 		if (transitionState(DEPLOYING, RUNNING)) {
-			vertex.notifyStateTransition(attemptId, RUNNING, null);
+			return true;
 		}
 		else {
 			// something happened while the call was in progress.
@@ -501,10 +521,10 @@ public class Execution {
 			ExecutionState currentState = this.state;
 			
 			if (currentState == FINISHED || currentState == CANCELED) {
-				// do nothing, this is nice, the task was really fast
+				// do nothing, the task was really fast (nice)
+				// or it was canceled really fast
 			}
-			
-			if (currentState == CANCELING || currentState == FAILED) {
+			else if (currentState == CANCELING || currentState == FAILED) {
 				if (LOG.isDebugEnabled()) {
 					LOG.debug(String.format("Concurrent canceling/failing of %s while deployment was in progress.", getVertexWithAttempt()));
 				}
@@ -524,13 +544,15 @@ public class Execution {
 				// record the failure
 				markFailed(new Exception(message));
 			}
+			
+			return false;
 		}
 	}
 	
 	private void sendCancelRpcCall() {
 		final AllocatedSlot slot = this.assignedResource;
 		if (slot == null) {
-			throw new IllegalStateException("Cannot cancel when task was not running or deployed.");
+			return;
 		}
 		
 		Runnable cancelAction = new Runnable() {
@@ -578,8 +600,21 @@ public class Execution {
 	// --------------------------------------------------------------------------------------------
 	
 	private boolean transitionState(ExecutionState currentState, ExecutionState targetState) {
+		return transitionState(currentState, targetState, null);
+	}
+	
+	private boolean transitionState(ExecutionState currentState, ExecutionState targetState, Throwable error) {
 		if (STATE_UPDATER.compareAndSet(this, currentState, targetState)) {
 			markTimestamp(targetState);
+			
+			// make sure that the state transition completes normally.
+			// potential errors (in listeners may not affect the main logic)
+			try {
+				vertex.notifyStateTransition(attemptId, targetState, error);
+			}
+			catch (Throwable t) {
+				LOG.error("Error while notifying execution graph of execution state trnsition.", t);
+			}
 			return true;
 		} else {
 			return false;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 72525e9..3dab13e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.executiongraph;
 
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
@@ -37,15 +38,17 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.execution.ExecutionListener;
 import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
+import org.apache.flink.runtime.io.network.RemoteReceiver;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.util.ExceptionUtils;
 
@@ -105,7 +108,9 @@ public class ExecutionGraph {
 	private volatile JobStatus state = JobStatus.CREATED;
 	
 	
-	private boolean allowQueuedScheduling = false;
+	private Scheduler scheduler;
+	
+	private boolean allowQueuedScheduling = true;
 	
 	
 	public ExecutionGraph(JobID jobId, String jobName, Configuration jobConfig) {
@@ -265,12 +270,17 @@ public class ExecutionGraph {
 	//  Actions
 	// --------------------------------------------------------------------------------------------
 	
-	public void scheduleForExecution(DefaultScheduler scheduler) throws JobException {
+	public void scheduleForExecution(Scheduler scheduler) throws JobException {
 		if (scheduler == null) {
 			throw new IllegalArgumentException("Scheduler must not be null.");
 		}
 		
+		if (this.scheduler != null && this.scheduler != scheduler) {
+			throw new IllegalArgumentException("Cann not use different schedulers for the same job");
+		}
+		
 		if (transitionState(JobStatus.CREATED, JobStatus.RUNNING)) {
+			this.scheduler = scheduler;
 			
 			// initially, we simply take the ones without inputs.
 			// next, we implement the logic to go back from vertices that need computation
@@ -289,32 +299,35 @@ public class ExecutionGraph {
 	public void cancel() {
 		while (true) {
 			JobStatus current = state;
+			
 			if (current == JobStatus.RUNNING || current == JobStatus.CREATED) {
 				if (transitionState(current, JobStatus.CANCELLING)) {
 					for (ExecutionJobVertex ejv : verticesInCreationOrder) {
 						ejv.cancel();
 					}
+					return;
 				}
 			}
-			
-			// no need to treat other states
+			else {
+				// no need to treat other states
+				return;
+			}
 		}
 	}
 	
 	public void fail(Throwable t) {
-		if (LOG.isErrorEnabled()) {
-			LOG.error(String.format("Failing ExecutionGraph %s (%s): ", getJobID(), getJobName()), t);
-		}
-		
 		while (true) {
 			JobStatus current = state;
-			if (current != JobStatus.FAILED) {
-				if (transitionState(current, JobStatus.FAILED, t)) {
-					// cancel all. what is failed will not cancel but stay failed
-					for (ExecutionJobVertex ejv : verticesInCreationOrder) {
-						ejv.cancel();
-					}
+			if (current == JobStatus.FAILED || current == JobStatus.FAILING) {
+				return;
+			}
+			else if (transitionState(current, JobStatus.FAILING, t)) {
+				// cancel all. what is failed will not cancel but stay failed
+				for (ExecutionJobVertex ejv : verticesInCreationOrder) {
+					ejv.cancel();
 				}
+				
+				return;
 			}
 			
 			// no need to treat other states
@@ -362,6 +375,7 @@ public class ExecutionGraph {
 				nextVertexToFinish = nextPos;
 				
 				if (nextPos == verticesInCreationOrder.size()) {
+					
 					// we are done, transition to the final state
 					
 					while (true) {
@@ -372,7 +386,7 @@ public class ExecutionGraph {
 						if (current == JobStatus.CANCELLING && transitionState(current, JobStatus.CANCELED)) {
 							break;
 						}
-						if (current == JobStatus.FAILED) {
+						if (current == JobStatus.FAILING && transitionState(current, JobStatus.FAILED)) {
 							break;
 						}
 						if (current == JobStatus.CANCELED || current == JobStatus.CREATED || current == JobStatus.FINISHED) {
@@ -417,99 +431,118 @@ public class ExecutionGraph {
 	}
 	
 	public ConnectionInfoLookupResponse lookupConnectionInfoAndDeployReceivers(InstanceConnectionInfo caller, ChannelID sourceChannelID) {
-		//TODO
-		return null;
 		
-//		final InternalJobStatus jobStatus = eg.getJobStatus();
-//		if (jobStatus == InternalJobStatus.FAILING || jobStatus == InternalJobStatus.CANCELING) {
-//			return ConnectionInfoLookupResponse.createJobIsAborting();
-//		}
-//
-//		final ExecutionEdge edge = eg.getEdgeByID(sourceChannelID);
-//		if (edge == null) {
-//			LOG.error("Cannot find execution edge associated with ID " + sourceChannelID);
-//			return ConnectionInfoLookupResponse.createReceiverNotFound();
-//		}
-//
-//		if (sourceChannelID.equals(edge.getInputChannelID())) {
-//			// Request was sent from an input channel
-//
-//			final ExecutionVertex connectedVertex = edge.getOutputGate().getVertex();
-//
-//			final Instance assignedInstance = connectedVertex.getAllocatedResource().getInstance();
-//			if (assignedInstance == null) {
-//				LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getOutputGateIndex()
-//					+ " but no instance assigned");
-//				// LOG.info("Created receiverNotReady for " + connectedVertex + " 1");
-//				return ConnectionInfoLookupResponse.createReceiverNotReady();
-//			}
-//
-//			// Check execution state
-//			final ExecutionState executionState = connectedVertex.getExecutionState();
-//			if (executionState == ExecutionState.FINISHED) {
-//				// that should not happen. if there is data pending, the receiver cannot be ready
-//				return ConnectionInfoLookupResponse.createReceiverNotFound();
-//			}
-//
-//			// running is common, finishing is happens when the lookup is for the close event
-//			if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING) {
-//				// LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2");
-//				return ConnectionInfoLookupResponse.createReceiverNotReady();
-//			}
-//
-//			if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
-//				// Receiver runs on the same task manager
-//				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getOutputChannelID());
-//			} else {
-//				// Receiver runs on a different task manager
-//
-//				final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
-//				final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
-//
-//				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
-//			}
-//		}
-//		// else, the request is for an output channel
-//		// Find vertex of connected input channel
-//		final ExecutionVertex targetVertex = edge.getInputGate().getVertex();
-//
-//		// Check execution state
-//		final ExecutionState executionState = targetVertex.getExecutionState();
-//
-//		// check whether the task needs to be deployed
-//		if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) {
-//
-//			if (executionState == ExecutionState.ASSIGNED) {
-//				final Runnable command = new Runnable() {
-//					@Override
-//					public void run() {
-//						scheduler.deployAssignedVertices(targetVertex);
-//					}
-//				};
-//				eg.executeCommand(command);
-//			}
-//
-//			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3");
-//			return ConnectionInfoLookupResponse.createReceiverNotReady();
-//		}
-//
-//		final Instance assignedInstance = targetVertex.getAllocatedResource().getInstance();
-//		if (assignedInstance == null) {
-//			LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getInputChannelID() + " but no instance assigned");
-//			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4");
-//			return ConnectionInfoLookupResponse.createReceiverNotReady();
-//		}
-//
-//		if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
-//			// Receiver runs on the same task manager
-//			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelID());
-//		} else {
-//			// Receiver runs on a different task manager
-//			final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
-//			final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
-//
-//			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
-//		}
+		final ExecutionEdge edge = edges.get(sourceChannelID);
+		if (edge == null) {
+			// that is bad, we need to fail the job
+			LOG.error("Cannot find execution edge associated with ID " + sourceChannelID);
+			fail(new Exception("Channels are not correctly registered"));
+			return ConnectionInfoLookupResponse.createReceiverNotFound();
+		}
+		
+		
+		//  ----- Request was sent from an input channel (receiver side), requesting the output channel (sender side) ------
+		//  -----                               This is the case for backwards events                                 ------
+
+		if (sourceChannelID.equals(edge.getInputChannelId())) {
+			final ExecutionVertex targetVertex = edge.getSource().getProducer();
+			final ExecutionState executionState = targetVertex.getExecutionState();
+			
+			// common case - found task running
+			if (executionState == ExecutionState.RUNNING) {
+				Instance location = targetVertex.getCurrentAssignedResource().getInstance();
+				
+				if (location.getInstanceConnectionInfo().equals(caller)) {
+					// Receiver runs on the same task manager
+					return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getOutputChannelId());
+				}
+				else {
+					// Receiver runs on a different task manager
+					final InstanceConnectionInfo ici = location.getInstanceConnectionInfo();
+					final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
+
+					int connectionIdx = edge.getSource().getIntermediateResult().getConnectionIndex();
+					return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, connectionIdx));
+				}
+			}
+			else if (executionState == ExecutionState.FINISHED) {
+				// that should not happen. if there is data pending, the sender cannot yet be done
+				// we need to fail the whole affair
+				LOG.error("Receiver " + targetVertex + " set to FINISHED even though data is pending");
+				fail(new Exception("Channels are not correctly registered"));
+				return ConnectionInfoLookupResponse.createReceiverNotFound();
+			}
+			else if (executionState == ExecutionState.FAILED || executionState == ExecutionState.CANCELED ||
+					executionState == ExecutionState.CANCELING)
+			{
+				return ConnectionInfoLookupResponse.createJobIsAborting();
+			}
+			else {
+				// all other states should not be, because the sender cannot be in CREATED, SCHEDULED, or DEPLOYING
+				// state when the receiver is already running
+				LOG.error("Channel lookup (backwards) - sender " + targetVertex + " found in inconsistent state " + executionState);
+				fail(new Exception("Channels are not correctly registered"));
+				return ConnectionInfoLookupResponse.createReceiverNotFound();
+			}
+		}
+		
+		//  ----- Request was sent from an output channel (sender side), requesting the input channel (receiver side) ------
+		//  -----                                 This is the case for forward data                                   ------
+		
+		final ExecutionVertex targetVertex = edge.getTarget();
+		final ExecutionState executionState = targetVertex.getExecutionState();
+
+		if (executionState == ExecutionState.RUNNING) {
+			
+			// already online
+			Instance location = targetVertex.getCurrentAssignedResource().getInstance();
+			
+			if (location.getInstanceConnectionInfo().equals(caller)) {
+				// Receiver runs on the same task manager
+				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelId());
+			}
+			else {
+				// Receiver runs on a different task manager
+				final InstanceConnectionInfo ici = location.getInstanceConnectionInfo();
+				final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
+
+				final int connectionIdx = edge.getSource().getIntermediateResult().getConnectionIndex();
+				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, connectionIdx));
+			}
+		}
+		else if (executionState == ExecutionState.DEPLOYING || executionState == ExecutionState.SCHEDULED) {
+			return ConnectionInfoLookupResponse.createReceiverNotReady();
+		}
+		else if (executionState == ExecutionState.CREATED) {
+			// bring the receiver online
+			try {
+				edge.getTarget().scheduleForExecution(scheduler, false);
+				
+				// delay the requester
+				return ConnectionInfoLookupResponse.createReceiverNotReady();
+			}
+			catch (JobException e) {
+				fail(new Exception("Cannot schedule the receivers, not enough resources."));
+				return ConnectionInfoLookupResponse.createJobIsAborting();
+			}
+		}
+		else if (executionState == ExecutionState.CANCELED || executionState == ExecutionState.CANCELING ||
+				executionState == ExecutionState.FAILED)
+		{
+			return ConnectionInfoLookupResponse.createJobIsAborting();
+		}
+		else {
+			// illegal state for all other states - or all the other state, since the only remaining state is FINISHED
+			// state when the receiver is already running
+			String message = "Channel lookup (forward) - receiver " + targetVertex + " found in inconsistent state " + executionState;
+			LOG.error(message);
+			fail(new Exception(message));
+			return ConnectionInfoLookupResponse.createReceiverNotFound();
+		}
+	}
+	
+	public Map<ExecutionAttemptID, Execution> getRegisteredExecutions() {
+		return Collections.unmodifiableMap(currentExecutions);
 	}
 	
 	void registerExecution(Execution exec) {
@@ -521,18 +554,15 @@ public class ExecutionGraph {
 	
 	void deregisterExecution(Execution exec) {
 		Execution contained = currentExecutions.remove(exec.getAttemptId());
+		
 		if (contained != null && contained != exec) {
 			fail(new Exception("De-registering execution " + exec + " failed. Found for same ID execution " + contained));
 		}
 	}
 	
-	Map<ExecutionAttemptID, Execution> getRegisteredExecutions() {
-		return Collections.unmodifiableMap(currentExecutions);
-	}
-	
 	void registerExecutionEdge(ExecutionEdge edge) {
-		ChannelID source = edge.getInputChannelId();
 		ChannelID target = edge.getInputChannelId();
+		ChannelID source = edge.getOutputChannelId();
 		edges.put(source, edge);
 		edges.put(target, edge);
 	}
@@ -581,6 +611,7 @@ public class ExecutionGraph {
 	 */
 	void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionAttemptID executionId, ExecutionState newExecutionState, Throwable error) {
 		
+		// we must be very careful here with exceptions 
 		if (this.executionListeners.size() > 0) {
 			
 			String message = error == null ? null : ExceptionUtils.stringifyException(error);
@@ -592,11 +623,11 @@ public class ExecutionGraph {
 					LOG.error("Notification of execution state change caused an error.", t);
 				}
 			}
-			
-			// see what this means for us. currently, the first FAILED state means -> FAILED
-			if (newExecutionState == ExecutionState.FAILED) {
-				fail(error);
-			}
+		}
+		
+		// see what this means for us. currently, the first FAILED state means -> FAILED
+		if (newExecutionState == ExecutionState.FAILED) {
+			fail(error);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
index 440566d..1884ce0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -32,7 +32,7 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 
@@ -225,7 +225,7 @@ public class ExecutionJobVertex {
 	//  Actions
 	//---------------------------------------------------------------------------------------------
 	
-	public void scheduleAll(DefaultScheduler scheduler, boolean queued) throws NoResourceAvailableException {
+	public void scheduleAll(Scheduler scheduler, boolean queued) throws NoResourceAvailableException {
 		for (ExecutionVertex ev : getTaskVertices()) {
 			ev.scheduleForExecution(scheduler, queued);
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index bbc0c97..3c65f2e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -28,7 +28,6 @@ import java.util.List;
 import java.util.concurrent.CopyOnWriteArrayList;
 
 import org.slf4j.Logger;
-
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
@@ -39,7 +38,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 
 /**
@@ -305,7 +304,7 @@ public class ExecutionVertex {
 		}
 	}
 	
-	public void scheduleForExecution(DefaultScheduler scheduler, boolean queued) throws NoResourceAvailableException {
+	public void scheduleForExecution(Scheduler scheduler, boolean queued) throws NoResourceAvailableException {
 		this.currentExecution.scheduleForExecution(scheduler, queued);
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
index 0b822ab..f770b87 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
@@ -34,6 +34,8 @@ public class IntermediateResult {
 	
 	private int numConsumers;
 	
+	private final int connectionIndex;
+	
 	
 	public IntermediateResult(IntermediateDataSetID id, ExecutionJobVertex producer, int numParallelProducers) {
 		this.id = id;
@@ -43,6 +45,9 @@ public class IntermediateResult {
 		
 		// we do not set the intermediate result partitions here, because we let them be initialized by
 		// the execution vertex that produces them
+		
+		// assign a random connection index
+		this.connectionIndex = (int) (Math.random() * Integer.MAX_VALUE);
 	}
 	
 	public void setPartition(int partitionNumber, IntermediateResultPartition partition) {
@@ -87,4 +92,8 @@ public class IntermediateResult {
 		}
 		return index;
 	}
+	
+	public int getConnectionIndex() {
+		return connectionIndex;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
index 1c4e1fb..1cc5e13 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
@@ -39,6 +39,7 @@ public class IntermediateResultPartition {
 		this.consumers = new ArrayList<List<ExecutionEdge>>(0);
 	}
 	
+	
 	public ExecutionVertex getProducer() {
 		return producer;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java
index 42b9817..69e41d2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java
@@ -1,17 +1,20 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.instance;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
index 3066bb5..602f88b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
@@ -366,7 +366,7 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 			}
 			else if (lookupResponse.receiverNotReady()) {
 				try {
-					Thread.sleep(500);
+					Thread.sleep(100);
 				} catch (InterruptedException e) {
 					if (reportException) {
 						throw new IOException("Lookup was interrupted.");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index f8b5ab9..85978fe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -71,6 +71,8 @@ public class JobGraph implements IOReadableWritable {
 	/** Name of this job. */
 	private String jobName;
 	
+	private boolean allowQueuedScheduling;
+	
 	// --------------------------------------------------------------------------------------------
 	
 	/**
@@ -162,6 +164,14 @@ public class JobGraph implements IOReadableWritable {
 	public Configuration getJobConfiguration() {
 		return this.jobConfiguration;
 	}
+	
+	public void setAllowQueuedScheduling(boolean allowQueuedScheduling) {
+		this.allowQueuedScheduling = allowQueuedScheduling;
+	}
+	
+	public boolean getAllowQueuedScheduling() {
+		return allowQueuedScheduling;
+	}
 
 	/**
 	 * Adds a new task vertex to the job graph if it is not already included.
@@ -304,6 +314,7 @@ public class JobGraph implements IOReadableWritable {
 		this.jobID.read(in);
 		this.jobName = StringValue.readString(in);
 		this.jobConfiguration.read(in);
+		this.allowQueuedScheduling = in.readBoolean();
 		
 		final int numVertices = in.readInt();
 		
@@ -332,6 +343,7 @@ public class JobGraph implements IOReadableWritable {
 		this.jobID.write(out);
 		StringValue.writeString(this.jobName, out);
 		this.jobConfiguration.write(out);
+		out.writeBoolean(allowQueuedScheduling);
 		
 		// write the task vertices using java serialization (to resolve references in the object graph)
 		out.writeInt(taskVertices.size());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
index 60b2edf..f5a2e9c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobStatus.java
@@ -24,20 +24,35 @@ package org.apache.flink.runtime.jobgraph;
 public enum JobStatus {
 
 	/** Job is newly created, no task has started to run. */
-	CREATED,
+	CREATED(false),
 
 	/** Some tasks are scheduled or running, some may be pending, some may be finished. */
-	RUNNING,
+	RUNNING(false),
 
+	/** The job has failed and is currently waiting for the cleanup to complete */
+	FAILING(false),
+	
 	/** The job has failed to to non-recoverable task failure */
-	FAILED,
+	FAILED(true),
 
 	/** Job is being cancelled */
-	CANCELLING,
+	CANCELLING(false),
 	
 	/** Job has been cancelled */
-	CANCELED,
+	CANCELED(true),
 
 	/** All of the job's tasks have successfully finished. */
-	FINISHED
+	FINISHED(true);
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private final boolean terminalState;
+	
+	private JobStatus(boolean terminalState) {
+		this.terminalState = terminalState;
+	}
+	
+	public boolean isTerminalState() {
+		return terminalState;
+	}
 };

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
index a45507a..551dce2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
@@ -26,6 +26,8 @@ import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.flink.runtime.event.job.AbstractEvent;
 import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent;
 import org.apache.flink.runtime.event.job.JobEvent;
@@ -52,6 +54,8 @@ import org.apache.flink.runtime.profiling.types.ProfilingEvent;
  */
 public final class EventCollector extends TimerTask implements ProfilingListener {
 
+	private static final Log LOG = LogFactory.getLog(EventCollector.class);
+
 	/**
 	 * The execution listener wrapper is an auxiliary class. It is required
 	 * because the job vertex ID and the management vertex ID cannot be accessed from
@@ -93,6 +97,8 @@ public final class EventCollector extends TimerTask implements ProfilingListener
 					executionId, newExecutionState);
 
 			this.eventCollector.addEvent(jobID, executionStateChangeEvent);
+			
+			LOG.info(vertexEvent.toString());
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index d3a920c..3526e15 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -79,7 +79,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager;
 import org.apache.flink.runtime.jobmanager.archive.ArchiveListener;
 import org.apache.flink.runtime.jobmanager.archive.MemoryArchivist;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.web.WebInfoServer;
 import org.apache.flink.runtime.protocols.AccumulatorProtocol;
 import org.apache.flink.runtime.protocols.ChannelLookupProtocol;
@@ -122,7 +122,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 	private final InstanceManager instanceManager;
 	
 	/** Assigns tasks to slots and keeps track on available and allocated task slots*/
-	private final DefaultScheduler scheduler;
+	private final Scheduler scheduler;
 	
 	/** The currently running jobs */
 	private final ConcurrentHashMap<JobID, ExecutionGraph> currentJobs;
@@ -143,7 +143,6 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 	
 	private volatile boolean isShutDown;
 	
-	
 	private WebInfoServer server;
 	
 	
@@ -222,7 +221,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 		}
 
 		// create the scheduler and make it listen at the availability of new instances
-		this.scheduler = new DefaultScheduler();
+		this.scheduler = new Scheduler(this.executorService);
 		this.instanceManager.addInstanceListener(this.scheduler);
 	}
 
@@ -349,6 +348,9 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 			if (LOG.isDebugEnabled()) {
 				LOG.debug(String.format("Successfully created execution graph from job graph %s (%s)", job.getJobID(), job.getName()));
 			}
+			
+			// should the job fail if a vertex cannot be deployed immediately (streams, closed iterations)
+			executionGraph.setQueuedSchedulingAllowed(job.getAllowQueuedScheduling());
 	
 			// Register job with the progress collector
 			if (this.eventCollector != null) {
@@ -461,12 +463,12 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 		
 		if (LOG.isInfoEnabled()) {
 			String message = optionalMessage == null ? "." : ": " + optionalMessage;
-			LOG.info(String.format("Status of job %s (%s) changed to %s%s", 
+			LOG.info(String.format("Job %s (%s) switched to %s%s", 
 					jid, executionGraph.getJobName(), newJobStatus, message));
 		}
 
 		// remove the job graph if the state is any terminal state
-		if (newJobStatus == JobStatus.FINISHED || newJobStatus == JobStatus.CANCELED || newJobStatus == JobStatus.FAILED) {
+		if (newJobStatus.isTerminalState()) {
 			this.currentJobs.remove(jid);
 			
 			try {
@@ -643,7 +645,11 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 
 	@Override
 	public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) {
-		return this.instanceManager.registerTaskManager(instanceConnectionInfo, hardwareDescription, numberOfSlots);
+		if (this.instanceManager != null) {
+			return this.instanceManager.registerTaskManager(instanceConnectionInfo, hardwareDescription, numberOfSlots);
+		} else {
+			return null;
+		}
 	}
 	
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
deleted file mode 100644
index 54e16b9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
+++ /dev/null
@@ -1,413 +0,0 @@
-/**
- * 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.jobmanager.scheduler;
-
-import java.util.ArrayDeque;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Queue;
-import java.util.Set;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.instance.AllocatedSlot;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.InstanceDiedException;
-import org.apache.flink.runtime.instance.InstanceListener;
-
-/**
- * The scheduler is responsible for distributing the ready-to-run tasks and assigning them to instances and
- * slots.
- */
-public class DefaultScheduler implements InstanceListener, SlotAvailablilityListener {
-
-	private static final Logger LOG = LoggerFactory.getLogger(DefaultScheduler.class);
-	
-	
-	private final Object globalLock = new Object();
-	
-	
-	/** All instances that the scheduler can deploy to */
-	private final Set<Instance> allInstances = new HashSet<Instance>();
-	
-	/** All instances that still have available resources */
-	private final Queue<Instance> instancesWithAvailableResources = new SetQueue<Instance>();
-	
-	/** All tasks pending to be scheduled */
-	private final Queue<QueuedTask> taskQueue = new ArrayDeque<QueuedTask>();
-	
-	
-	private int unconstrainedAssignments = 0;
-	
-	private int localizedAssignments = 0;
-	
-	private int nonLocalizedAssignments = 0;
-	
-	
-	public DefaultScheduler() {
-	}
-	
-	/**
-	 * Shuts the scheduler down. After shut down no more tasks can be added to the scheduler.
-	 */
-	public void shutdown() {
-		synchronized (globalLock) {
-			for (Instance i : allInstances) {
-				i.removeSlotListener();
-				i.cancelAndReleaseAllSlots();
-			}
-			allInstances.clear();
-			instancesWithAvailableResources.clear();
-			taskQueue.clear();
-		}
-	}
-
-	/**
-	 * 
-	 * NOTE: In the presence of multi-threaded operations, this number may be inexact.
-	 * 
-	 * @return The number of empty slots, for tasks.
-	 */
-	public int getNumberOfAvailableSlots() {
-		int count = 0;
-		
-		synchronized (globalLock) {
-			for (Instance instance : instancesWithAvailableResources) {
-				count += instance.getNumberOfAvailableSlots();
-			}
-		}
-		
-		return count;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  Scheduling
-	// --------------------------------------------------------------------------------------------
-	
-	public AllocatedSlot scheduleImmediately(ScheduledUnit task) throws NoResourceAvailableException {
-		Object ret = scheduleTask(task, false);
-		if (ret instanceof AllocatedSlot) {
-			return (AllocatedSlot) ret;
-		}
-		else {
-			throw new RuntimeException();
-		}
-	}
-	
-	public SlotAllocationFuture scheduleQueued(ScheduledUnit task) throws NoResourceAvailableException {
-		Object ret = scheduleTask(task, true);
-		if (ret instanceof AllocatedSlot) {
-			return new SlotAllocationFuture((AllocatedSlot) ret);
-		}
-		if (ret instanceof SlotAllocationFuture) {
-			return (SlotAllocationFuture) ret;
-		}
-		else {
-			throw new RuntimeException();
-		}
-	}
-	
-	/**
-	 * Returns either an {@link AllocatedSlot}, or an {@link SlotAllocationFuture}.
-	 */
-	private Object scheduleTask(ScheduledUnit task, boolean queueIfNoResource) throws NoResourceAvailableException {
-		if (task == null) {
-			throw new IllegalArgumentException();
-		}
-		
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Scheduling task " + task);
-		}
-		
-		final ExecutionVertex vertex = task.getTaskToExecute().getVertex();
-	
-		synchronized (globalLock) {
-			// 1) If the task has a strict co-schedule hint, obey it, if it has been assigned.
-//			CoLocationHint hint = task.getCoScheduleHint();
-//			if (hint != null) {
-//				
-//				// try to add to the slot, or make it wait on the hint and schedule the hint itself
-//				if () {
-//					return slot;
-//				}
-//			}
-		
-			// 2) See if we can place the task somewhere together with another existing task.
-			//    This is defined by the slot sharing groups
-			SlotSharingGroup sharingUnit = task.getSlotSharingGroup();
-			if (sharingUnit != null) {
-				// see if we can add the task to the current sharing group.
-				SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
-				AllocatedSlot slot = assignment.getSlotForTask(vertex.getJobvertexId(), vertex);
-				if (slot != null) {
-					return slot;
-				}
-			}
-		
-			// 3) We could not schedule it to an existing slot, so we need to get a new one or queue the task
-			
-			// we need potentially to loop multiple times, because there may be false positives
-			// in the set-with-available-instances
-			while (true) {
-				
-				
-				Instance instanceToUse = getFreeInstanceForTask(task.getTaskToExecute().getVertex());
-			
-				if (instanceToUse != null) {
-					try {
-						AllocatedSlot slot = instanceToUse.allocateSlot(vertex.getJobId());
-						
-						// if the instance has further available slots, re-add it to the set of available resources.
-						if (instanceToUse.hasResourcesAvailable()) {
-							this.instancesWithAvailableResources.add(instanceToUse);
-						}
-						
-						if (slot != null) {
-							
-							// if the task is in a shared group, assign the slot to that group
-							// and get a sub slot in turn
-							if (sharingUnit != null) {
-								slot = sharingUnit.getTaskAssignment().addSlotWithTask(slot, task.getJobVertexId());
-							}
-							
-							return slot;
-						}
-					}
-					catch (InstanceDiedException e) {
-						// the instance died it has not yet been propagated to this scheduler
-						// remove the instance from the set of available instances
-						this.allInstances.remove(instanceToUse);
-						this.instancesWithAvailableResources.remove(instanceToUse);
-					}
-				}
-				else {
-					// no resource available now, so queue the request
-					if (queueIfNoResource) {
-						SlotAllocationFuture future = new SlotAllocationFuture();
-						this.taskQueue.add(new QueuedTask(task, future));
-						return future;
-					}
-					else {
-						throw new NoResourceAvailableException(task);
-					}
-				}
-			}
-		}
-	}
-		
-	/**
-	 * Gets a suitable instance to schedule the vertex execution to.
-	 * <p>
-	 * NOTE: This method does is not thread-safe, it needs to be synchronized by the caller.
-	 * 
-	 * @param vertex The task to run. 
-	 * @return The instance to run the vertex on, it {@code null}, if no instance is available.
-	 */
-	protected Instance getFreeInstanceForTask(ExecutionVertex vertex) {
-		if (this.instancesWithAvailableResources.isEmpty()) {
-			return null;
-		}
-		
-		Iterable<Instance> locationsIterable = vertex.getPreferredLocations();
-		Iterator<Instance> locations = locationsIterable == null ? null : locationsIterable.iterator();
-		
-		if (locations != null && locations.hasNext()) {
-			
-			while (locations.hasNext()) {
-				Instance location = locations.next();
-				
-				if (location != null && this.instancesWithAvailableResources.remove(location)) {
-					
-					localizedAssignments++;
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Local assignment: " + vertex.getSimpleName() + " --> " + location);
-					}
-					
-					return location;
-				}
-			}
-			
-			Instance instance = this.instancesWithAvailableResources.poll();
-			nonLocalizedAssignments++;
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Non-local assignment: " + vertex.getSimpleName() + " --> " + instance);
-			}
-			return instance;
-		}
-		else {
-			Instance instance = this.instancesWithAvailableResources.poll();
-			unconstrainedAssignments++;
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Unconstrained assignment: " + vertex.getSimpleName() + " --> " + instance);
-			}
-			
-			return instance;
-		}
-	}
-	
-	@Override
-	public void newSlotAvailable(Instance instance) {
-		
-		// global lock before instance lock, so that the order of acquiring locks is always 1) global, 2) instance
-		synchronized (globalLock) {
-			QueuedTask queued = taskQueue.peek();
-			
-			// the slot was properly released, we can allocate a new one from that instance
-			
-			if (queued != null) {
-				ScheduledUnit task = queued.getTask();
-				ExecutionVertex vertex = task.getTaskToExecute().getVertex();
-				
-				try {
-					AllocatedSlot newSlot = instance.allocateSlot(vertex.getJobId());
-					if (newSlot != null) {
-						
-						// success, remove from the task queue and notify the future
-						taskQueue.poll();
-						if (queued.getFuture() != null) {
-							try {
-								queued.getFuture().setSlot(newSlot);
-							}
-							catch (Throwable t) {
-								LOG.error("Error calling allocation future for task " + vertex.getSimpleName(), t);
-								task.getTaskToExecute().fail(t);
-							}
-						}
-					}
-				}
-				catch (InstanceDiedException e) {
-					this.allInstances.remove(instance);
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Instance " + instance + " was marked dead asynchronously.");
-					}
-				}
-			}
-			else {
-				this.instancesWithAvailableResources.add(instance);
-			}
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//  Instance Availability
-	// --------------------------------------------------------------------------------------------
-	
-	@Override
-	public void newInstanceAvailable(Instance instance) {
-		if (instance == null) {
-			throw new IllegalArgumentException();
-		}
-		if (instance.getNumberOfAvailableSlots() <= 0) {
-			throw new IllegalArgumentException("The given instance has no resources.");
-		}
-		if (!instance.isAlive()) {
-			throw new IllegalArgumentException("The instance is not alive.");
-		}
-		
-		// synchronize globally for instance changes
-		synchronized (this.globalLock) {
-			
-			// check we do not already use this instance
-			if (!this.allInstances.add(instance)) {
-				throw new IllegalArgumentException("The instance is already contained.");
-			}
-			
-			try {
-				instance.setSlotAvailabilityListener(this);
-			}
-			catch (IllegalStateException e) {
-				this.allInstances.remove(instance);
-				LOG.error("Scheduler could not attach to the instance as a listener.");
-			}
-			
-			
-			// add it to the available resources and let potential waiters know
-			this.instancesWithAvailableResources.add(instance);
-			
-			for (int i = 0; i < instance.getNumberOfAvailableSlots(); i++) {
-				newSlotAvailable(instance);
-			}
-		}
-	}
-	
-	@Override
-	public void instanceDied(Instance instance) {
-		if (instance == null) {
-			throw new IllegalArgumentException();
-		}
-		
-		instance.markDead();
-		
-		// we only remove the instance from the pools, we do not care about the 
-		synchronized (this.globalLock) {
-			// the instance must not be available anywhere any more
-			this.allInstances.remove(instance);
-			this.instancesWithAvailableResources.remove(instance);
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//  Status reporting
-	// --------------------------------------------------------------------------------------------
-
-	public int getNumberOfAvailableInstances() {
-		return allInstances.size();
-	}
-	
-	public int getNumberOfInstancesWithAvailableSlots() {
-		return instancesWithAvailableResources.size();
-	}
-	
-	public int getNumberOfUnconstrainedAssignments() {
-		return unconstrainedAssignments;
-	}
-	
-	public int getNumberOfLocalizedAssignments() {
-		return localizedAssignments;
-	}
-	
-	public int getNumberOfNonLocalizedAssignments() {
-		return nonLocalizedAssignments;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private static final class QueuedTask {
-		
-		private final ScheduledUnit task;
-		
-		private final SlotAllocationFuture future;
-		
-		
-		public QueuedTask(ScheduledUnit task, SlotAllocationFuture future) {
-			this.task = task;
-			this.future = future;
-		}
-
-		public ScheduledUnit getTask() {
-			return task;
-		}
-
-		public SlotAllocationFuture getFuture() {
-			return future;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
new file mode 100644
index 0000000..cd57454
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -0,0 +1,447 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import java.util.ArrayDeque;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertex;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceDiedException;
+import org.apache.flink.runtime.instance.InstanceListener;
+
+/**
+ * The scheduler is responsible for distributing the ready-to-run tasks and assigning them to instances and
+ * slots.
+ */
+public class Scheduler implements InstanceListener, SlotAvailablilityListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(Scheduler.class);
+	
+	
+	private final Object globalLock = new Object();
+	
+	private final ExecutorService executor;
+	
+	
+	/** All instances that the scheduler can deploy to */
+	private final Set<Instance> allInstances = new HashSet<Instance>();
+	
+	/** All instances that still have available resources */
+	private final Queue<Instance> instancesWithAvailableResources = new SetQueue<Instance>();
+	
+	/** All tasks pending to be scheduled */
+	private final Queue<QueuedTask> taskQueue = new ArrayDeque<QueuedTask>();
+	
+	
+	private int unconstrainedAssignments = 0;
+	
+	private int localizedAssignments = 0;
+	
+	private int nonLocalizedAssignments = 0;
+	
+	
+	public Scheduler() {
+		this(null);
+	}
+	
+	public Scheduler(ExecutorService executorService) {
+		this.executor = executorService;
+	}
+	
+	
+	/**
+	 * Shuts the scheduler down. After shut down no more tasks can be added to the scheduler.
+	 */
+	public void shutdown() {
+		synchronized (globalLock) {
+			for (Instance i : allInstances) {
+				i.removeSlotListener();
+				i.cancelAndReleaseAllSlots();
+			}
+			allInstances.clear();
+			instancesWithAvailableResources.clear();
+			taskQueue.clear();
+		}
+	}
+
+	/**
+	 * 
+	 * NOTE: In the presence of multi-threaded operations, this number may be inexact.
+	 * 
+	 * @return The number of empty slots, for tasks.
+	 */
+	public int getNumberOfAvailableSlots() {
+		int count = 0;
+		
+		synchronized (globalLock) {
+			for (Instance instance : instancesWithAvailableResources) {
+				count += instance.getNumberOfAvailableSlots();
+			}
+		}
+		
+		return count;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Scheduling
+	// --------------------------------------------------------------------------------------------
+	
+	public AllocatedSlot scheduleImmediately(ScheduledUnit task) throws NoResourceAvailableException {
+		Object ret = scheduleTask(task, false);
+		if (ret instanceof AllocatedSlot) {
+			return (AllocatedSlot) ret;
+		}
+		else {
+			throw new RuntimeException();
+		}
+	}
+	
+	public SlotAllocationFuture scheduleQueued(ScheduledUnit task) throws NoResourceAvailableException {
+		Object ret = scheduleTask(task, true);
+		if (ret instanceof AllocatedSlot) {
+			return new SlotAllocationFuture((AllocatedSlot) ret);
+		}
+		if (ret instanceof SlotAllocationFuture) {
+			return (SlotAllocationFuture) ret;
+		}
+		else {
+			throw new RuntimeException();
+		}
+	}
+	
+	/**
+	 * Returns either an {@link AllocatedSlot}, or an {@link SlotAllocationFuture}.
+	 */
+	private Object scheduleTask(ScheduledUnit task, boolean queueIfNoResource) throws NoResourceAvailableException {
+		if (task == null) {
+			throw new IllegalArgumentException();
+		}
+		
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Scheduling task " + task);
+		}
+		
+		final ExecutionVertex vertex = task.getTaskToExecute().getVertex();
+	
+		synchronized (globalLock) {
+			// 1) If the task has a strict co-schedule hint, obey it, if it has been assigned.
+//			CoLocationHint hint = task.getCoScheduleHint();
+//			if (hint != null) {
+//				
+//				// try to add to the slot, or make it wait on the hint and schedule the hint itself
+//				if () {
+//					return slot;
+//				}
+//			}
+		
+			// 2) See if we can place the task somewhere together with another existing task.
+			//    This is defined by the slot sharing groups
+			SlotSharingGroup sharingUnit = task.getSlotSharingGroup();
+			if (sharingUnit != null) {
+				// see if we can add the task to the current sharing group.
+				SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
+				AllocatedSlot slot = assignment.getSlotForTask(vertex.getJobvertexId(), vertex);
+				if (slot != null) {
+					return slot;
+				}
+			}
+		
+			// 3) We could not schedule it to an existing slot, so we need to get a new one or queue the task
+			
+			// we need potentially to loop multiple times, because there may be false positives
+			// in the set-with-available-instances
+			while (true) {
+				
+				
+				Instance instanceToUse = getFreeInstanceForTask(task.getTaskToExecute().getVertex());
+			
+				if (instanceToUse != null) {
+					try {
+						AllocatedSlot slot = instanceToUse.allocateSlot(vertex.getJobId());
+						
+						// if the instance has further available slots, re-add it to the set of available resources.
+						if (instanceToUse.hasResourcesAvailable()) {
+							this.instancesWithAvailableResources.add(instanceToUse);
+						}
+						
+						if (slot != null) {
+							
+							// if the task is in a shared group, assign the slot to that group
+							// and get a sub slot in turn
+							if (sharingUnit != null) {
+								slot = sharingUnit.getTaskAssignment().addSlotWithTask(slot, task.getJobVertexId());
+							}
+							
+							return slot;
+						}
+					}
+					catch (InstanceDiedException e) {
+						// the instance died it has not yet been propagated to this scheduler
+						// remove the instance from the set of available instances
+						this.allInstances.remove(instanceToUse);
+						this.instancesWithAvailableResources.remove(instanceToUse);
+					}
+				}
+				else {
+					// no resource available now, so queue the request
+					if (queueIfNoResource) {
+						SlotAllocationFuture future = new SlotAllocationFuture();
+						this.taskQueue.add(new QueuedTask(task, future));
+						return future;
+					}
+					else {
+						throw new NoResourceAvailableException(task);
+					}
+				}
+			}
+		}
+	}
+		
+	/**
+	 * Gets a suitable instance to schedule the vertex execution to.
+	 * <p>
+	 * NOTE: This method does is not thread-safe, it needs to be synchronized by the caller.
+	 * 
+	 * @param vertex The task to run. 
+	 * @return The instance to run the vertex on, it {@code null}, if no instance is available.
+	 */
+	protected Instance getFreeInstanceForTask(ExecutionVertex vertex) {
+		if (this.instancesWithAvailableResources.isEmpty()) {
+			return null;
+		}
+		
+		Iterable<Instance> locationsIterable = vertex.getPreferredLocations();
+		Iterator<Instance> locations = locationsIterable == null ? null : locationsIterable.iterator();
+		
+		if (locations != null && locations.hasNext()) {
+			
+			while (locations.hasNext()) {
+				Instance location = locations.next();
+				
+				if (location != null && this.instancesWithAvailableResources.remove(location)) {
+					
+					localizedAssignments++;
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Local assignment: " + vertex.getSimpleName() + " --> " + location);
+					}
+					
+					return location;
+				}
+			}
+			
+			Instance instance = this.instancesWithAvailableResources.poll();
+			nonLocalizedAssignments++;
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Non-local assignment: " + vertex.getSimpleName() + " --> " + instance);
+			}
+			return instance;
+		}
+		else {
+			Instance instance = this.instancesWithAvailableResources.poll();
+			unconstrainedAssignments++;
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Unconstrained assignment: " + vertex.getSimpleName() + " --> " + instance);
+			}
+			
+			return instance;
+		}
+	}
+	
+	@Override
+	public void newSlotAvailable(final Instance instance) {
+		
+		// WARNING: The asynchrony here is necessary, because  we cannot guarantee the order
+		// of lock acquisition (global scheduler, instance) and otherwise lead to potential deadlocks:
+		// 
+		// -> The scheduler needs to grab them (1) global scheduler lock
+		//                                     (2) slot/instance lock
+		// -> The slot releasing grabs (1) slot/instance (for releasing) and
+		//                             (2) scheduler (to check whether to take a new task item
+		// 
+		// that leads with a high probability to deadlocks, when scheduling fast
+		
+		if (this.executor != null) {
+			this.executor.execute(new Runnable() {
+				@Override
+				public void run() {
+					handleNewSlot(instance);
+				}
+			});
+		}
+		else {
+			// for tests, we use the synchronous variant
+			handleNewSlot(instance);
+		}
+	}
+	
+	private void handleNewSlot(Instance instance) {
+		
+		synchronized (globalLock) {
+			QueuedTask queued = taskQueue.peek();
+			
+			// the slot was properly released, we can allocate a new one from that instance
+			
+			if (queued != null) {
+				ScheduledUnit task = queued.getTask();
+				ExecutionVertex vertex = task.getTaskToExecute().getVertex();
+				
+				try {
+					AllocatedSlot newSlot = instance.allocateSlot(vertex.getJobId());
+					if (newSlot != null) {
+						
+						// success, remove from the task queue and notify the future
+						taskQueue.poll();
+						if (queued.getFuture() != null) {
+							try {
+								queued.getFuture().setSlot(newSlot);
+							}
+							catch (Throwable t) {
+								LOG.error("Error calling allocation future for task " + vertex.getSimpleName(), t);
+								task.getTaskToExecute().fail(t);
+							}
+						}
+					}
+				}
+				catch (InstanceDiedException e) {
+					this.allInstances.remove(instance);
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Instance " + instance + " was marked dead asynchronously.");
+					}
+				}
+			}
+			else {
+				this.instancesWithAvailableResources.add(instance);
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Instance Availability
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void newInstanceAvailable(Instance instance) {
+		if (instance == null) {
+			throw new IllegalArgumentException();
+		}
+		if (instance.getNumberOfAvailableSlots() <= 0) {
+			throw new IllegalArgumentException("The given instance has no resources.");
+		}
+		if (!instance.isAlive()) {
+			throw new IllegalArgumentException("The instance is not alive.");
+		}
+		
+		// synchronize globally for instance changes
+		synchronized (this.globalLock) {
+			
+			// check we do not already use this instance
+			if (!this.allInstances.add(instance)) {
+				throw new IllegalArgumentException("The instance is already contained.");
+			}
+			
+			try {
+				instance.setSlotAvailabilityListener(this);
+			}
+			catch (IllegalStateException e) {
+				this.allInstances.remove(instance);
+				LOG.error("Scheduler could not attach to the instance as a listener.");
+			}
+			
+			
+			// add it to the available resources and let potential waiters know
+			this.instancesWithAvailableResources.add(instance);
+			
+			for (int i = 0; i < instance.getNumberOfAvailableSlots(); i++) {
+				newSlotAvailable(instance);
+			}
+		}
+	}
+	
+	@Override
+	public void instanceDied(Instance instance) {
+		if (instance == null) {
+			throw new IllegalArgumentException();
+		}
+		
+		instance.markDead();
+		
+		// we only remove the instance from the pools, we do not care about the 
+		synchronized (this.globalLock) {
+			// the instance must not be available anywhere any more
+			this.allInstances.remove(instance);
+			this.instancesWithAvailableResources.remove(instance);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Status reporting
+	// --------------------------------------------------------------------------------------------
+
+	public int getNumberOfAvailableInstances() {
+		return allInstances.size();
+	}
+	
+	public int getNumberOfInstancesWithAvailableSlots() {
+		return instancesWithAvailableResources.size();
+	}
+	
+	public int getNumberOfUnconstrainedAssignments() {
+		return unconstrainedAssignments;
+	}
+	
+	public int getNumberOfLocalizedAssignments() {
+		return localizedAssignments;
+	}
+	
+	public int getNumberOfNonLocalizedAssignments() {
+		return nonLocalizedAssignments;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static final class QueuedTask {
+		
+		private final ScheduledUnit task;
+		
+		private final SlotAllocationFuture future;
+		
+		
+		public QueuedTask(ScheduledUnit task, SlotAllocationFuture future) {
+			this.task = task;
+			this.future = future;
+		}
+
+		public ScheduledUnit getTask() {
+			return task;
+		}
+
+		public SlotAllocationFuture getFuture() {
+			return future;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 7b692e3..6ac7c54 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -185,11 +185,15 @@ public final class Task {
 		while (true) {
 			ExecutionState current = this.executionState;
 			
-			if (current == ExecutionState.CANCELED || current == ExecutionState.CANCELING) {
+			// if canceled, fine. we are done, and the jobmanager has been told
+			if (current == ExecutionState.CANCELED) {
 				return;
 			}
 			
-			if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) {
+			// if canceling, we are done, but we cannot be sure that the jobmanager has been told.
+			// after all, we may have recognized our failure state before the cancelling and never sent a canceled
+			// message back
+			else if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.FAILED)) {
 				notifyObservers(ExecutionState.FAILED, ExceptionUtils.stringifyException(error));
 				taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.FAILED, error);
 				return;
@@ -201,16 +205,18 @@ public final class Task {
 		while (true) {
 			ExecutionState current = this.executionState;
 			
-			// if the task is already canceled (or canceling) or finished, then we
-			// need not do anything
+			// if the task is already canceled (or canceling) or finished or failed,
+			// then we need not do anything
 			if (current == ExecutionState.FINISHED || current == ExecutionState.CANCELED ||
-					current == ExecutionState.CANCELING) {
+					current == ExecutionState.CANCELING || current == ExecutionState.FAILED)
+			{
 				return;
 			}
 			
 			if (current == ExecutionState.DEPLOYING) {
 				// directly set to canceled
 				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.CANCELED)) {
+					
 					notifyObservers(ExecutionState.CANCELED, null);
 					taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.CANCELED, null);
 					return;
@@ -219,6 +225,7 @@ public final class Task {
 			else if (current == ExecutionState.RUNNING) {
 				// go to canceling and perform the actual task canceling
 				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.CANCELING)) {
+					
 					notifyObservers(ExecutionState.CANCELING, null);
 					try {
 						this.environment.cancelExecution();
@@ -236,9 +243,21 @@ public final class Task {
 	}
 	
 	public void cancelingDone() {
-		if (STATE_UPDATER.compareAndSet(this, ExecutionState.CANCELING, ExecutionState.CANCELED)) {
-			notifyObservers(ExecutionState.CANCELED, null);
-			taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.CANCELED, null);
+		while (true) {
+			ExecutionState current = this.executionState;
+			
+			if (current == ExecutionState.CANCELED) {
+				return;
+			}
+			if (!(current == ExecutionState.RUNNING || current == ExecutionState.CANCELING)) {
+				LOG.error(String.format("Unexpected state transition in Task: %s -> %s", current, ExecutionState.CANCELED));
+			}
+			
+			if (STATE_UPDATER.compareAndSet(this, current, ExecutionState.CANCELED)) {
+				notifyObservers(ExecutionState.CANCELED, null);
+				taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState.CANCELED, null);
+				return;
+			}
 		}
 	}
 


[28/63] [abbrv] git commit: Adapt RPC to support primitive types as parameters and return values.

Posted by se...@apache.org.
Adapt RPC to support primitive types as parameters and return values.


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/aa7550aa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/aa7550aa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/aa7550aa

Branch: refs/heads/master
Commit: aa7550aa2d05940eed695cfa7a1790197255e47d
Parents: 2d6199f
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Jul 21 19:09:26 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../flink/types/JavaToValueConverter.java       |  98 +++
 .../java/org/apache/flink/util/ClassUtils.java  |  32 +
 .../flink/types/JavaToValueConverterTest.java   |  96 +++
 .../flink/runtime/instance/AllocatedSlot.java   |  15 +-
 .../org/apache/flink/runtime/ipc/Client.java    |  14 +-
 .../java/org/apache/flink/runtime/ipc/RPC.java  | 166 ++--
 .../org/apache/flink/runtime/ipc/Server.java    |  17 +-
 .../jobmanager/scheduler/DefaultScheduler.java  | 800 +++----------------
 .../jobmanager/scheduler/ScheduledUnit.java     |   4 +
 .../instance/LocalInstanceManagerTest.java      |  10 +
 .../org/apache/flink/runtime/ipc/RpcTest.java   | 130 +++
 11 files changed, 580 insertions(+), 802 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-core/src/main/java/org/apache/flink/types/JavaToValueConverter.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/types/JavaToValueConverter.java b/flink-core/src/main/java/org/apache/flink/types/JavaToValueConverter.java
new file mode 100644
index 0000000..1e07e3e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/types/JavaToValueConverter.java
@@ -0,0 +1,98 @@
+/**
+ * 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.types;
+
+public class JavaToValueConverter {
+
+	public static Value convertBoxedJavaType(Object boxed) {
+		if (boxed == null) {
+			return null;
+		}
+		
+		final Class<?> clazz = boxed.getClass();
+		
+		if (clazz == String.class) {
+			return new StringValue((String) boxed);
+		}
+		else if (clazz == Integer.class) {
+			return new IntValue((Integer) boxed);
+		}
+		else if (clazz == Long.class) {
+			return new LongValue((Long) boxed);
+		}
+		else if (clazz == Double.class) {
+			return new DoubleValue((Double) boxed);
+		}
+		else if (clazz == Float.class) {
+			return new FloatValue((Float) boxed);
+		}
+		else if (clazz == Boolean.class) {
+			return new BooleanValue((Boolean) boxed);
+		}
+		else if (clazz == Byte.class) {
+			return new ByteValue((Byte) boxed);
+		}
+		else if (clazz == Short.class) {
+			return new ShortValue((Short) boxed);
+		}
+		else if (clazz == Character.class) {
+			return new CharValue((Character) boxed);
+		}
+		else {
+			throw new IllegalArgumentException("Object is no primitive Java type.");
+		}
+	}
+	
+	public static Object convertValueType(Value value) {
+		if (value == null) {
+			return null;
+		}
+		
+		if (value instanceof StringValue) {
+			return ((StringValue) value).getValue();
+		}
+		else if (value instanceof IntValue) {
+			return ((IntValue) value).getValue();
+		}
+		else if (value instanceof  LongValue) {
+			return ((LongValue) value).getValue();
+		}
+		else if (value instanceof DoubleValue) {
+			return ((DoubleValue) value).getValue();
+		}
+		else if (value instanceof FloatValue) {
+			return ((FloatValue) value).getValue();
+		}
+		else if (value instanceof BooleanValue) {
+			return ((BooleanValue) value).getValue();
+		}
+		else if (value instanceof ByteValue) {
+			return ((ByteValue) value).getValue();
+		}
+		else if (value instanceof ShortValue) {
+			return ((ShortValue) value).getValue();
+		}
+		else if (value instanceof CharValue) {
+			return ((CharValue) value).getValue();
+		}
+		else {
+			throw new IllegalArgumentException("Object is no primitive Java type.");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java b/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
index 03a3a23..00ad8e5 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
@@ -19,6 +19,9 @@
 
 package org.apache.flink.util;
 
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.protocols.VersionedProtocol;
@@ -93,4 +96,33 @@ public final class ClassUtils {
 	private static ClassLoader getClassLoader() {
 		return ClassUtils.class.getClassLoader();
 	}
+	
+	public static Class<?> resolveClassPrimitiveAware(String className) throws ClassNotFoundException {
+		if (className == null) {
+			throw new NullPointerException();
+		}
+		
+		Class<?> primClass = PRIMITIVE_TYPES.get(className);
+		if (primClass != null) {
+			return primClass;
+		} else {
+			return Class.forName(className);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static final Map<String, Class<?>> PRIMITIVE_TYPES = new HashMap<String, Class<?>>(9);
+	
+	static {
+		PRIMITIVE_TYPES.put("byte", byte.class);
+		PRIMITIVE_TYPES.put("short", short.class);
+		PRIMITIVE_TYPES.put("int", int.class);
+		PRIMITIVE_TYPES.put("long", long.class);
+		PRIMITIVE_TYPES.put("float", float.class);
+		PRIMITIVE_TYPES.put("double", double.class);
+		PRIMITIVE_TYPES.put("boolean", boolean.class);
+		PRIMITIVE_TYPES.put("char", char.class);
+		PRIMITIVE_TYPES.put("void", void.class);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-core/src/test/java/org/apache/flink/types/JavaToValueConverterTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/JavaToValueConverterTest.java b/flink-core/src/test/java/org/apache/flink/types/JavaToValueConverterTest.java
new file mode 100644
index 0000000..cef639e
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/types/JavaToValueConverterTest.java
@@ -0,0 +1,96 @@
+/**
+ * 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.types;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.junit.Test;
+
+public class JavaToValueConverterTest {
+
+	@Test
+	public void testJavaToValueConversion() {
+		try {
+			assertNull(JavaToValueConverter.convertBoxedJavaType(null));
+			
+			assertEquals(new StringValue("123Test"), JavaToValueConverter.convertBoxedJavaType("123Test"));
+			assertEquals(new ByteValue((byte) 44), JavaToValueConverter.convertBoxedJavaType((byte) 44));
+			assertEquals(new ShortValue((short) 10000), JavaToValueConverter.convertBoxedJavaType((short) 10000));
+			assertEquals(new IntValue(3567564), JavaToValueConverter.convertBoxedJavaType(3567564));
+			assertEquals(new LongValue(767692734), JavaToValueConverter.convertBoxedJavaType(767692734L));
+			assertEquals(new FloatValue(17.5f), JavaToValueConverter.convertBoxedJavaType(17.5f));
+			assertEquals(new DoubleValue(3.1415926), JavaToValueConverter.convertBoxedJavaType(3.1415926));
+			assertEquals(new BooleanValue(true), JavaToValueConverter.convertBoxedJavaType(true));
+			assertEquals(new CharValue('@'), JavaToValueConverter.convertBoxedJavaType('@'));
+			
+			try {
+				JavaToValueConverter.convertBoxedJavaType(new ArrayList<Object>());
+				fail("Accepted invalid type.");
+			} catch (IllegalArgumentException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testValueToJavaConversion() {
+		try {
+			assertNull(JavaToValueConverter.convertValueType(null));
+			
+			assertEquals("123Test", JavaToValueConverter.convertValueType(new StringValue("123Test")));
+			assertEquals((byte) 44, JavaToValueConverter.convertValueType(new ByteValue((byte) 44)));
+			assertEquals((short) 10000, JavaToValueConverter.convertValueType(new ShortValue((short) 10000)));
+			assertEquals(3567564, JavaToValueConverter.convertValueType(new IntValue(3567564)));
+			assertEquals(767692734L, JavaToValueConverter.convertValueType(new LongValue(767692734)));
+			assertEquals(17.5f, JavaToValueConverter.convertValueType(new FloatValue(17.5f)));
+			assertEquals(3.1415926, JavaToValueConverter.convertValueType(new DoubleValue(3.1415926)));
+			assertEquals(true, JavaToValueConverter.convertValueType(new BooleanValue(true)));
+			assertEquals('@', JavaToValueConverter.convertValueType(new CharValue('@')));
+			
+			try {
+				JavaToValueConverter.convertValueType(new MyValue());
+				fail("Accepted invalid type.");
+			} catch (IllegalArgumentException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private static final class MyValue implements Value {
+		private static final long serialVersionUID = 1L;
+
+		@Override
+		public void write(DataOutputView out) {}
+
+		@Override
+		public void read(DataInputView in) {}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
index 71af9db..6289d45 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
@@ -18,6 +18,8 @@
 
 package org.apache.flink.runtime.instance;
 
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobmanager.scheduler.ResourceId;
@@ -38,6 +40,9 @@ public class AllocatedSlot {
 	
 	/** The number of the slot on which the task is deployed */
 	private final int slotNumber;
+	
+	/** Flag that marks the schedule as active */
+	private final AtomicBoolean active = new AtomicBoolean(true);
 
 
 	public AllocatedSlot(JobID jobID, ResourceId resourceId, Instance instance, int slotNumber) {
@@ -72,8 +77,14 @@ public class AllocatedSlot {
 	
 	// --------------------------------------------------------------------------------------------
 	
-	public void runTask(ExecutionVertex2 vertex) {
-		
+	/**
+	 * @param vertex
+	 * 
+	 * @return True, if the task was scheduled correctly, false if the slot was asynchronously deallocated
+	 *         in the meantime.
+	 */
+	public boolean runTask(ExecutionVertex2 vertex) {
+		return true;
 	}
 	
 	public void cancelResource() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Client.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Client.java b/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Client.java
index 87267fe..3c98e78 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Client.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Client.java
@@ -56,13 +56,7 @@ import java.util.Map.Entry;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 
-/**
- * A client for an IPC service. IPC calls take a single {@link Writable} as a
- * parameter, and return a {@link Writable} as their value. A service runs on
- * a port and is defined by a parameter class and a value class.
- * 
- * @see Server
- */
+
 public class Client {
 
 	public static final Logger LOG = LoggerFactory.getLogger(Client.class);
@@ -633,9 +627,6 @@ public class Client {
 		}
 	}
 
-	/**
-	 * Construct an IPC client whose values are of the given {@link Writable} class.
-	 */
 	public Client(final SocketFactory factory) {
 		this.maxIdleTime = 1000;
 		this.maxRetries = 10;
@@ -646,9 +637,6 @@ public class Client {
 
 	/**
 	 * Construct an IPC client with the default SocketFactory
-	 * 
-	 * @param valueClass
-	 * @param conf
 	 */
 	public Client() {
 		this(NetUtils.getDefaultSocketFactory());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/RPC.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/RPC.java b/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/RPC.java
index 1768687..d127d89 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/RPC.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/RPC.java
@@ -46,47 +46,33 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.protocols.VersionedProtocol;
 import org.apache.flink.runtime.net.NetUtils;
+import org.apache.flink.types.JavaToValueConverter;
+import org.apache.flink.types.Value;
 import org.apache.flink.util.ClassUtils;
 
-/**
- * A simple RPC mechanism.
- * A <i>protocol</i> is a Java interface. All parameters and return types must
- * be one of:
- * <ul>
- * <li>a primitive type, <code>boolean</code>, <code>byte</code>, <code>char</code>, <code>short</code>,
- * <code>int</code>, <code>long</code>, <code>float</code>, <code>double</code>, or <code>void</code>; or</li>
- * <li>a {@link String}; or</li>
- * <li>a {@link Writable}; or</li>
- * <li>an array of the above types</li>
- * </ul>
- * All methods in the protocol should throw only IOException. No field data of
- * the protocol instance is transmitted.
- */
+
 public class RPC {
 
 	private static final Logger LOG = LoggerFactory.getLogger(RPC.class);
 
-	private RPC() {
-	} // no public ctor
+	private RPC() {}
 
 	/** A method invocation, including the method name and its parameters. */
 	private static class Invocation implements IOReadableWritable {
 
 		private String methodName;
 
-		private Class<? extends IOReadableWritable>[] parameterClasses;
+		private Class<?>[] parameterClasses;
 
-		private IOReadableWritable[] parameters;
+		private Object[] parameters;
 
 		@SuppressWarnings("unused")
 		public Invocation() {
 		}
 
-		// TODO: See if type safety can be improved here
-		@SuppressWarnings("unchecked")
-		public Invocation(Method method, IOReadableWritable[] parameters) {
+		public Invocation(Method method, Object[] parameters) {
 			this.methodName = method.getName();
-			this.parameterClasses = (Class<? extends IOReadableWritable>[]) method.getParameterTypes();
+			this.parameterClasses = method.getParameterTypes();
 			this.parameters = parameters;
 		}
 
@@ -96,21 +82,21 @@ public class RPC {
 		}
 
 		/** The parameter classes. */
-		public Class<? extends IOReadableWritable>[] getParameterClasses() {
+		public Class<?>[] getParameterClasses() {
 			return parameterClasses;
 		}
 
 		/** The parameter instances. */
-		public IOReadableWritable[] getParameters() {
+		public Object[] getParameters() {
 			return parameters;
 		}
 
-		// TODO: See if type safety can be improved here
+
 		@SuppressWarnings("unchecked")
 		public void read(DataInputView in) throws IOException {
 
 			this.methodName = StringRecord.readString(in);
-			this.parameters = new IOReadableWritable[in.readInt()];
+			this.parameters = new Object[in.readInt()];
 			this.parameterClasses = new Class[parameters.length];
 
 			for (int i = 0; i < parameters.length; i++) {
@@ -118,27 +104,28 @@ public class RPC {
 				// Read class name for parameter and try to get class to that name
 				final String className = StringRecord.readString(in);
 				try {
-					parameterClasses[i] = ClassUtils.getRecordByName(className);
-				} catch (ClassNotFoundException cnfe) {
-					throw new IOException(cnfe.toString());
+					parameterClasses[i] = ClassUtils.resolveClassPrimitiveAware(className);
+				} 
+				catch (ClassNotFoundException e) {
+					throw new IOException(e);
 				}
 
 				// See if parameter is null
 				if (in.readBoolean()) {
+					IOReadableWritable value;
 					try {
 						final String parameterClassName = StringRecord.readString(in);
-						final Class<? extends IOReadableWritable> parameterClass = ClassUtils
-							.getRecordByName(parameterClassName);
-						parameters[i] = parameterClass.newInstance();
-					} catch (IllegalAccessException iae) {
-						throw new IOException(iae.toString());
-					} catch (InstantiationException ie) {
-						throw new IOException(ie.toString());
-					} catch (ClassNotFoundException cnfe) {
-						throw new IOException(cnfe.toString());
+						final Class<? extends IOReadableWritable> parameterClass =
+								(Class<? extends IOReadableWritable>) ClassUtils.resolveClassPrimitiveAware(parameterClassName);
+						
+						value = parameterClass.newInstance();
+						parameters[i] = value;
+					}
+					catch (Exception e) {
+						throw new IOException(e);
 					}
 					// Object will do everything else on its own
-					parameters[i].read(in);
+					value.read(in);
 				} else {
 					parameters[i] = null;
 				}
@@ -148,6 +135,8 @@ public class RPC {
 		public void write(DataOutputView out) throws IOException {
 			StringRecord.writeString(out, methodName);
 			out.writeInt(parameterClasses.length);
+
+			// at this point, type conversion should have happened
 			for (int i = 0; i < parameterClasses.length; i++) {
 				StringRecord.writeString(out, parameterClasses[i].getName());
 				if (parameters[i] == null) {
@@ -155,7 +144,39 @@ public class RPC {
 				} else {
 					out.writeBoolean(true);
 					StringRecord.writeString(out, parameters[i].getClass().getName());
-					parameters[i].write(out);
+					((IOReadableWritable) parameters[i]).write(out);
+				}
+			}
+		}
+		
+		public void doTypeConversion() throws IOException {
+			try {
+				for (int i = 0; i < parameterClasses.length; i++) {
+					if (!IOReadableWritable.class.isAssignableFrom(parameterClasses[i])) {
+						try {
+							parameters[i] = JavaToValueConverter.convertBoxedJavaType(parameters[i]);
+						}
+						catch (IllegalArgumentException e) {
+							throw new IOException("Argument " + i + " of method " + methodName
+									+ " is not a primitive type (or boxed primitive) and not of type IOReadableWriteable");
+						}
+					}
+				}
+			}
+			catch (IOException e) {
+				LOG.error(e.getMessage(), e);
+				throw e;
+			}
+			catch (Exception e) {
+				LOG.error(e.getMessage(), e);
+				throw new IOException(e);
+			}
+		}
+		
+		public void undoTypeConversion() {
+			for (int i = 0; i < parameterClasses.length; i++) {
+				if (!IOReadableWritable.class.isAssignableFrom(parameterClasses[i])) {
+					parameters[i] = JavaToValueConverter.convertValueType((Value) parameters[i]);
 				}
 			}
 		}
@@ -180,14 +201,6 @@ public class RPC {
 	static private class ClientCache {
 		private Map<SocketFactory, Client> clients = new HashMap<SocketFactory, Client>();
 
-		/**
-		 * Construct & cache an IPC client with the user-provided SocketFactory
-		 * if no cached client exists.
-		 * 
-		 * @param conf
-		 *        Configuration
-		 * @return an IPC client
-		 */
 		private synchronized Client getClient(SocketFactory factory) {
 			// Construct & cache client. The configuration is only used for timeout,
 			// and Clients have connection pools. So we can either (a) lose some
@@ -236,26 +249,17 @@ public class RPC {
 		}
 
 		public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
-
-			// TODO clean up
-			IOReadableWritable[] castArgs = null;
-			if (args != null) {
-				castArgs = new IOReadableWritable[args.length];
-
-				// Check if args are instances of ReadableWritable
-				for (int i = 0; i < args.length; i++) {
-					if ((args[i] != null) && !(args[i] instanceof IOReadableWritable)) {
-						throw new IOException("Argument " + i + " of method " + method.getName()
-							+ " is not of type IOReadableWriteable");
-					} else {
-						castArgs[i] = (IOReadableWritable) args[i];
-					}
-				}
+			Invocation invocation = new Invocation(method, args);
+			invocation.doTypeConversion();
+			
+			Object retValue = this.client.call(invocation, this.address, method.getDeclaringClass());
+			
+			if (IOReadableWritable.class.isAssignableFrom(method.getReturnType())) {
+				return retValue;
+			}
+			else {
+				return JavaToValueConverter.convertValueType((Value) retValue);
 			}
-			final IOReadableWritable value = this.client.call(new Invocation(method, castArgs), this.address, method
-				.getDeclaringClass());
-
-			return value;
 		}
 
 		/* close the IPC client that's responsible for this invoker's RPCs */
@@ -328,11 +332,6 @@ public class RPC {
 
 	/**
 	 * Construct a client-side proxy object with the default SocketFactory
-	 * 
-	 * @param protocol
-	 * @param addr
-	 * @return
-	 * @throws IOException
 	 */
 	public static <V extends VersionedProtocol> V getProxy(Class<V> protocol, InetSocketAddress addr)
 			throws IOException {
@@ -370,8 +369,6 @@ public class RPC {
 		 * 
 		 * @param instance
 		 *        the instance whose methods will be called
-		 * @param conf
-		 *        the configuration to use
 		 * @param bindAddress
 		 *        the address to bind on to listen for connection
 		 * @param port
@@ -395,8 +392,6 @@ public class RPC {
 		 * 
 		 * @param instance
 		 *        the instance whose methods will be called
-		 * @param conf
-		 *        the configuration to use
 		 * @param bindAddress
 		 *        the address to bind on to listen for connection
 		 * @param port
@@ -415,12 +410,26 @@ public class RPC {
 			try {
 				
 				final Invocation call = (Invocation) param;
+				call.undoTypeConversion();
 				
 				final Method method = protocol.getMethod(call.getMethodName(), call.getParameterClasses());
 				method.setAccessible(true);
 
 				final Object value = method.invoke((Object) instance, (Object[]) call.getParameters());
-				return (IOReadableWritable) value;
+
+				if (IOReadableWritable.class.isAssignableFrom(method.getReturnType())) {
+					return (IOReadableWritable) value;
+				}
+				else {
+					try {
+						return JavaToValueConverter.convertBoxedJavaType(value);
+					}
+					catch (IllegalArgumentException e) {
+						throw new IOException("The return type of method " + method.getName()
+								+ " is not a primitive type (or boxed primitive) and not of type IOReadableWriteable");
+					}
+				}
+
 			} catch (InvocationTargetException e) {
 				
 				final Throwable target = e.getTargetException();
@@ -431,7 +440,8 @@ public class RPC {
 					ioe.setStackTrace(target.getStackTrace());
 					throw ioe;
 				}
-			} catch (Throwable e) {
+			}
+			catch (Throwable e) {
 				final IOException ioe = new IOException(e.toString());
 				ioe.setStackTrace(e.getStackTrace());
 				throw ioe;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Server.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Server.java b/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Server.java
index bde6847..3dcb5b1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Server.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/ipc/Server.java
@@ -67,13 +67,7 @@ import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
 import org.apache.flink.core.protocols.VersionedProtocol;
 import org.apache.flink.util.ClassUtils;
 
-/**
- * An abstract IPC service. IPC calls take a single {@link Writable} as a
- * parameter, and return a {@link Writable} as their value. A service runs on
- * a port and is defined by a parameter class and a value class.
- * 
- * @see Client
- */
+
 public abstract class Server {
 
 	public static final Logger LOG = LoggerFactory.getLogger(Server.class);
@@ -101,12 +95,7 @@ public abstract class Server {
 		return protocol;
 	}
 
-	/**
-	 * Returns the server instance called under or null. May be called under {@link #call(Writable, long)}
-	 * implementations, and under {@link Writable} methods of paramters and return values. Permits applications to
-	 * access
-	 * the server context.
-	 */
+
 	public static Server get() {
 		return SERVER.get();
 	}
@@ -119,7 +108,7 @@ public abstract class Server {
 
 	/**
 	 * Returns the remote side ip address when invoked inside an RPC
-	 * Returns null incase of an error.
+	 * Returns null in case of an error.
 	 */
 	public static InetAddress getRemoteIp() {
 		Call call = CurCall.get();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
index d23d35f..e038d7d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
@@ -19,9 +19,8 @@
 package org.apache.flink.runtime.jobmanager.scheduler;
 
 import java.lang.Thread.UncaughtExceptionHandler;
-import java.util.HashMap;
+import java.util.ArrayDeque;
 import java.util.HashSet;
-import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
@@ -37,6 +36,9 @@ import org.apache.flink.runtime.instance.InstanceDiedException;
 import org.apache.flink.runtime.instance.InstanceListener;
 import org.apache.flink.runtime.jobgraph.JobID;
 
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+
 /**
  * The scheduler is responsible for distributing the ready-to-run tasks and assigning them to instances and
  * slots.
@@ -66,9 +68,8 @@ public class DefaultScheduler implements InstanceListener {
 //	 */
 //	private final Cache<ResourceId, Instance> ghostCache;
 	
-	
 	/** All tasks pending to be scheduled */
-	private final LinkedBlockingQueue<ScheduledUnit> taskQueue = new LinkedBlockingQueue<ScheduledUnit>();
+	private final Queue<ScheduledUnit> taskQueue = new ArrayDeque<ScheduledUnit>();
 
 	
 	/** The thread that runs the scheduling loop, picking up tasks to be scheduled and scheduling them. */
@@ -91,7 +92,6 @@ public class DefaultScheduler implements InstanceListener {
 	public DefaultScheduler(boolean rejectIfNoResourceAvailable) {
 		this.rejectIfNoResourceAvailable = rejectIfNoResourceAvailable;
 		
-		
 //		this.ghostCache = CacheBuilder.newBuilder()
 //				.initialCapacity(64)	// easy start
 //				.maximumSize(1024)		// retain some history
@@ -144,639 +144,36 @@ public class DefaultScheduler implements InstanceListener {
 		this.schedulerThread.setUncaughtExceptionHandler(handler);
 	}
 
-//	/**
-//	 * Removes the job represented by the given {@link ExecutionGraph} from the scheduler.
-//	 *
-//	 * @param executionGraphToRemove
-//	 *        the job to be removed
-//	 */
-//	void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) {
-//
-//		boolean removedFromQueue = false;
-//
-//		synchronized (this.jobQueue) {
-//
-//			final Iterator<ExecutionGraph> it = this.jobQueue.iterator();
-//			while (it.hasNext()) {
-//
-//				final ExecutionGraph executionGraph = it.next();
-//				if (executionGraph.getJobID().equals(executionGraphToRemove.getJobID())) {
-//					removedFromQueue = true;
-//					it.remove();
-//					break;
-//				}
-//			}
-//		}
-//
-//		if (!removedFromQueue) {
-//			LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " ("
-//					+ executionGraphToRemove.getJobID() + ") to remove");
-//		}
-//	}
-//
-//	/**
-//	 * Adds a job represented by an {@link ExecutionGraph} object to the scheduler. The job is then executed according
-//	 * to the strategies of the concrete scheduler implementation.
-//	 *
-//	 * @param executionGraph
-//	 *        the job to be added to the scheduler
-//	 * @throws SchedulingException
-//	 *         thrown if an error occurs and the scheduler does not accept the new job
-//	 */
-//	public void scheduleJob(final ExecutionGraph executionGraph) throws SchedulingException {
-//
-//		final int requiredSlots = executionGraph.getRequiredSlots();
-//		final int availableSlots = this.getInstanceManager().getNumberOfSlots();
-//
-//		if(requiredSlots > availableSlots){
-//			throw new SchedulingException(String.format(
-//					"Not enough available task slots to run job %s (%s). Required: %d Available: %d . "
-//					+ "Either reduce the parallelism of your program, wait for other programs to finish, or increase "
-//					+ "the number of task slots in the cluster by adding more machines or increasing the number of slots "
-//					+ "per machine in conf/flink-conf.yaml .", 
-//					executionGraph.getJobName(), executionGraph.getJobID(), requiredSlots, availableSlots));
-//		}
-//
-//		// Subscribe to job status notifications
-//		executionGraph.registerJobStatusListener(this);
-//
-//		// Register execution listener for each vertex
-//		final ExecutionGraphIterator it2 = new ExecutionGraphIterator(executionGraph, true);
-//		while (it2.hasNext()) {
-//
-//			final ExecutionVertex vertex = it2.next();
-//			vertex.registerExecutionListener(new DefaultExecutionListener(this, vertex));
-//		}
-//
-//		// Register the scheduler as an execution stage listener
-//		executionGraph.registerExecutionStageListener(this);
-//
-//		// Add job to the job queue (important to add job to queue before requesting instances)
-//		synchronized (this.jobQueue) {
-//			this.jobQueue.add(executionGraph);
-//		}
-//
-//		// Request resources for the first stage of the job
-//
-//		final ExecutionStage executionStage = executionGraph.getCurrentExecutionStage();
-//		try {
-//			requestInstances(executionStage);
-//		} catch (InstanceException e) {
-//			final String exceptionMessage = StringUtils.stringifyException(e);
-//			LOG.error(exceptionMessage);
-//			this.jobQueue.remove(executionGraph);
-//			throw new SchedulingException(exceptionMessage);
-//		}
-//	}
-//
-//	/**
-//	 * Returns the execution graph which is associated with the given job ID.
-//	 *
-//	 * @param jobID
-//	 *        the job ID to search the execution graph for
-//	 * @return the execution graph which belongs to the given job ID or <code>null</code if no such execution graph
-//	 *         exists
-//	 */
-//	public ExecutionGraph getExecutionGraphByID(final JobID jobID) {
-//
-//		synchronized (this.jobQueue) {
-//
-//			final Iterator<ExecutionGraph> it = this.jobQueue.iterator();
-//			while (it.hasNext()) {
-//
-//				final ExecutionGraph executionGraph = it.next();
-//				if (executionGraph.getJobID().equals(jobID)) {
-//					return executionGraph;
-//				}
-//			}
-//		}
-//
-//		return null;
-//	}
-//
-//
-//
-//	public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
-//									final String optionalMessage) {
-//
-//		if (newJobStatus == InternalJobStatus.FAILED || newJobStatus == InternalJobStatus.FINISHED
-//				|| newJobStatus == InternalJobStatus.CANCELED) {
-//			removeJobFromSchedule(executionGraph);
-//		}
-//	}
-//
-//	public void nextExecutionStageEntered(final JobID jobID, final ExecutionStage executionStage) {
-//
-//		// Request new instances if necessary
-//		try {
-//			requestInstances(executionStage);
-//		} catch (InstanceException e) {
-//			// TODO: Handle error correctly
-//			LOG.error(StringUtils.stringifyException(e));
-//		}
-//
-//		// Deploy the assigned vertices
-//		deployAssignedInputVertices(executionStage.getExecutionGraph());
-//	}
-//
-//
-//	/**
-//	 * Returns the {@link org.apache.flink.runtime.instance.InstanceManager} object which is used by the current scheduler.
-//	 * 
-//	 * @return the {@link org.apache.flink.runtime.instance.InstanceManager} object which is used by the current scheduler
-//	 */
-//	public InstanceManager getInstanceManager() {
-//		return this.instanceManager;
-//	}
-//
-//
-//	/**
-//	 * Collects the instances required to run the job from the given {@link ExecutionStage} and requests them at the
-//	 * loaded instance manager.
-//	 * 
-//	 * @param executionStage
-//	 *        the execution stage to collect the required instances from
-//	 * @throws InstanceException
-//	 *         thrown if the given execution graph is already processing its final stage
-//	 */
-//	protected void requestInstances(final ExecutionStage executionStage) throws InstanceException {
-//
-//		final ExecutionGraph executionGraph = executionStage.getExecutionGraph();
-//
-//		synchronized (executionStage) {
-//
-//			final int requiredSlots = executionStage.getRequiredSlots();
-//
-//			LOG.info("Requesting " + requiredSlots + " slots for job " + executionGraph.getJobID());
-//
-//			this.instanceManager.requestInstance(executionGraph.getJobID(), executionGraph.getJobConfiguration(),
-//				requiredSlots);
-//
-//			// Switch vertex state to assigning
-//			final ExecutionGraphIterator it2 = new ExecutionGraphIterator(executionGraph, executionGraph
-//				.getIndexOfCurrentExecutionStage(), true, true);
-//			while (it2.hasNext()) {
-//
-//				it2.next().compareAndUpdateExecutionState(ExecutionState.CREATED, ExecutionState.SCHEDULED);
-//			}
-//		}
-//	}
-//
-//	void findVerticesToBeDeployed(final ExecutionVertex vertex,
-//			final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed,
-//			final Set<ExecutionVertex> alreadyVisited) {
-//
-//		if (!alreadyVisited.add(vertex)) {
-//			return;
-//		}
-//
-//		if (vertex.compareAndUpdateExecutionState(ExecutionState.ASSIGNED, ExecutionState.READY)) {
-//			final Instance instance = vertex.getAllocatedResource().getInstance();
-//
-//			if (instance instanceof DummyInstance) {
-//				LOG.error("Inconsistency: Vertex " + vertex + " is about to be deployed on a DummyInstance");
-//			}
-//
-//			List<ExecutionVertex> verticesForInstance = verticesToBeDeployed.get(instance);
-//			if (verticesForInstance == null) {
-//				verticesForInstance = new ArrayList<ExecutionVertex>();
-//				verticesToBeDeployed.put(instance, verticesForInstance);
-//			}
-//
-//			verticesForInstance.add(vertex);
-//		}
-//
-//		final int numberOfOutputGates = vertex.getNumberOfOutputGates();
-//		for (int i = 0; i < numberOfOutputGates; ++i) {
-//
-//			final ExecutionGate outputGate = vertex.getOutputGate(i);
-//			boolean deployTarget;
-//
-//			switch (outputGate.getChannelType()) {
-//			case NETWORK:
-//				deployTarget = false;
-//				break;
-//			case IN_MEMORY:
-//				deployTarget = true;
-//				break;
-//			default:
-//				throw new IllegalStateException("Unknown channel type");
-//			}
-//
-//			if (deployTarget) {
-//
-//				final int numberOfOutputChannels = outputGate.getNumberOfEdges();
-//				for (int j = 0; j < numberOfOutputChannels; ++j) {
-//					final ExecutionEdge outputChannel = outputGate.getEdge(j);
-//					final ExecutionVertex connectedVertex = outputChannel.getInputGate().getVertex();
-//					findVerticesToBeDeployed(connectedVertex, verticesToBeDeployed, alreadyVisited);
-//				}
-//			}
-//		}
-//	}
-//
-//	/**
-//	 * Collects all execution vertices with the state ASSIGNED starting from the given start vertex and
-//	 * deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
-//	 * 
-//	 * @param startVertex
-//	 *        the execution vertex to start the deployment from
-//	 */
-//	public void deployAssignedVertices(final ExecutionVertex startVertex) {
-//
-//		final JobID jobID = startVertex.getExecutionGraph().getJobID();
-//
-//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-//
-//		findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited);
-//
-//		if (!verticesToBeDeployed.isEmpty()) {
-//
-//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-//				.entrySet()
-//				.iterator();
-//
-//			while (it2.hasNext()) {
-//
-//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-//				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
-//			}
-//		}
-//	}
-//
-//	/**
-//	 * Collects all execution vertices with the state ASSIGNED from the given pipeline and deploys them on the assigned
-//	 * {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
-//	 * 
-//	 * @param pipeline
-//	 *        the execution pipeline to be deployed
-//	 */
-//	public void deployAssignedPipeline(final ExecutionPipeline pipeline) {
-//
-//		final JobID jobID = null;
-//
-//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-//
-//		final Iterator<ExecutionVertex> it = pipeline.iterator();
-//		while (it.hasNext()) {
-//			findVerticesToBeDeployed(it.next(), verticesToBeDeployed, alreadyVisited);
-//		}
-//
-//		if (!verticesToBeDeployed.isEmpty()) {
-//
-//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-//				.entrySet()
-//				.iterator();
-//
-//			while (it2.hasNext()) {
-//
-//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-//				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
-//			}
-//		}
-//	}
-//
-//	/**
-//	 * Collects all execution vertices with the state ASSIGNED starting from the given collection of start vertices and
-//	 * deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
-//	 * 
-//	 * @param startVertices
-//	 *        the collection of execution vertices to start the deployment from
-//	 */
-//	public void deployAssignedVertices(final Collection<ExecutionVertex> startVertices) {
-//
-//		JobID jobID = null;
-//
-//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-//
-//		for (final ExecutionVertex startVertex : startVertices) {
-//
-//			if (jobID == null) {
-//				jobID = startVertex.getExecutionGraph().getJobID();
-//			}
-//
-//			findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited);
-//		}
-//
-//		if (!verticesToBeDeployed.isEmpty()) {
-//
-//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-//				.entrySet()
-//				.iterator();
-//
-//			while (it2.hasNext()) {
-//
-//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-//				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
-//			}
-//		}
-//	}
-//
-//	/**
-//	 * Collects all execution vertices with the state ASSIGNED starting from the input vertices of the current execution
-//	 * stage and deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
-//	 * 
-//	 * @param executionGraph
-//	 *        the execution graph to collect the vertices from
-//	 */
-//	public void deployAssignedInputVertices(final ExecutionGraph executionGraph) {
-//
-//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-//		final ExecutionStage executionStage = executionGraph.getCurrentExecutionStage();
-//
-//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-//
-//		for (int i = 0; i < executionStage.getNumberOfStageMembers(); ++i) {
-//
-//			final ExecutionGroupVertex startVertex = executionStage.getStageMember(i);
-//			if (!startVertex.isInputVertex()) {
-//				continue;
-//			}
-//
-//			for (int j = 0; j < startVertex.getCurrentNumberOfGroupMembers(); ++j) {
-//				final ExecutionVertex vertex = startVertex.getGroupMember(j);
-//				findVerticesToBeDeployed(vertex, verticesToBeDeployed, alreadyVisited);
-//			}
-//		}
-//
-//		if (!verticesToBeDeployed.isEmpty()) {
-//
-//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-//				.entrySet()
-//				.iterator();
-//
-//			while (it2.hasNext()) {
-//
-//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-//				this.deploymentManager.deploy(executionGraph.getJobID(), entry.getKey(), entry.getValue());
-//			}
-//		}
-//	}
-//
-//
-//	@Override
-//	public void resourcesAllocated(final JobID jobID, final List<AllocatedResource> allocatedResources) {
-//
-//		if (allocatedResources == null) {
-//			LOG.error("Resource to lock is null!");
-//			return;
-//		}
-//
-//		for (final AllocatedResource allocatedResource : allocatedResources) {
-//			if (allocatedResource.getInstance() instanceof DummyInstance) {
-//				LOG.debug("Available instance is of type DummyInstance!");
-//				return;
-//			}
-//		}
-//
-//		final ExecutionGraph eg = getExecutionGraphByID(jobID);
-//
-//		if (eg == null) {
-//			/*
-//			 * The job have have been canceled in the meantime, in this case
-//			 * we release the instance immediately.
-//			 */
-//			try {
-//				for (final AllocatedResource allocatedResource : allocatedResources) {
-//					getInstanceManager().releaseAllocatedResource(allocatedResource);
-//				}
-//			} catch (InstanceException e) {
-//				LOG.error(e);
-//			}
-//			return;
-//		}
-//
-//		final Runnable command = new Runnable() {
-//
-//			/**
-//			 * {@inheritDoc}
-//			 */
-//			@Override
-//			public void run() {
-//
-//				final ExecutionStage stage = eg.getCurrentExecutionStage();
-//
-//				synchronized (stage) {
-//
-//					for (final AllocatedResource allocatedResource : allocatedResources) {
-//
-//						AllocatedResource resourceToBeReplaced = null;
-//						// Important: only look for instances to be replaced in the current stage
-//						final Iterator<ExecutionGroupVertex> groupIterator = new ExecutionGroupVertexIterator(eg, true,
-//							stage.getStageNumber());
-//						while (groupIterator.hasNext()) {
-//
-//							final ExecutionGroupVertex groupVertex = groupIterator.next();
-//							for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); ++i) {
-//
-//								final ExecutionVertex vertex = groupVertex.getGroupMember(i);
-//
-//								if (vertex.getExecutionState() == ExecutionState.SCHEDULED
-//									&& vertex.getAllocatedResource() != null) {
-//										resourceToBeReplaced = vertex.getAllocatedResource();
-//										break;
-//								}
-//							}
-//
-//							if (resourceToBeReplaced != null) {
-//								break;
-//							}
-//						}
-//
-//						// For some reason, we don't need this instance
-//						if (resourceToBeReplaced == null) {
-//							LOG.error("Instance " + allocatedResource.getInstance() + " is not required for job"
-//								+ eg.getJobID());
-//							try {
-//								getInstanceManager().releaseAllocatedResource(allocatedResource);
-//							} catch (InstanceException e) {
-//								LOG.error(e);
-//							}
-//							return;
-//						}
-//
-//						// Replace the selected instance
-//						final Iterator<ExecutionVertex> it = resourceToBeReplaced.assignedVertices();
-//						while (it.hasNext()) {
-//							final ExecutionVertex vertex = it.next();
-//							vertex.setAllocatedResource(allocatedResource);
-//							vertex.updateExecutionState(ExecutionState.ASSIGNED);
-//						}
-//					}
-//				}
-//
-//				// Deploy the assigned vertices
-//				deployAssignedInputVertices(eg);
-//
-//			}
-//
-//		};
-//
-//		eg.executeCommand(command);
-//	}
-//
-//	/**
-//	 * Checks if the given {@link AllocatedResource} is still required for the
-//	 * execution of the given execution graph. If the resource is no longer
-//	 * assigned to a vertex that is either currently running or about to run
-//	 * the given resource is returned to the instance manager for deallocation.
-//	 * 
-//	 * @param executionGraph
-//	 *        the execution graph the provided resource has been used for so far
-//	 * @param allocatedResource
-//	 *        the allocated resource to check the assignment for
-//	 */
-//	public void checkAndReleaseAllocatedResource(final ExecutionGraph executionGraph,
-//			final AllocatedResource allocatedResource) {
-//
-//		if (allocatedResource == null) {
-//			LOG.error("Resource to lock is null!");
-//			return;
-//		}
-//
-//		if (allocatedResource.getInstance() instanceof DummyInstance) {
-//			LOG.debug("Available instance is of type DummyInstance!");
-//			return;
-//		}
-//
-//		boolean resourceCanBeReleased = true;
-//		final Iterator<ExecutionVertex> it = allocatedResource.assignedVertices();
-//		while (it.hasNext()) {
-//			final ExecutionVertex vertex = it.next();
-//			final ExecutionState state = vertex.getExecutionState();
-//
-//			if (state != ExecutionState.CREATED && state != ExecutionState.FINISHED
-//				&& state != ExecutionState.FAILED && state != ExecutionState.CANCELED) {
-//
-//				resourceCanBeReleased = false;
-//				break;
-//			}
-//		}
-//
-//		if (resourceCanBeReleased) {
-//
-//			LOG.info("Releasing instance " + allocatedResource.getInstance());
-//			try {
-//				getInstanceManager().releaseAllocatedResource(allocatedResource);
-//			} catch (InstanceException e) {
-//				LOG.error(StringUtils.stringifyException(e));
-//			}
-//		}
-//	}
-//
-//	DeploymentManager getDeploymentManager() {
-//		return this.deploymentManager;
-//	}
-//
-//
-//
-//	@Override
-//	public void allocatedResourcesDied(final JobID jobID, final List<AllocatedResource> allocatedResources) {
-//
-//		final ExecutionGraph eg = getExecutionGraphByID(jobID);
-//
-//		if (eg == null) {
-//			LOG.error("Cannot find execution graph for job with ID " + jobID);
-//			return;
-//		}
-//
-//		final Runnable command = new Runnable() {
-//
-//			/**
-//			 * {@inheritDoc}
-//			 */
-//			@Override
-//			public void run() {
-//
-//				synchronized (eg) {
-//
-//					for (final AllocatedResource allocatedResource : allocatedResources) {
-//
-//						LOG.info("Resource " + allocatedResource.getInstance().getName() + " for Job " + jobID
-//							+ " died.");
-//
-//						final ExecutionGraph executionGraph = getExecutionGraphByID(jobID);
-//
-//						if (executionGraph == null) {
-//							LOG.error("Cannot find execution graph for job " + jobID);
-//							return;
-//						}
-//
-//						Iterator<ExecutionVertex> vertexIter = allocatedResource.assignedVertices();
-//
-//						// Assign vertices back to a dummy resource.
-//						final DummyInstance dummyInstance = DummyInstance.createDummyInstance();
-//						final AllocatedResource dummyResource = new AllocatedResource(dummyInstance,
-//								new AllocationID());
-//
-//						while (vertexIter.hasNext()) {
-//							final ExecutionVertex vertex = vertexIter.next();
-//							vertex.setAllocatedResource(dummyResource);
-//						}
-//
-//						final String failureMessage = allocatedResource.getInstance().getName() + " died";
-//
-//						vertexIter = allocatedResource.assignedVertices();
-//
-//						while (vertexIter.hasNext()) {
-//							final ExecutionVertex vertex = vertexIter.next();
-//							final ExecutionState state = vertex.getExecutionState();
-//
-//							switch (state) {
-//							case ASSIGNED:
-//							case READY:
-//							case STARTING:
-//							case RUNNING:
-//							case FINISHING:
-//
-//							vertex.updateExecutionState(ExecutionState.FAILED, failureMessage);
-//
-//							break;
-//						default:
-//							}
-//					}
-//
-//					// TODO: Fix this
-//					/*
-//					 * try {
-//					 * requestInstances(this.executionVertex.getGroupVertex().getExecutionStage());
-//					 * } catch (InstanceException e) {
-//					 * e.printStackTrace();
-//					 * // TODO: Cancel the entire job in this case
-//					 * }
-//					 */
-//				}
-//			}
-//
-//			final InternalJobStatus js = eg.getJobStatus();
-//			if (js != InternalJobStatus.FAILING && js != InternalJobStatus.FAILED) {
-//
-//				// TODO: Fix this
-//				// deployAssignedVertices(eg);
-//
-//				final ExecutionStage stage = eg.getCurrentExecutionStage();
-//
-//				try {
-//					requestInstances(stage);
-//				} catch (InstanceException e) {
-//					e.printStackTrace();
-//					// TODO: Cancel the entire job in this case
-//				}
-//			}
-//		}
-//		};
-//
-//		eg.executeCommand(command);
-//	}
 	
 	// --------------------------------------------------------------------------------------------
-	//  Canceling
+	//  Scheduling
 	// --------------------------------------------------------------------------------------------
 	
-	public void removeAllTasksForJob(JobID job) {
+	/**
+	 * @param task
+	 * @param queueIfNoResource If true, this call will queue the request if no resource is immediately
+	 *                          available. If false, it will throw a {@link NoResourceAvailableException}
+	 *                          if no resource is immediately available.
+	 */
+	public void scheduleTask(ScheduledUnit task, boolean queueIfNoResource) {
+		if (task == null) {
+			throw new IllegalArgumentException();
+		}
+		
+		// if there is already a slot for that resource
+		AllocatedSlot existing = this.allocatedSlots.get(task.getSharedResourceId());
+		if (existing != null) {
+			// try to attach to the existing slot
+			if (existing.runTask(task.getTaskVertex())) {
+				// all good, we are done
+				return;
+			}
+			// else: the slot was deallocated, we need to proceed as if there was none
+		}
+		
+		// check if there is a slot that has an available sub-slot for that group-vertex
+		// TODO
+		
 		
 	}
 
@@ -836,101 +233,114 @@ public class DefaultScheduler implements InstanceListener {
 	//  Scheduling
 	// --------------------------------------------------------------------------------------------
 	
-	/**
-	 * Schedules the given unit to an available resource. This call blocks if no resource
-	 * is currently available
-	 * 
-	 * @param unit The unit to be scheduled.
-	 */
-	protected void scheduleNextUnit(ScheduledUnit unit) {
-		if (unit == null) {
-			throw new IllegalArgumentException("Unit to schedule must not be null.");
-		}
-		
-		// see if the resource Id has already an assigned resource
-		AllocatedSlot resource = this.allocatedSlots.get(unit.getSharedResourceId());
-		
-		if (resource == null) {
-			// not yet allocated. find a slot to schedule to
-			try {
-				resource = getResourceToScheduleUnit(unit, this.rejectIfNoResourceAvailable);
-				if (resource == null) {
-					throw new RuntimeException("Error: The resource to schedule to is null.");
-				}
-			}
-			catch (Exception e) {
-				// we cannot go on, the task needs to know what to do now.
-				unit.getTaskVertex().handleException(e);
-				return;
-			}
-		}
-		
-		resource.runTask(unit.getTaskVertex());
-	}
+//	/**
+//	 * Schedules the given unit to an available resource. This call blocks if no resource
+//	 * is currently available
+//	 * 
+//	 * @param unit The unit to be scheduled.
+//	 */
+//	protected void scheduleQueuedUnit(ScheduledUnit unit) {
+//		if (unit == null) {
+//			throw new IllegalArgumentException("Unit to schedule must not be null.");
+//		}
+//		
+//		// see if the resource Id has already an assigned resource
+//		AllocatedSlot resource = this.allocatedSlots.get(unit.getSharedResourceId());
+//		
+//		if (resource == null) {
+//			// not yet allocated. find a slot to schedule to
+//			try {
+//				resource = getResourceToScheduleUnit(unit, this.rejectIfNoResourceAvailable);
+//				if (resource == null) {
+//					throw new RuntimeException("Error: The resource to schedule to is null.");
+//				}
+//			}
+//			catch (Exception e) {
+//				// we cannot go on, the task needs to know what to do now.
+//				unit.getTaskVertex().handleException(e);
+//				return;
+//			}
+//		}
+//		
+//		resource.runTask(unit.getTaskVertex());
+//	}
 	
 	/**
 	 * Acquires a resource to schedule the given unit to. This call may block if no
 	 * resource is currently available, or throw an exception, based on the given flag.
 	 * 
 	 * @param unit The unit to find a resource for.
-	 * @param exceptionOnNoAvailability If true, this call should not block is no resource is available,
-	 *                                  but throw a {@link NoResourceAvailableException}.
 	 * @return The resource to schedule the execution of the given unit on.
 	 * 
 	 * @throws NoResourceAvailableException If the {@code exceptionOnNoAvailability} flag is true and the scheduler
 	 *                                      has currently no resources available.
 	 */
-	protected AllocatedSlot getResourceToScheduleUnit(ScheduledUnit unit, boolean exceptionOnNoAvailability) 
+	protected AllocatedSlot getNewSlotForTask(ScheduledUnit unit, boolean queueIfNoResource) 
 		throws NoResourceAvailableException
 	{
-		AllocatedSlot slot = null;
-		
-		while (true) {
-			synchronized (this.lock) {
-				Instance instanceToUse = this.instancesWithAvailableResources.poll();
-				
-				// if there is nothing, throw an exception or wait, depending on what is configured
-				if (instanceToUse == null) {
-					if (exceptionOnNoAvailability) {
-						throw new NoResourceAvailableException(unit);
+		synchronized (this.lock) {
+			Instance instanceToUse = this.instancesWithAvailableResources.poll();
+			
+			// if there is nothing, throw an exception or wait, depending on what is configured
+			if (instanceToUse != null) {
+				try {
+					AllocatedSlot slot = instanceToUse.allocateSlot(unit.getJobId(), unit.getSharedResourceId());
+					
+					// if the instance has further available slots, re-add it to the set of available resources.
+					if (instanceToUse.hasResourcesAvailable()) {
+						this.instancesWithAvailableResources.add(instanceToUse);
 					}
-					else {
-						try {
-							do {
-								this.lock.wait(2000);
-							}
-							while (!shutdown.get() && 
-									(instanceToUse = this.instancesWithAvailableResources.poll()) == null);
-						}
-						catch (InterruptedException e) {
-							throw new NoResourceAvailableException("The scheduler was interrupted.");
+					
+					if (slot != null) {
+						AllocatedSlot previous = this.allocatedSlots.putIfAbsent(unit.getSharedResourceId(), slot);
+						if (previous != null) {
+							// concurrently, someone allocated a slot for that ID
+							// release the new one
+							slot.cancelResource();
+							slot = previous;
 						}
 					}
-				}
-				
-				// at this point, we have an instance. request a slot from the instance
-				try {
-					slot = instanceToUse.allocateSlot(unit.getJobId(), unit.getSharedResourceId());
+					// else fall through the loop
 				}
 				catch (InstanceDiedException e) {
 					// the instance died it has not yet been propagated to this scheduler
 					// remove the instance from the set of available instances
 					this.allInstances.remove(instanceToUse);
 				}
+			}
+				
+			
+			if (queueIfNoResource) {
+				this.taskQueue.add(unit);
+			}
+			else {
+				throw new NoResourceAvailableException(unit);
+			}
+				// at this point, we have an instance. request a slot from the instance
+				
 				
 				// if the instance has further available slots, re-add it to the set of available
 				// resources.
-				// if it does not, but asynchronously 
+				// if it does not, but asynchronously a slot became available, we may attempt to add the
+				// instance twice, which does not matter because of the set semantics of the "instancesWithAvailableResources"
 				if (instanceToUse.hasResourcesAvailable()) {
 					this.instancesWithAvailableResources.add(instanceToUse);
 				}
 				
 				if (slot != null) {
-					return slot;
+					AllocatedSlot previous = this.allocatedSlots.putIfAbsent(unit.getSharedResourceId(), slot);
+					if (previous != null) {
+						// concurrently, someone allocated a slot for that ID
+						// release the new one
+						slot.cancelResource();
+						slot = previous;
+					}
 				}
 				// else fall through the loop
 			}
 		}
+		
+		return slot;
 	}
 	
 	protected void runSchedulerLoop() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
index 8caf64a..338529f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
@@ -15,6 +15,8 @@
 
 package org.apache.flink.runtime.jobmanager.scheduler;
 
+import java.util.concurrent.atomic.AtomicBoolean;
+
 import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
 import org.apache.flink.runtime.jobgraph.JobID;
 
@@ -26,6 +28,8 @@ public class ScheduledUnit {
 	
 	private final ResourceId resourceId;
 	
+	private final AtomicBoolean scheduled = new AtomicBoolean(false);
+	
 	
 	public ScheduledUnit(JobID jobId, ExecutionVertex2 taskVertex) {
 		this(jobId, taskVertex, new ResourceId());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
index 02c814c..b047222 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
@@ -251,5 +251,15 @@ public class LocalInstanceManagerTest {
 
 		@Override
 		public void deploy(JobID jobID, Instance instance, List<ExecutionVertex> verticesToBeDeployed) {}
+
+		@Override
+		public boolean sendHeartbeat(InstanceID taskManagerId) {
+			return true;
+		}
+
+		@Override
+		public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) {
+			return new InstanceID();
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/aa7550aa/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java
new file mode 100644
index 0000000..d45b131
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/ipc/RpcTest.java
@@ -0,0 +1,130 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.*;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+
+import org.apache.flink.core.protocols.VersionedProtocol;
+import org.apache.flink.runtime.net.NetUtils;
+import org.apache.flink.types.DoubleValue;
+import org.apache.flink.types.LongValue;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.util.LogUtils;
+
+
+public class RpcTest {
+
+	@BeforeClass
+	public static void initLogger() {
+		LogUtils.initializeDefaultConsoleLogger();
+	}
+	
+	
+	@Test
+	public void testRpc() {
+		try { 
+			Server server = null;
+			TestProtocol proxy = null;
+			
+			try {
+				// setup the RPCs
+				int port = getAvailablePort();
+				server = RPC.getServer(new TestProtocolImpl(), "localhost", port, 4);
+				server.start();
+				
+				proxy = RPC.getProxy(TestProtocol.class, new InetSocketAddress("localhost", port), NetUtils.getSocketFactory());
+				
+				// make a few calls with various types
+//				proxy.methodWithNoParameters();
+				
+				assertEquals(19, proxy.methodWithPrimitives(16, new StringValue("abc")));
+				assertEquals(new DoubleValue(17.0), proxy.methodWithWritables(new LongValue(17)));
+			}
+			finally {
+				if (proxy != null) {
+					RPC.stopProxy(proxy);
+				}
+				if (server != null) {
+					server.stop();
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	
+	private static final int getAvailablePort() throws IOException {
+		ServerSocket serverSocket = null;
+		for (int i = 0; i < 50; i++){
+			try {
+				serverSocket = new ServerSocket(0);
+				int port = serverSocket.getLocalPort();
+				if (port != 0) {
+					return port;
+				}
+			}
+			catch (IOException e) {}
+			finally {
+				if (serverSocket != null) {
+					serverSocket.close();
+				}
+			}
+		}
+		
+		throw new IOException("Could not find a free port.");
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static interface TestProtocol extends VersionedProtocol {
+		
+		public void methodWithNoParameters();
+		
+		public int methodWithPrimitives(int intParam, StringValue writableParam);
+		
+		public DoubleValue methodWithWritables(LongValue writableParam);
+	}
+	
+	
+	public static final class TestProtocolImpl implements TestProtocol {
+
+		@Override
+		public void methodWithNoParameters() {}
+
+		@Override
+		public int methodWithPrimitives(int intParam, StringValue writableParam) {
+			return intParam + writableParam.length();
+		}
+
+		@Override
+		public DoubleValue methodWithWritables(LongValue writableParam) {
+			return new DoubleValue(writableParam.getValue());
+		}
+	}
+}


[56/63] [abbrv] git commit: Fix Java6 errors in AbstractID, pull JobManager changes into YARN app master

Posted by se...@apache.org.
Fix Java6 errors in AbstractID, pull JobManager changes into YARN app master


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/73ebd3e0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/73ebd3e0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/73ebd3e0

Branch: refs/heads/master
Commit: 73ebd3e0b1039d9509499f9953424e07286ea50d
Parents: cdee875
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 13:04:04 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:01 2014 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/yarn/appMaster/ApplicationMaster.java  | 2 +-
 .../src/main/java/org/apache/flink/runtime/AbstractID.java       | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73ebd3e0/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/ApplicationMaster.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/ApplicationMaster.java b/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/ApplicationMaster.java
index afda8e0..3a7701b 100644
--- a/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/ApplicationMaster.java
+++ b/flink-addons/flink-yarn/src/main/java/org/apache/flink/yarn/appMaster/ApplicationMaster.java
@@ -486,7 +486,7 @@ public class ApplicationMaster implements YARNClientMasterProtocol {
 			amStatus.setNumSlots(0);
 		} else {
 			amStatus.setNumTaskManagers(jobManager.getNumberOfTaskManagers());
-			amStatus.setNumSlots(jobManager.getAvailableSlots());
+			amStatus.setNumSlots(jobManager.getTotalNumberOfRegisteredSlots());
 		}
 		amStatus.setMessageCount(messages.size());
 		amStatus.setFailed(isFailed);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/73ebd3e0/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
index e651f6c..0834222 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
@@ -199,8 +199,8 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 	
 	@Override
 	public int compareTo(AbstractID o) {
-		int diff1 = Long.compare(this.upperPart, o.upperPart);
-		int diff2 = Long.compare(this.lowerPart, o.lowerPart);
+		int diff1 = (this.upperPart < o.upperPart) ? -1 : ((this.upperPart == o.upperPart) ? 0 : 1);
+		int diff2 = (this.lowerPart < o.lowerPart) ? -1 : ((this.lowerPart == o.lowerPart) ? 0 : 1);
 		return diff1 == 0 ? diff2 : diff1;
 	}
 }


[60/63] [abbrv] git commit: Fix TaskDeploymentDescriptor serialization

Posted by se...@apache.org.
Fix TaskDeploymentDescriptor serialization


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/a00ef7a7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/a00ef7a7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/a00ef7a7

Branch: refs/heads/master
Commit: a00ef7a7cb0b4ee63046e0897c7988d673c4701c
Parents: cd699c5
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 15:57:07 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:02 2014 +0200

----------------------------------------------------------------------
 .../flink/runtime/deployment/TaskDeploymentDescriptor.java     | 6 +++---
 .../flink/runtime/deployment/TaskDeploymentDescriptorTest.java | 4 ++++
 2 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a00ef7a7/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
index e1e80f9..330790d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
@@ -280,9 +280,9 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 		inputGates = readGateList(in);
 		outputGates = readGateList(in);
 
-		String[] jarFiles = new String[in.readInt()];
-		for (int i = 0; i < jarFiles.length; i++) {
-			jarFiles[i] = StringValue.readString(in);
+		this.requiredJarFiles = new String[in.readInt()];
+		for (int i = 0; i < this.requiredJarFiles.length; i++) {
+			this.requiredJarFiles[i] = StringValue.readString(in);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/a00ef7a7/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
index 1f66f93..5b62f3d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
@@ -20,9 +20,11 @@ package org.apache.flink.runtime.deployment;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.flink.configuration.Configuration;
@@ -70,6 +72,8 @@ public class TaskDeploymentDescriptorTest {
 			assertEquals(orig.getCurrentNumberOfSubtasks(), copy.getCurrentNumberOfSubtasks());
 			assertEquals(orig.getOutputGates(), copy.getOutputGates());
 			assertEquals(orig.getInputGates(), copy.getInputGates());
+			
+			assertTrue(Arrays.equals(orig.getRequiredJarFiles(), copy.getRequiredJarFiles()));
 		}
 		catch (Exception e) {
 			e.printStackTrace();


[52/63] [abbrv] git commit: Adjust tests to new JobGraphModel

Posted by se...@apache.org.
Adjust tests to new JobGraphModel


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/caa4ebef
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/caa4ebef
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/caa4ebef

Branch: refs/heads/master
Commit: caa4ebef82d84b5b4ffb945f90c06e0e82b9a102
Parents: e56d883
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 16:54:38 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:50 2014 +0200

----------------------------------------------------------------------
 .../client/program/PackagedProgramTest.java     |   1 -
 .../runtime/jobgraph/AbstractJobVertex.java     |   6 +-
 .../runtime/jobgraph/JobManagerTestUtils.java   |  89 +++++++++
 .../jobmanager/CoLocationConstraintITCase.java  | 111 +++++++++++
 .../runtime/jobmanager/JobManagerITCase.java    | 148 +--------------
 .../runtime/jobmanager/SlotSharingITCase.java   | 186 +++++++++++++++++++
 .../tasks/AgnosticBinaryReceiver.java           |  41 ++++
 .../jobmanager/tasks/AgnosticReceiver.java      |  38 ++++
 .../runtime/jobmanager/tasks/Receiver.java      |  44 +++++
 .../flink/runtime/jobmanager/tasks/Sender.java  |  46 +++++
 .../BroadcastVarsNepheleITCase.java             |  26 +--
 .../KMeansIterativeNepheleITCase.java           |  29 +--
 .../test/cancelling/CancellingTestBase.java     |   1 -
 .../test/iterative/nephele/JobGraphUtils.java   |  48 ++---
 .../test/runtime/NetworkStackThroughput.java    |  54 +++---
 flink-tests/src/test/resources/logback-test.xml |   4 +-
 16 files changed, 655 insertions(+), 217 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java
index 372c65b..4adfdb8 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramTest.java
@@ -31,7 +31,6 @@ public class PackagedProgramTest {
 	@Test
 	public void testGetPreviewPlan() {
 		try {
-			
 			PackagedProgram prog = new PackagedProgram(new File(CliFrontendTestUtils.getTestJarPath()));
 			Assert.assertNotNull(prog.getPreviewPlan());
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index 82823b2..d2462ba 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -62,11 +62,13 @@ public class AbstractJobVertex implements java.io.Serializable {
 	/** Optionally, a source of input splits */
 	private InputSplitSource<?> inputSplitSource;
 	
+	/** The name of the vertex */
+	private String name;
+	
 	/** Optionally, a sharing group that allows subtasks from different job vertices to run concurrently in one slot */
 	private SlotSharingGroup slotSharingGroup;
 	
-	/** The name of the vertex */
-	private String name;
+//	private AbstractJobVertex coLocatedWith
 
 	// --------------------------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java
new file mode 100644
index 0000000..14a73e1
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java
@@ -0,0 +1,89 @@
+/**
+ * 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.jobgraph;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.net.ServerSocket;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.runtime.ExecutionMode;
+import org.apache.flink.runtime.jobmanager.JobManager;
+
+public class JobManagerTestUtils {
+
+	public static final JobManager startJobManager(int numSlots) throws Exception {
+		Configuration cfg = new Configuration();
+		cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
+		cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, getAvailablePort());
+		cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10);
+		cfg.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots);
+		
+		GlobalConfiguration.includeConfiguration(cfg);
+		
+		JobManager jm = new JobManager(ExecutionMode.LOCAL);
+		
+		// we need to wait until the taskmanager is registered
+		// max time is 5 seconds
+		long deadline = System.currentTimeMillis() + 5000;
+		
+		while (jm.getAvailableSlots() < numSlots && System.currentTimeMillis() < deadline) {
+			Thread.sleep(10);
+		}
+		
+		assertEquals(numSlots, jm.getAvailableSlots());
+		
+		return jm;
+	}
+	
+	public static int getAvailablePort() throws IOException {
+		for (int i = 0; i < 50; i++) {
+			ServerSocket serverSocket = null;
+			try {
+				serverSocket = new ServerSocket(0);
+				int port = serverSocket.getLocalPort();
+				if (port != 0) {
+					return port;
+				}
+			} finally {
+				serverSocket.close();
+			}
+		}
+		
+		throw new IOException("could not find free port");
+	}
+	
+	public static void waitForTaskThreadsToBeTerminated() throws InterruptedException {
+		Thread[] threads = new Thread[Thread.activeCount()];
+		Thread.enumerate(threads);
+		
+		for (Thread t : threads) {
+			if (t == null) {
+				continue;
+			}
+			ThreadGroup tg = t.getThreadGroup();
+			if (tg != null && tg.getName() != null && tg.getName().equals("Task Threads")) {
+				t.join();
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
new file mode 100644
index 0000000..9bda8ed
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
@@ -0,0 +1,111 @@
+/**
+ * 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.jobmanager;
+
+import static org.apache.flink.runtime.jobgraph.JobManagerTestUtils.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.runtime.client.AbstractJobResult;
+import org.apache.flink.runtime.client.JobSubmissionResult;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.instance.LocalInstanceManager;
+import org.apache.flink.runtime.io.network.bufferprovider.GlobalBufferPool;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmanager.tasks.Receiver;
+import org.apache.flink.runtime.jobmanager.tasks.Sender;
+import org.junit.Test;
+
+public class CoLocationConstraintITCase {
+
+	
+	/**
+	 * This job runs in N slots with N senders and N receivers. Unless slot sharing is used, it cannot complete.
+	 */
+	@Test
+	public void testForwardJob() {
+		
+		final int NUM_TASKS = 31;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(Receiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(sender.getID(), receiver.getID());
+			sender.setSlotSharingGroup(sharingGroup);
+			receiver.setSlotSharingGroup(sharingGroup);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			final JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
+			try {
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FINISHED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index f4d74a3..44d1c11 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -18,17 +18,12 @@
 
 package org.apache.flink.runtime.jobmanager;
 
+import static org.apache.flink.runtime.jobgraph.JobManagerTestUtils.*;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
-import java.net.ServerSocket;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.client.AbstractJobResult;
 import org.apache.flink.runtime.client.JobSubmissionResult;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
@@ -42,8 +37,12 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.tasks.AgnosticBinaryReceiver;
+import org.apache.flink.runtime.jobmanager.tasks.AgnosticReceiver;
 import org.apache.flink.runtime.jobmanager.tasks.BlockingNoOpInvokable;
 import org.apache.flink.runtime.jobmanager.tasks.NoOpInvokable;
+import org.apache.flink.runtime.jobmanager.tasks.Receiver;
+import org.apache.flink.runtime.jobmanager.tasks.Sender;
 import org.apache.flink.runtime.types.IntegerRecord;
 import org.junit.Test;
 
@@ -837,144 +836,9 @@ public class JobManagerITCase {
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	
-	private static final JobManager startJobManager(int numSlots) throws Exception {
-		Configuration cfg = new Configuration();
-		cfg.setString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, "localhost");
-		cfg.setInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, getAvailablePort());
-		cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10);
-		cfg.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, numSlots);
-		
-		GlobalConfiguration.includeConfiguration(cfg);
-		
-		JobManager jm = new JobManager(ExecutionMode.LOCAL);
-		
-		// we need to wait until the taskmanager is registered
-		// max time is 5 seconds
-		long deadline = System.currentTimeMillis() + 5000;
-		
-		while (jm.getAvailableSlots() < numSlots && System.currentTimeMillis() < deadline) {
-			Thread.sleep(10);
-		}
-		
-		assertEquals(numSlots, jm.getAvailableSlots());
-		
-		return jm;
-	}
-	
-	private static int getAvailablePort() throws IOException {
-		for (int i = 0; i < 50; i++) {
-			ServerSocket serverSocket = null;
-			try {
-				serverSocket = new ServerSocket(0);
-				int port = serverSocket.getLocalPort();
-				if (port != 0) {
-					return port;
-				}
-			} finally {
-				serverSocket.close();
-			}
-		}
-		
-		throw new IOException("could not find free port");
-	}
-	
-	private static void waitForTaskThreadsToBeTerminated() throws InterruptedException {
-		Thread[] threads = new Thread[Thread.activeCount()];
-		Thread.enumerate(threads);
-		
-		for (Thread t : threads) {
-			if (t == null) {
-				continue;
-			}
-			ThreadGroup tg = t.getThreadGroup();
-			if (tg != null && tg.getName() != null && tg.getName().equals("Task Threads")) {
-				t.join();
-			}
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
 	//  Simple test tasks
 	// --------------------------------------------------------------------------------------------
 	
-	public static final class Sender extends AbstractInvokable {
-
-		private RecordWriter<IntegerRecord> writer;
-		
-		@Override
-		public void registerInputOutput() {
-			writer = new RecordWriter<IntegerRecord>(this);
-		}
-
-		@Override
-		public void invoke() throws Exception {
-			try {
-				writer.initializeSerializers();
-				writer.emit(new IntegerRecord(42));
-				writer.emit(new IntegerRecord(1337));
-				writer.flush();
-			}
-			finally {
-				writer.clearBuffers();
-			}
-		}
-	}
-	
-	public static final class Receiver extends AbstractInvokable {
-
-		private RecordReader<IntegerRecord> reader;
-		
-		@Override
-		public void registerInputOutput() {
-			reader = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
-		}
-
-		@Override
-		public void invoke() throws Exception {
-			IntegerRecord i1 = reader.next();
-			IntegerRecord i2 = reader.next();
-			IntegerRecord i3 = reader.next();
-			
-			if (i1.getValue() != 42 || i2.getValue() != 1337 || i3 != null) {
-				throw new Exception("Wrong Data Received");
-			}
-		}
-	}
-	
-	public static final class AgnosticReceiver extends AbstractInvokable {
-
-		private RecordReader<IntegerRecord> reader;
-		
-		@Override
-		public void registerInputOutput() {
-			reader = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
-		}
-
-		@Override
-		public void invoke() throws Exception {
-			while (reader.next() != null);
-		}
-	}
-	
-	public static final class AgnosticBinaryReceiver extends AbstractInvokable {
-
-		private RecordReader<IntegerRecord> reader1;
-		private RecordReader<IntegerRecord> reader2;
-		
-		@Override
-		public void registerInputOutput() {
-			reader1 = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
-			reader2 = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
-		}
-
-		@Override
-		public void invoke() throws Exception {
-			while (reader1.next() != null);
-			while (reader2.next() != null);
-		}
-	}
-	
 	public static final class ExceptionSender extends AbstractInvokable {
 
 		private RecordWriter<IntegerRecord> writer;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java
new file mode 100644
index 0000000..98abc8d
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java
@@ -0,0 +1,186 @@
+/**
+ * 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.jobmanager;
+
+import static org.apache.flink.runtime.jobgraph.JobManagerTestUtils.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.flink.runtime.client.AbstractJobResult;
+import org.apache.flink.runtime.client.JobSubmissionResult;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.instance.LocalInstanceManager;
+import org.apache.flink.runtime.io.network.bufferprovider.GlobalBufferPool;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.apache.flink.runtime.jobmanager.tasks.AgnosticBinaryReceiver;
+import org.apache.flink.runtime.jobmanager.tasks.Receiver;
+import org.apache.flink.runtime.jobmanager.tasks.Sender;
+import org.junit.Test;
+
+public class SlotSharingITCase {
+
+	
+	/**
+	 * This job runs in N slots with N senders and N receivers. Unless slot sharing is used, it cannot complete.
+	 */
+	@Test
+	public void testForwardJob() {
+		
+		final int NUM_TASKS = 31;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(Receiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(sender.getID(), receiver.getID());
+			sender.setSlotSharingGroup(sharingGroup);
+			receiver.setSlotSharingGroup(sharingGroup);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			final JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
+			try {
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FINISHED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	/**
+	 * This job runs in N slots with 2 * N senders and N receivers. Unless slot sharing is used, it cannot complete.
+	 */
+	@Test
+	public void testTwoInputJob() {
+		
+		final int NUM_TASKS = 11;
+		
+		try {
+			final AbstractJobVertex sender1 = new AbstractJobVertex("Sender1");
+			final AbstractJobVertex sender2 = new AbstractJobVertex("Sender2");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender1.setInvokableClass(Sender.class);
+			sender2.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(AgnosticBinaryReceiver.class);
+			
+			sender1.setParallelism(NUM_TASKS);
+			sender2.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(sender1.getID(), sender2.getID(), receiver.getID());
+			sender1.setSlotSharingGroup(sharingGroup);
+			sender2.setSlotSharingGroup(sharingGroup);
+			receiver.setSlotSharingGroup(sharingGroup);;
+			
+			receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE);
+			receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE);
+			
+			final JobGraph jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2);
+			
+			JobManager jm = startJobManager(NUM_TASKS);
+			
+			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
+								.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
+			
+			try {
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FINISHED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				// make sure that in any case, the network buffers are all returned
+				waitForTaskThreadsToBeTerminated();
+				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticBinaryReceiver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticBinaryReceiver.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticBinaryReceiver.java
new file mode 100644
index 0000000..3784205
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticBinaryReceiver.java
@@ -0,0 +1,41 @@
+/**
+ * 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.jobmanager.tasks;
+
+import org.apache.flink.runtime.io.network.api.RecordReader;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.types.IntegerRecord;
+
+public final class AgnosticBinaryReceiver extends AbstractInvokable {
+
+	private RecordReader<IntegerRecord> reader1;
+	private RecordReader<IntegerRecord> reader2;
+	
+	@Override
+	public void registerInputOutput() {
+		reader1 = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+		reader2 = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+	}
+
+	@Override
+	public void invoke() throws Exception {
+		while (reader1.next() != null);
+		while (reader2.next() != null);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticReceiver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticReceiver.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticReceiver.java
new file mode 100644
index 0000000..ce38b46
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/AgnosticReceiver.java
@@ -0,0 +1,38 @@
+/**
+ * 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.jobmanager.tasks;
+
+import org.apache.flink.runtime.io.network.api.RecordReader;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.types.IntegerRecord;
+
+public final class AgnosticReceiver extends AbstractInvokable {
+
+	private RecordReader<IntegerRecord> reader;
+	
+	@Override
+	public void registerInputOutput() {
+		reader = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+	}
+
+	@Override
+	public void invoke() throws Exception {
+		while (reader.next() != null);
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Receiver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Receiver.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Receiver.java
new file mode 100644
index 0000000..298673a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Receiver.java
@@ -0,0 +1,44 @@
+/**
+ * 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.jobmanager.tasks;
+
+import org.apache.flink.runtime.io.network.api.RecordReader;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.types.IntegerRecord;
+
+public final class Receiver extends AbstractInvokable {
+
+	private RecordReader<IntegerRecord> reader;
+	
+	@Override
+	public void registerInputOutput() {
+		reader = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+	}
+
+	@Override
+	public void invoke() throws Exception {
+		IntegerRecord i1 = reader.next();
+		IntegerRecord i2 = reader.next();
+		IntegerRecord i3 = reader.next();
+		
+		if (i1.getValue() != 42 || i2.getValue() != 1337 || i3 != null) {
+			throw new Exception("Wrong Data Received");
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Sender.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Sender.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Sender.java
new file mode 100644
index 0000000..340465b
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/tasks/Sender.java
@@ -0,0 +1,46 @@
+/**
+ * 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.jobmanager.tasks;
+
+import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.types.IntegerRecord;
+
+public final class Sender extends AbstractInvokable {
+
+	private RecordWriter<IntegerRecord> writer;
+	
+	@Override
+	public void registerInputOutput() {
+		writer = new RecordWriter<IntegerRecord>(this);
+	}
+
+	@Override
+	public void invoke() throws Exception {
+		try {
+			writer.initializeSerializers();
+			writer.emit(new IntegerRecord(42));
+			writer.emit(new IntegerRecord(1337));
+			writer.flush();
+		}
+		finally {
+			writer.clearBuffers();
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
index 33112af..947a448 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
@@ -34,12 +34,12 @@ import org.apache.flink.api.java.typeutils.runtime.record.RecordSerializerFactor
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.io.network.channels.ChannelType;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.RegularPactTask;
@@ -254,8 +254,8 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 		return modelsInput;
 	}
 
-	private static JobTaskVertex createMapper(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
-		JobTaskVertex pointsInput = JobGraphUtils.createTask(RegularPactTask.class, "Map[DotProducts]", jobGraph, numSubTasks);
+	private static AbstractJobVertex createMapper(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
+		AbstractJobVertex pointsInput = JobGraphUtils.createTask(RegularPactTask.class, "Map[DotProducts]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
@@ -300,7 +300,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	// Unified solution set and workset tail update
 	// -------------------------------------------------------------------------------------------------------------
 
-	private JobGraph createJobGraphV1(String pointsPath, String centersPath, String resultPath, int numSubTasks) throws JobGraphDefinitionException {
+	private JobGraph createJobGraphV1(String pointsPath, String centersPath, String resultPath, int numSubTasks) {
 
 		// -- init -------------------------------------------------------------------------------------------------
 		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
@@ -310,7 +310,7 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 		// -- vertices ---------------------------------------------------------------------------------------------
 		InputFormatVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
 		InputFormatVertex models = createModelsInput(jobGraph, centersPath, numSubTasks, serializer);
-		JobTaskVertex mapper = createMapper(jobGraph, numSubTasks, serializer);
+		AbstractJobVertex mapper = createMapper(jobGraph, numSubTasks, serializer);
 		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
 
 		// -- edges ------------------------------------------------------------------------------------------------
@@ -319,9 +319,13 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 		JobGraphUtils.connect(mapper, output, ChannelType.NETWORK, DistributionPattern.POINTWISE);
 
 		// -- instance sharing -------------------------------------------------------------------------------------
-		points.setVertexToShareInstancesWith(output);
-		models.setVertexToShareInstancesWith(output);
-		mapper.setVertexToShareInstancesWith(output);
+		
+		SlotSharingGroup sharing = new SlotSharingGroup();
+		
+		points.setSlotSharingGroup(sharing);
+		models.setSlotSharingGroup(sharing);
+		mapper.setSlotSharingGroup(sharing);
+		output.setSlotSharingGroup(sharing);
 
 		return jobGraph;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
index 678a7e5..a31539f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -31,10 +31,11 @@ import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.GroupReduceDriver;
@@ -154,8 +155,8 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return output;
 	}
 	
-	private static JobTaskVertex createIterationHead(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
-		JobTaskVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks);
+	private static AbstractJobVertex createIterationHead(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> serializer) {
+		AbstractJobVertex head = JobGraphUtils.createTask(IterationHeadPactTask.class, "Iteration Head", jobGraph, numSubTasks);
 
 		TaskConfig headConfig = new TaskConfig(head.getConfiguration());
 		headConfig.setIterationId(ITERATION_ID);
@@ -188,11 +189,11 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return head;
 	}
 	
-	private static JobTaskVertex createMapper(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> inputSerializer,
+	private static AbstractJobVertex createMapper(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> inputSerializer,
 			TypeSerializerFactory<?> broadcastVarSerializer, TypeSerializerFactory<?> outputSerializer,
 			TypeComparatorFactory<?> outputComparator)
 	{
-		JobTaskVertex mapper = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
+		AbstractJobVertex mapper = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
 			"Map (Select nearest center)", jobGraph, numSubTasks);
 		
 		TaskConfig intermediateConfig = new TaskConfig(mapper.getConfiguration());
@@ -217,12 +218,12 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return mapper;
 	}
 	
-	private static JobTaskVertex createReducer(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> inputSerializer,
+	private static AbstractJobVertex createReducer(JobGraph jobGraph, int numSubTasks, TypeSerializerFactory<?> inputSerializer,
 			TypeComparatorFactory<?> inputComparator, TypeSerializerFactory<?> outputSerializer)
 	{
 		// ---------------- the tail (co group) --------------------
 		
-		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "Reduce / Iteration Tail", jobGraph,
+		AbstractJobVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "Reduce / Iteration Tail", jobGraph,
 			numSubTasks);
 		
 		TaskConfig tailConfig = new TaskConfig(tail.getConfiguration());
@@ -252,8 +253,8 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return tail;
 	}
 	
-	private static OutputFormatVertex createSync(JobGraph jobGraph, int numIterations, int dop) {
-		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, dop);
+	private static AbstractJobVertex createSync(JobGraph jobGraph, int numIterations, int dop) {
+		AbstractJobVertex sync = JobGraphUtils.createSync(jobGraph, dop);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.setIterationId(ITERATION_ID);
@@ -264,7 +265,7 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	// Unified solution set and workset tail update
 	// -------------------------------------------------------------------------------------------------------------
 
-	private static JobGraph createJobGraph(String pointsPath, String centersPath, String resultPath, int numSubTasks, int numIterations) throws JobGraphDefinitionException {
+	private static JobGraph createJobGraph(String pointsPath, String centersPath, String resultPath, int numSubTasks, int numIterations) {
 
 		// -- init -------------------------------------------------------------------------------------------------
 		final TypeSerializerFactory<?> serializer = RecordSerializerFactory.get();
@@ -277,14 +278,14 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		InputFormatVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
 		InputFormatVertex centers = createCentersInput(jobGraph, centersPath, numSubTasks, serializer);
 		
-		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer);
-		JobTaskVertex mapper = createMapper(jobGraph, numSubTasks, serializer, serializer, serializer, int0Comparator);
+		AbstractJobVertex head = createIterationHead(jobGraph, numSubTasks, serializer);
+		AbstractJobVertex mapper = createMapper(jobGraph, numSubTasks, serializer, serializer, serializer, int0Comparator);
 		
-		JobTaskVertex reducer = createReducer(jobGraph, numSubTasks, serializer, int0Comparator, serializer);
+		AbstractJobVertex reducer = createReducer(jobGraph, numSubTasks, serializer, int0Comparator, serializer);
 		
 		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		
-		OutputFormatVertex sync = createSync(jobGraph, numIterations, numSubTasks);
+		AbstractJobVertex sync = createSync(jobGraph, numIterations, numSubTasks);
 		
 		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
index c512525..8bf74c0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancellingTestBase.java
@@ -196,7 +196,6 @@ public abstract class CancellingTestBase {
 						case CANCELED:
 							exitLoop = true;
 							break;
-						case SCHEDULED: // okay
 						case RUNNING:
 							break;
 						default:

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
index 82bd046..2b4b779 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/JobGraphUtils.java
@@ -34,7 +34,6 @@ import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
 import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
 import org.apache.flink.runtime.operators.DataSinkTask;
 import org.apache.flink.runtime.operators.DataSourceTask;
@@ -45,8 +44,8 @@ public class JobGraphUtils {
 
 	public static final long MEGABYTE = 1024l * 1024l;
 
-	private JobGraphUtils() {
-	}
+	private JobGraphUtils() {}
+	
 
 	public static void submit(JobGraph graph, Configuration nepheleConfig) throws IOException, JobExecutionException {
 		JobClient client = new JobClient(graph, nepheleConfig, JobGraphUtils.class.getClassLoader());
@@ -63,10 +62,10 @@ public class JobGraphUtils {
 	private static <T extends InputFormat<?,?>> InputFormatVertex createInput(UserCodeWrapper<T> stub, String name, JobGraph graph,
 			int degreeOfParallelism)
 	{
-		InputFormatVertex inputVertex = new InputFormatVertex(graph, name);
+		InputFormatVertex inputVertex = new InputFormatVertex(name);
+		graph.addVertex(inputVertex);
 		
 		inputVertex.setInvokableClass(DataSourceTask.class);
-		
 		inputVertex.setParallelism(degreeOfParallelism);
 
 		TaskConfig inputConfig = new TaskConfig(inputVertex.getConfiguration());
@@ -83,42 +82,49 @@ public class JobGraphUtils {
 //	}
 	
 	public static void connect(AbstractJobVertex source, AbstractJobVertex target, ChannelType channelType,
-			DistributionPattern distributionPattern) throws JobGraphDefinitionException
+			DistributionPattern distributionPattern)
 	{
-		source.connectTo(target, channelType, distributionPattern);
+		target.connectNewDataSetAsInput(source, distributionPattern);
 	}
 
-	public static JobTaskVertex createTask(@SuppressWarnings("rawtypes") Class<? extends RegularPactTask> task, String name, JobGraph graph,
-			int degreeOfParallelism)
+	@SuppressWarnings("rawtypes") 
+	public static AbstractJobVertex createTask(Class<? extends RegularPactTask> task, String name, JobGraph graph, int parallelism)
 	{
-		JobTaskVertex taskVertex = new JobTaskVertex(name, graph);
+		AbstractJobVertex taskVertex = new AbstractJobVertex(name);
+		graph.addVertex(taskVertex);
+		
 		taskVertex.setInvokableClass(task);
-		taskVertex.setNumberOfSubtasks(degreeOfParallelism);
+		taskVertex.setParallelism(parallelism);
 		return taskVertex;
 	}
 
-	public static OutputFormatVertex createSync(JobGraph jobGraph, int degreeOfParallelism) {
-		OutputFormatVertex sync = new OutputFormatVertex(jobGraph, "BulkIterationSync");
+	public static AbstractJobVertex createSync(JobGraph jobGraph, int parallelism) {
+		AbstractJobVertex sync = new AbstractJobVertex("BulkIterationSync");
+		jobGraph.addVertex(sync);
+		
 		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
 		sync.setParallelism(1);
+		
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, degreeOfParallelism);
+		syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, parallelism);
 		return sync;
 	}
 
-	public static OutputFormatVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
-	{
-		OutputFormatVertex outputVertex = new OutputFormatVertex(jobGraph, name);
+	public static OutputFormatVertex createFakeOutput(JobGraph jobGraph, String name, int degreeOfParallelism) {
+		OutputFormatVertex outputVertex = new OutputFormatVertex(name);
+		jobGraph.addVertex(outputVertex);
+		
 		outputVertex.setInvokableClass(FakeOutputTask.class);
 		outputVertex.setParallelism(degreeOfParallelism);
 		return outputVertex;
 	}
 
-	public static OutputFormatVertex createFileOutput(JobGraph jobGraph, String name, int degreeOfParallelism)
-	{
-		OutputFormatVertex sinkVertex = new OutputFormatVertex(jobGraph, name);
+	public static OutputFormatVertex createFileOutput(JobGraph jobGraph, String name, int parallelism) {
+		OutputFormatVertex sinkVertex = new OutputFormatVertex(name);
+		jobGraph.addVertex(sinkVertex);
+		
 		sinkVertex.setInvokableClass(DataSinkTask.class);
-		sinkVertex.setParallelism(degreeOfParallelism);
+		sinkVertex.setParallelism(parallelism);
 		return sinkVertex;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
index c365378..7c79e35 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
@@ -28,14 +28,11 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.io.network.api.RecordReader;
 import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.SimpleInputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.test.util.RecordAPITestBase;
 import org.junit.After;
 
@@ -95,38 +92,44 @@ public class NetworkStackThroughput {
 		}
 
 		private JobGraph createJobGraph(int dataVolumeGb, boolean useForwarder, boolean isSlowSender,
-				boolean isSlowReceiver, int numSubtasks) throws JobGraphDefinitionException {
-
+				boolean isSlowReceiver, int numSubtasks)
+		{
 			JobGraph jobGraph = new JobGraph("Speed Test");
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
 
-			SimpleInputVertex producer = new SimpleInputVertex("Speed Test Producer", jobGraph);
+			AbstractJobVertex producer = new AbstractJobVertex("Speed Test Producer");
+			jobGraph.addVertex(producer);
+			producer.setSlotSharingGroup(sharingGroup);
+			
 			producer.setInvokableClass(SpeedTestProducer.class);
-			producer.setNumberOfSubtasks(numSubtasks);
+			producer.setParallelism(numSubtasks);
 			producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
 			producer.getConfiguration().setBoolean(IS_SLOW_SENDER_CONFIG_KEY, isSlowSender);
 
-			JobTaskVertex forwarder = null;
+			AbstractJobVertex forwarder = null;
 			if (useForwarder) {
-				forwarder = new JobTaskVertex("Speed Test Forwarder", jobGraph);
+				forwarder = new AbstractJobVertex("Speed Test Forwarder");
+				jobGraph.addVertex(forwarder);
+				forwarder.setSlotSharingGroup(sharingGroup);
+				
 				forwarder.setInvokableClass(SpeedTestForwarder.class);
-				forwarder.setNumberOfSubtasks(numSubtasks);
+				forwarder.setParallelism(numSubtasks);
 			}
 
-			SimpleOutputVertex consumer = new SimpleOutputVertex("Speed Test Consumer", jobGraph);
+			AbstractJobVertex consumer = new AbstractJobVertex("Speed Test Consumer");
+			jobGraph.addVertex(consumer);
+			consumer.setSlotSharingGroup(sharingGroup);
+			
 			consumer.setInvokableClass(SpeedTestConsumer.class);
-			consumer.setNumberOfSubtasks(numSubtasks);
+			consumer.setParallelism(numSubtasks);
 			consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);
 
 			if (useForwarder) {
-				producer.connectTo(forwarder, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-				forwarder.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-
-				forwarder.setVertexToShareInstancesWith(producer);
-				consumer.setVertexToShareInstancesWith(producer);
+				forwarder.connectNewDataSetAsInput(producer, DistributionPattern.BIPARTITE);
+				consumer.connectNewDataSetAsInput(forwarder, DistributionPattern.BIPARTITE);
 			}
 			else {
-				producer.connectTo(consumer, ChannelType.NETWORK, DistributionPattern.BIPARTITE);
-				producer.setVertexToShareInstancesWith(consumer);
+				consumer.connectNewDataSetAsInput(producer, DistributionPattern.BIPARTITE);
 			}
 
 			return jobGraph;
@@ -285,9 +288,12 @@ public class NetworkStackThroughput {
 			TestBaseWrapper test = new TestBaseWrapper(config);
 
 			test.startCluster();
-			test.testJob();
-			test.calculateThroughput();
-			test.stopCluster();
+			try {
+				test.testJob();
+				test.calculateThroughput();
+			} finally {
+				test.stopCluster();
+			}
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/caa4ebef/flink-tests/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/resources/logback-test.xml b/flink-tests/src/test/resources/logback-test.xml
index 7c47e0b..ec37329 100644
--- a/flink-tests/src/test/resources/logback-test.xml
+++ b/flink-tests/src/test/resources/logback-test.xml
@@ -23,12 +23,14 @@
         </encoder>
     </appender>
 
-    <root level="WARN">
+    <root level="INFO">
         <appender-ref ref="STDOUT"/>
     </root>
 
+<!-- 
     <logger name="org.apache.flink.test.recordJobs.relational.query1Util.LineItemFilter" level="ERROR"/>
     <logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>
     <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>
+    -->
 </configuration>
\ No newline at end of file


[45/63] [abbrv] git commit: Add proper locality of scheduling tracking to scheduler. Add local scheduling to slot sharing groups.

Posted by se...@apache.org.
Add proper locality of scheduling tracking to scheduler. Add local scheduling to slot sharing groups.


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/b3c30ca3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/b3c30ca3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/b3c30ca3

Branch: refs/heads/master
Commit: b3c30ca3486080e6e4d97f2d0757fe03a969fbe6
Parents: 22d8bf8
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Sep 14 21:23:28 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../flink/runtime/instance/AllocatedSlot.java   |  19 +-
 .../runtime/jobmanager/scheduler/Locality.java  |  37 ++++
 .../runtime/jobmanager/scheduler/Scheduler.java | 212 +++++++++++--------
 .../scheduler/SlotSharingGroupAssignment.java   |  30 ++-
 .../scheduler/SchedulerSlotSharingTest.java     |  73 +++++++
 5 files changed, 280 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3c30ca3/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
index 2f3fa10..36e6553 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
@@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobmanager.scheduler.Locality;
 
 /**
  * An allocated slot is the unit in which resources are allocated on instances.
@@ -55,6 +56,8 @@ public class AllocatedSlot {
 	/** The state of the vertex, only atomically updated */
 	private volatile int status = ALLOCATED_AND_ALIVE;
 	
+	private Locality locality = Locality.UNCONSTRAINED;
+	
 
 	public AllocatedSlot(JobID jobID, Instance instance, int slotNumber) {
 		if (jobID == null || instance == null || slotNumber < 0) {
@@ -89,6 +92,14 @@ public class AllocatedSlot {
 		return executedTask;
 	}
 	
+	public Locality getLocality() {
+		return locality;
+	}
+	
+	public void setLocality(Locality locality) {
+		this.locality = locality;
+	}
+	
 	public boolean setExecutedVertex(Execution executedVertex) {
 		if (executedVertex == null) {
 			throw new NullPointerException();
@@ -143,9 +154,11 @@ public class AllocatedSlot {
 	public void releaseSlot() {
 		// cancel everything, if there is something. since this is atomically status based,
 		// it will not happen twice if another attempt happened before or concurrently
-		cancel();
-		
-		this.instance.returnAllocatedSlot(this);
+		try {
+			cancel();
+		} finally {
+			this.instance.returnAllocatedSlot(this);
+		}
 	}
 	
 	protected boolean markReleased() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3c30ca3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java
new file mode 100644
index 0000000..c9833d7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Locality.java
@@ -0,0 +1,37 @@
+/**
+ * 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.jobmanager.scheduler;
+
+public enum Locality {
+	
+	/**
+	 * No constraint existed on the task placement.
+	 */
+	UNCONSTRAINED,
+	
+	/**
+	 * The task was scheduled respecting its locality preferences.
+	 */
+	LOCAL,
+	
+	/**
+	 * The task was scheduled to a destination not included in its locality preferences.
+	 */
+	NON_LOCAL
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3c30ca3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index cd57454..cec8fb7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -149,76 +149,76 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 		final ExecutionVertex vertex = task.getTaskToExecute().getVertex();
 	
 		synchronized (globalLock) {
-			// 1) If the task has a strict co-schedule hint, obey it, if it has been assigned.
-//			CoLocationHint hint = task.getCoScheduleHint();
-//			if (hint != null) {
-//				
-//				// try to add to the slot, or make it wait on the hint and schedule the hint itself
-//				if () {
-//					return slot;
-//				}
-//			}
+			// 1)  === If the task has a strict co-schedule hint, obey it ===
+
 		
-			// 2) See if we can place the task somewhere together with another existing task.
-			//    This is defined by the slot sharing groups
+			// 2)  === If the task has a slot sharing group, schedule with shared slots ===
+			
 			SlotSharingGroup sharingUnit = task.getSlotSharingGroup();
 			if (sharingUnit != null) {
-				// see if we can add the task to the current sharing group.
-				SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
-				AllocatedSlot slot = assignment.getSlotForTask(vertex.getJobvertexId(), vertex);
-				if (slot != null) {
-					return slot;
-				}
-			}
-		
-			// 3) We could not schedule it to an existing slot, so we need to get a new one or queue the task
-			
-			// we need potentially to loop multiple times, because there may be false positives
-			// in the set-with-available-instances
-			while (true) {
+				final SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
 				
+				AllocatedSlot newSlot = null;
+				AllocatedSlot slotFromGroup = assignment.getSlotForTask(vertex.getJobvertexId(), vertex, true);
 				
-				Instance instanceToUse = getFreeInstanceForTask(task.getTaskToExecute().getVertex());
-			
-				if (instanceToUse != null) {
-					try {
-						AllocatedSlot slot = instanceToUse.allocateSlot(vertex.getJobId());
-						
-						// if the instance has further available slots, re-add it to the set of available resources.
-						if (instanceToUse.hasResourcesAvailable()) {
-							this.instancesWithAvailableResources.add(instanceToUse);
+				try {
+					if (slotFromGroup != null) {
+						// local (or unconstrained in the current group)
+						if (slotFromGroup.getLocality() != Locality.NON_LOCAL) {
+							updateLocalityCounters(slotFromGroup.getLocality());
+							return slotFromGroup;
 						}
-						
-						if (slot != null) {
-							
-							// if the task is in a shared group, assign the slot to that group
-							// and get a sub slot in turn
-							if (sharingUnit != null) {
-								slot = sharingUnit.getTaskAssignment().addSlotWithTask(slot, task.getJobVertexId());
-							}
-							
-							return slot;
+					}
+					
+					// get another new slot, since we could not place it into the group,
+					// (or we could not place it locally)
+					newSlot = getFreeSlotForTask(vertex);
+					AllocatedSlot toUse;
+					
+					if (newSlot == null) {
+						if (slotFromGroup == null) {
+							throw new NoResourceAvailableException();
+						} else {
+							toUse = slotFromGroup;
 						}
 					}
-					catch (InstanceDiedException e) {
-						// the instance died it has not yet been propagated to this scheduler
-						// remove the instance from the set of available instances
-						this.allInstances.remove(instanceToUse);
-						this.instancesWithAvailableResources.remove(instanceToUse);
+					else if (slotFromGroup == null || newSlot.getLocality() == Locality.LOCAL) {
+						toUse = sharingUnit.getTaskAssignment().addSlotWithTask(newSlot, task.getJobVertexId());
+					} else {
+						toUse = slotFromGroup;
 					}
+					
+					updateLocalityCounters(toUse.getLocality());
+					return toUse;
 				}
-				else {
-					// no resource available now, so queue the request
-					if (queueIfNoResource) {
-						SlotAllocationFuture future = new SlotAllocationFuture();
-						this.taskQueue.add(new QueuedTask(task, future));
-						return future;
+				catch (Throwable t) {
+					if (slotFromGroup != null) {
+						slotFromGroup.releaseSlot();
 					}
-					else {
-						throw new NoResourceAvailableException(task);
+					if (newSlot != null) {
+						newSlot.releaseSlot();
 					}
 				}
 			}
+		
+			// 3) === schedule without hints and sharing ===
+			
+			AllocatedSlot slot = getFreeSlotForTask(vertex);
+			if (slot != null) {
+				updateLocalityCounters(slot.getLocality());
+				return slot;
+			}
+			else {
+				// no resource available now, so queue the request
+				if (queueIfNoResource) {
+					SlotAllocationFuture future = new SlotAllocationFuture();
+					this.taskQueue.add(new QueuedTask(task, future));
+					return future;
+				}
+				else {
+					throw new NoResourceAvailableException(task);
+				}
+			}
 		}
 	}
 		
@@ -230,45 +230,75 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 	 * @param vertex The task to run. 
 	 * @return The instance to run the vertex on, it {@code null}, if no instance is available.
 	 */
-	protected Instance getFreeInstanceForTask(ExecutionVertex vertex) {
-		if (this.instancesWithAvailableResources.isEmpty()) {
-			return null;
-		}
-		
-		Iterable<Instance> locationsIterable = vertex.getPreferredLocations();
-		Iterator<Instance> locations = locationsIterable == null ? null : locationsIterable.iterator();
+	protected AllocatedSlot getFreeSlotForTask(ExecutionVertex vertex) {
 		
-		if (locations != null && locations.hasNext()) {
+		// we need potentially to loop multiple times, because there may be false positives
+		// in the set-with-available-instances
+		while (true) {
+			if (this.instancesWithAvailableResources.isEmpty()) {
+				return null;
+			}
+			
+			Iterable<Instance> locationsIterable = vertex.getPreferredLocations();
+			Iterator<Instance> locations = locationsIterable == null ? null : locationsIterable.iterator();
 			
-			while (locations.hasNext()) {
-				Instance location = locations.next();
+			Instance instanceToUse = null;
+			Locality locality = Locality.UNCONSTRAINED;
+			
+			if (locations != null && locations.hasNext()) {
+				// we have a locality preference
 				
-				if (location != null && this.instancesWithAvailableResources.remove(location)) {
+				while (locations.hasNext()) {
+					Instance location = locations.next();
 					
-					localizedAssignments++;
+					if (location != null && this.instancesWithAvailableResources.remove(location)) {
+						instanceToUse = location;
+						locality = Locality.LOCAL;
+						
+						if (LOG.isDebugEnabled()) {
+							LOG.debug("Local assignment: " + vertex.getSimpleName() + " --> " + location);
+						}
+						
+						break;
+					}
+				}
+				
+				if (instanceToUse == null) {					
+					instanceToUse = this.instancesWithAvailableResources.poll();
+					locality = Locality.NON_LOCAL;
 					if (LOG.isDebugEnabled()) {
-						LOG.debug("Local assignment: " + vertex.getSimpleName() + " --> " + location);
+						LOG.debug("Non-local assignment: " + vertex.getSimpleName() + " --> " + instanceToUse);
 					}
-					
-					return location;
+				}
+			}
+			else {
+				instanceToUse = this.instancesWithAvailableResources.poll();
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Unconstrained assignment: " + vertex.getSimpleName() + " --> " + instanceToUse);
 				}
 			}
 			
-			Instance instance = this.instancesWithAvailableResources.poll();
-			nonLocalizedAssignments++;
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Non-local assignment: " + vertex.getSimpleName() + " --> " + instance);
+			try {
+				AllocatedSlot slot = instanceToUse.allocateSlot(vertex.getJobId());
+				
+				// if the instance has further available slots, re-add it to the set of available resources.
+				if (instanceToUse.hasResourcesAvailable()) {
+					this.instancesWithAvailableResources.add(instanceToUse);
+				}
+				
+				if (slot != null) {
+					slot.setLocality(locality);
+					return slot;
+				}
 			}
-			return instance;
-		}
-		else {
-			Instance instance = this.instancesWithAvailableResources.poll();
-			unconstrainedAssignments++;
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Unconstrained assignment: " + vertex.getSimpleName() + " --> " + instance);
+			catch (InstanceDiedException e) {
+				// the instance died it has not yet been propagated to this scheduler
+				// remove the instance from the set of available instances
+				this.allInstances.remove(instanceToUse);
+				this.instancesWithAvailableResources.remove(instanceToUse);
 			}
 			
-			return instance;
+			// if we failed to get a slot, fall through the loop
 		}
 	}
 	
@@ -339,6 +369,22 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 			}
 		}
 	}
+	
+	private void updateLocalityCounters(Locality locality) {
+		switch (locality) {
+		case UNCONSTRAINED:
+			this.unconstrainedAssignments++;
+			break;
+		case LOCAL:
+			this.localizedAssignments++;
+			break;
+		case NON_LOCAL:
+			this.nonLocalizedAssignments++;
+			break;
+		default:
+			throw new RuntimeException(locality.name());
+		}
+	}
 
 	// --------------------------------------------------------------------------------------------
 	//  Instance Availability

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3c30ca3/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
index 4599d68..290381c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
@@ -55,6 +55,10 @@ public class SlotSharingGroupAssignment {
 			
 			// allocate us a sub slot to return
 			SubSlot subslot = sharedSlot.allocateSubSlot(jid);
+			
+			// preserve the locality information
+			subslot.setLocality(slot.getLocality());
+			
 			boolean entryForNewJidExists = false;
 			
 			// let the other vertex types know about this one as well
@@ -79,9 +83,9 @@ public class SlotSharingGroupAssignment {
 		}
 	}
 	
-	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex vertex) {
+	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex vertex, boolean localOnly) {
 		synchronized (allSlots) {
-			return getSlotForTaskInternal(jid, vertex.getPreferredLocations());
+			return getSlotForTaskInternal(jid, vertex.getPreferredLocations(), localOnly);
 		}
 	}
 	
@@ -143,7 +147,7 @@ public class SlotSharingGroupAssignment {
 	 * @param jid
 	 * @return An allocated sub slot, or {@code null}, if no slot is available.
 	 */
-	private AllocatedSlot getSlotForTaskInternal(JobVertexID jid, Iterable<Instance> preferredLocations) {
+	private AllocatedSlot getSlotForTaskInternal(JobVertexID jid, Iterable<Instance> preferredLocations, boolean localOnly) {
 		if (allSlots.isEmpty()) {
 			return null;
 		}
@@ -165,19 +169,35 @@ public class SlotSharingGroupAssignment {
 		}
 		
 		// check whether we can schedule the task to a preferred location
+		boolean didNotGetPreferred = false;
+		
 		if (preferredLocations != null) {
 			for (Instance location : preferredLocations) {
+				
+				// set the flag that we failed a preferred location. If one will be found,
+				// we return early anyways and skip the flag evaluation
+				didNotGetPreferred = true;
+				
 				SharedSlot slot = removeFromMultiMap(slotsForJid, location);
 				if (slot != null) {
-					return slot.allocateSubSlot(jid);
+					SubSlot subslot = slot.allocateSubSlot(jid);
+					subslot.setLocality(Locality.LOCAL);
+					return subslot;
 				}
 			}
 		}
 		
+		// if we want only local assignments, exit now with a "not found" result
+		if (didNotGetPreferred && localOnly) {
+			return null;
+		}
+		
 		// schedule the task to any available location
 		SharedSlot slot = pollFromMultiMap(slotsForJid);
 		if (slot != null) {
-			return slot.allocateSubSlot(jid);
+			SubSlot subslot = slot.allocateSubSlot(jid);
+			subslot.setLocality(didNotGetPreferred ? Locality.NON_LOCAL : Locality.UNCONSTRAINED);
+			return subslot;
 		}
 		else {
 			return null;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b3c30ca3/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
index c641524..9da993e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
@@ -109,6 +109,11 @@ public class SchedulerSlotSharingTest {
 			
 			// test that everything is released
 			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(8, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -224,6 +229,11 @@ public class SchedulerSlotSharingTest {
 			// test that everything is released
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
 			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(10, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -330,6 +340,11 @@ public class SchedulerSlotSharingTest {
 			
 			// test that everything is released
 			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(15, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -374,6 +389,11 @@ public class SchedulerSlotSharingTest {
 			assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfSlots());
 			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
 			assertEquals(1, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(4, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -514,6 +534,11 @@ public class SchedulerSlotSharingTest {
 			
 			// test that everything is released
 			assertEquals(5, scheduler.getNumberOfAvailableSlots());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(15, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -562,6 +587,8 @@ public class SchedulerSlotSharingTest {
 			
 			// check the scheduler's bookkeeping
 			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -610,6 +637,8 @@ public class SchedulerSlotSharingTest {
 			
 			// check the scheduler's bookkeeping
 			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();
@@ -623,7 +652,51 @@ public class SchedulerSlotSharingTest {
 	@Test
 	public void testLocalizedAssignment3() {
 		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			
+			Scheduler scheduler = new Scheduler();
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			
+			// schedule until the one instance is full
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 4, i1), sharingGroup));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 4, i1), sharingGroup));
+
+			// schedule two more with preference of same instance --> need to go to other instance
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 3, 4, i1), sharingGroup));
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 4, 4, i1), sharingGroup));
+			
+			assertNotNull(s1);
+			assertNotNull(s2);
+			assertNotNull(s3);
+			assertNotNull(s4);
+			assertNotNull(s5);
+			assertNotNull(s6);
+			
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			
+			assertEquals(0, i1.getNumberOfAvailableSlots());
+			assertEquals(0, i2.getNumberOfAvailableSlots());
+			
+			assertEquals(i1, s1.getInstance());
+			assertEquals(i1, s2.getInstance());
+			assertEquals(i1, s3.getInstance());
+			assertEquals(i1, s4.getInstance());
+			assertEquals(i2, s5.getInstance());
+			assertEquals(i2, s6.getInstance());
 			
+			// check the scheduler's bookkeeping
+			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(2, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
 		}
 		catch (Exception e) {
 			e.printStackTrace();


[11/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
index 09a6f5b..d703b4e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
@@ -1,59 +1,259 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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 java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
+import org.apache.commons.logging.Log;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
+import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
+import org.apache.flink.util.StringUtils;
+
+import static org.apache.flink.runtime.execution.ExecutionState2.*;
 
+/**
+ * 
+ * NOTE ABOUT THE DESIGN RATIONAL:
+ * 
+ * In several points of the code, we need to deal with possible concurrent state changes and actions.
+ * For example, while the call to deploy a task (send it to the TaskManager) happens, the task gets cancelled.
+ * 
+ * We could lock the entire portion of the code (decision to deploy, deploy, set state to running) such that
+ * it is guaranteed that any "cancel command" will only pick up after deployment is done and that the "cancel
+ * command" call will never overtake the deploying call.
+ * 
+ * This blocks the threads big time, because the remote calls may take long. Depending of their locking behavior, it
+ * may even result in distributed deadlocks (unless carefully avoided). We therefore use atomic state updates and
+ * occasional double-checking to ensure that the state after a completed call is as expected, and trigger correcting
+ * actions if it is not. Many actions are also idempotent (like canceling).
+ */
 public class ExecutionVertex2 {
+	
+	private static final AtomicReferenceFieldUpdater<ExecutionVertex2, ExecutionState2> STATE_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, ExecutionState2.class, "state");
+	
+	private static final AtomicReferenceFieldUpdater<ExecutionVertex2, AllocatedSlot> ASSIGNED_SLOT_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(ExecutionVertex2.class, AllocatedSlot.class, "assignedSlot");
 
-	private final JobVertexID jobVertexId;
+	private static final Log LOG = ExecutionGraph.LOG;
 	
-			
-			
-	public ExecutionVertex2() {
-		this(new JobVertexID());
-	}
+	private static final int NUM_CANCEL_CALL_TRIES = 3;
+	
+	// --------------------------------------------------------------------------------------------
 	
-	public ExecutionVertex2(JobVertexID jobVertexId) {
-		this.jobVertexId = jobVertexId;
+	private final ExecutionJobVertex jobVertex;
+	
+	private final IntermediateResultPartition[] resultPartitions;
+	
+	private final ExecutionEdge2[][] inputEdges;
+	
+	private final int subTaskIndex;
+	
+	
+	private volatile ExecutionState2 state = CREATED;
+	
+	private volatile AllocatedSlot assignedSlot;
+	
+	private volatile Throwable failureCause;
+	
+	
+	public ExecutionVertex2(ExecutionJobVertex jobVertex, int subTaskIndex, IntermediateResult[] producedDataSets) {
+		this.jobVertex = jobVertex;
+		this.subTaskIndex = subTaskIndex;
+		
+		this.resultPartitions = new IntermediateResultPartition[producedDataSets.length];
+		for (int i = 0; i < producedDataSets.length; i++) {
+			IntermediateResultPartition irp = new IntermediateResultPartition(producedDataSets[i], this, subTaskIndex);
+			this.resultPartitions[i] = irp;
+			producedDataSets[i].setPartition(subTaskIndex, irp);
+		}
+		
+		this.inputEdges = new ExecutionEdge2[jobVertex.getJobVertex().getInputs().size()][];
 	}
 	
 	
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
 	
 	public JobID getJobId() {
-		return new JobID();
+		return this.jobVertex.getJobId();
 	}
 	
-	
 	public JobVertexID getJobvertexId() {
-		return this.jobVertexId;
+		return this.jobVertex.getJobVertexId();
 	}
 	
 	public String getTaskName() {
-		return "task";
+		return this.jobVertex.getJobVertex().getName();
 	}
 	
 	public int getTotalNumberOfParallelSubtasks() {
-		return 1;
+		return this.jobVertex.getParallelism();
 	}
 	
 	public int getParallelSubtaskIndex() {
-		return 0;
+		return this.subTaskIndex;
+	}
+	
+	public int getNumberOfInputs() {
+		return this.inputEdges.length;
+	}
+	
+	public ExecutionEdge2[] getInputEdges(int input) {
+		if (input < 0 || input >= this.inputEdges.length) {
+			throw new IllegalArgumentException(String.format("Input %d is out of range [0..%d)", input, this.inputEdges.length));
+		}
+		return inputEdges[input];
+	}
+	
+	public ExecutionState2 getExecutionState() {
+		return state;
+	}
+	
+	public Throwable getFailureCause() {
+		return failureCause;
+	}
+	
+	public AllocatedSlot getAssignedResource() {
+		return assignedSlot;
+	}
+	
+	private ExecutionGraph getExecutionGraph() {
+		return this.jobVertex.getGraph();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Graph building
+	// --------------------------------------------------------------------------------------------
+	
+	public void connectSource(int inputNumber, IntermediateResult source, JobEdge edge, int consumerNumber) {
+		
+		final DistributionPattern pattern = edge.getDistributionPattern();
+		final IntermediateResultPartition[] sourcePartitions = source.getPartitions();
+		
+		ExecutionEdge2[] edges = null;
+		
+		switch (pattern) {
+			case POINTWISE:
+				edges = connectPointwise(sourcePartitions, inputNumber);
+				break;
+				
+			case BIPARTITE: 
+				edges = connectAllToAll(sourcePartitions, inputNumber);
+				break;
+				
+			default:
+				throw new RuntimeException("Unrecognized distribution pattern.");
+		
+		}
+		
+		this.inputEdges[inputNumber] = edges;
+		
+		// add the cousumers to the source
+		for (ExecutionEdge2 ee : edges) {
+			ee.getSource().addConsumer(ee, consumerNumber);
+		}
+	}
+	
+	private ExecutionEdge2[] connectAllToAll(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
+		ExecutionEdge2[] edges = new ExecutionEdge2[sourcePartitions.length];
+		
+		for (int i = 0; i < sourcePartitions.length; i++) {
+			IntermediateResultPartition irp = sourcePartitions[i];
+			edges[i] = new ExecutionEdge2(irp, this, inputNumber);
+		}
+		
+		return edges;
+	}
+	
+	private ExecutionEdge2[] connectPointwise(IntermediateResultPartition[] sourcePartitions, int inputNumber) {
+		final int numSources = sourcePartitions.length;
+		final int parallelism = getTotalNumberOfParallelSubtasks();
+		
+		// simple case same number of sources as targets
+		if (numSources == parallelism) {
+			return new ExecutionEdge2[] { new ExecutionEdge2(sourcePartitions[subTaskIndex], this, inputNumber) };
+		}
+		else if (numSources < parallelism) {
+			
+			int sourcePartition;
+			
+			// check if the pattern is regular or irregular
+			// we use int arithmetics for regular, and floating point with rounding for irregular
+			if (parallelism % numSources == 0) {
+				// same number of targets per source
+				int factor = parallelism / numSources;
+				sourcePartition = subTaskIndex / factor;
+			}
+			else {
+				// different number of targets per source
+				float factor = ((float) parallelism) / numSources;
+				sourcePartition = (int) (subTaskIndex / factor);
+			}
+			
+			return new ExecutionEdge2[] { new ExecutionEdge2(sourcePartitions[sourcePartition], this, inputNumber) };
+		}
+		else {
+			if (numSources % parallelism == 0) {
+				// same number of targets per source
+				int factor = numSources / parallelism;
+				int startIndex = subTaskIndex * factor;
+				
+				ExecutionEdge2[] edges = new ExecutionEdge2[factor];
+				for (int i = 0; i < factor; i++) {
+					edges[i] = new ExecutionEdge2(sourcePartitions[startIndex + i], this, inputNumber);
+				}
+				return edges;
+			}
+			else {
+				float factor = ((float) numSources) / parallelism;
+				
+				int start = (int) (subTaskIndex * factor);
+				int end = (subTaskIndex == getTotalNumberOfParallelSubtasks() - 1) ?
+						sourcePartitions.length : 
+						(int) ((subTaskIndex + 1) * factor);
+				
+				ExecutionEdge2[] edges = new ExecutionEdge2[end - start];
+				for (int i = 0; i < edges.length; i++) {
+					edges[i] = new ExecutionEdge2(sourcePartitions[start + i], this, inputNumber);
+				}
+				
+				return edges;
+			}
+		}
 	}
 
 	
@@ -61,15 +261,398 @@ public class ExecutionVertex2 {
 	//  Scheduling
 	// --------------------------------------------------------------------------------------------
 	
+	/**
+	 * NOTE: This method only throws exceptions if it is in an illegal state to be scheduled, or if the tasks needs
+	 *       to be scheduled immediately and no resource is available. If the task is accepted by the schedule, any
+	 *       error sets the vertex state to failed and triggers the recovery logic.
+	 * 
+	 * @param scheduler
+	 * 
+	 * @throws IllegalStateException Thrown, if the vertex is not in CREATED state, which is the only state that permits scheduling.
+	 * @throws NoResourceAvailableException Thrown is no queued scheduling is allowed and no resources are currently available.
+	 */
+	public void scheduleForExecution(DefaultScheduler scheduler) throws NoResourceAvailableException {
+		if (scheduler == null) {
+			throw new NullPointerException();
+		}
+		
+		if (STATE_UPDATER.compareAndSet(this, CREATED, SCHEDULED)) {
+			
+			getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, SCHEDULED, null);
+			
+			ScheduledUnit toSchedule = new ScheduledUnit(this, jobVertex.getSlotSharingGroup());
+		
+			// IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned
+			//     in all cases where the deployment failed. we use many try {} finally {} clauses to assure that
+			
+			boolean queued = jobVertex.getGraph().isQueuedSchedulingAllowed();
+			if (queued) {
+				SlotAllocationFuture future = scheduler.scheduleQueued(toSchedule);
+				
+				future.setFutureAction(new SlotAllocationFutureAction() {
+					@Override
+					public void slotAllocated(AllocatedSlot slot) {
+						try {
+							deployToSlot(slot);
+						}
+						catch (Throwable t) {
+							try {
+								slot.releaseSlot();
+							} finally {
+								fail(t);
+							}
+						}
+					}
+				});
+			}
+			else {
+				AllocatedSlot slot = scheduler.scheduleImmediately(toSchedule);
+				try {
+					deployToSlot(slot);
+				}
+				catch (Throwable t) {
+					try {
+						slot.releaseSlot();
+					} finally {
+						fail(t);
+					}
+				}
+			}
+		}
+		else if (this.state == CANCELED) {
+			// this can occur very rarely through heavy races. if the task was canceled, we do not
+			// schedule it
+			return;
+		}
+		else {
+			throw new IllegalStateException("The vertex must be in CREATED state to be scheduled.");
+		}
+	}
+	
+
+	public void deployToSlot(final AllocatedSlot slot) throws JobException {
+		// sanity checks
+		if (slot == null) {
+			throw new NullPointerException();
+		}
+		if (!slot.isAlive()) {
+			throw new IllegalArgumentException("Cannot deploy to a slot that is not alive.");
+		}
+		
+		// make sure exactly one deployment call happens from the correct state
+		// note: the transition from CREATED to DEPLOYING is for testing purposes only
+		ExecutionState2 previous = this.state;
+		if (previous == SCHEDULED || previous == CREATED) {
+			if (!STATE_UPDATER.compareAndSet(this, previous, DEPLOYING)) {
+				// race condition, someone else beat us to the deploying call.
+				// this should actually not happen and indicates a race somewhere else
+				throw new IllegalStateException("Cannot deploy task: Concurrent deployment call race.");
+			}
+			
+			getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, DEPLOYING, null);
+		}
+		else {
+			// vertex may have been cancelled, or it was already scheduled
+			throw new IllegalStateException("The vertex must be in CREATED or SCHEDULED state to be deployed. Found state " + previous);
+		}
+		
+		// good, we are allowed to deploy
+		if (!slot.setExecutedVertex(this)) {
+			throw new JobException("Could not assign the ExecutionVertex to the slot " + slot);
+		}
+		setAssignedSlot(slot);
+		
+		
+		final TaskDeploymentDescriptor deployment = createDeploymentDescriptor();
+		
+		// we execute the actual deploy call in a concurrent action to prevent this call from blocking for long
+		Runnable deployaction = new Runnable() {
+
+			@Override
+			public void run() {
+				try {
+					Instance instance = slot.getInstance();
+					instance.checkLibraryAvailability(getJobId());
+					
+					TaskOperationResult result = instance.getTaskManagerProxy().submitTask(deployment);
+					if (result.isSuccess()) {
+						switchToRunning();
+					}
+					else {
+						// deployment failed :(
+						fail(new Exception("Failed to deploy the tast to slot " + slot + ": " + result.getDescription()));
+					}
+				}
+				catch (Throwable t) {
+					// some error occurred. fail the task
+					fail(t);
+				}
+			}
+		};
+		
+		execute(deployaction);
+	}
+	
+	private void switchToRunning() {
+		
+		// transition state
+		if (STATE_UPDATER.compareAndSet(ExecutionVertex2.this, DEPLOYING, RUNNING)) {
+			
+			getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, RUNNING, null);
+			
+			this.jobVertex.vertexSwitchedToRunning(subTaskIndex);
+		}
+		else {
+			// something happened while the call was in progress.
+			// typically, that means canceling while deployment was in progress
+			
+			ExecutionState2 currentState = ExecutionVertex2.this.state;
+			
+			if (currentState == CANCELING) {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Concurrent canceling of task %s while deployment was in progress.", ExecutionVertex2.this.toString()));
+				}
+				
+				sendCancelRpcCall();
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Concurrent unexpected state transition of task %s while deployment was in progress.", ExecutionVertex2.this.toString()));
+				}
+				
+				// undo the deployment
+				sendCancelRpcCall();
+				
+				// record the failure
+				fail(new Exception("Asynchronous state error. Execution Vertex switched to " + currentState + " while deployment was in progress."));
+			}
+		}
+	}
+	
+	public void cancel() {
+		// depending on the previous state, we go directly to cancelled (no cancel call necessary)
+		// -- or to canceling (cancel call needs to be sent to the task manager)
+		
+		// because of several possibly previous states, we need to again loop until we make a
+		// successful atomic state transition
+		while (true) {
+			
+			ExecutionState2 current = this.state;
+			
+			if (current == CANCELING || current == CANCELED) {
+				// already taken care of, no need to cancel again
+				return;
+			}
+				
+			// these two are the common cases where we need to send a cancel call
+			else if (current == RUNNING || current == DEPLOYING) {
+				// try to transition to canceling, if successful, send the cancel call
+				if (STATE_UPDATER.compareAndSet(this, current, CANCELING)) {
+					
+					getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, CANCELING, null);
+					
+					sendCancelRpcCall();
+					return;
+				}
+				// else: fall through the loop
+			}
+			
+			else if (current == FINISHED || current == FAILED) {
+				// nothing to do any more. finished failed before it could be cancelled.
+				// in any case, the task is removed from the TaskManager already
+				return;
+			}
+			else if (current == CREATED || current == SCHEDULED) {
+				// from here, we can directly switch to cancelled, because the no task has been deployed
+				if (STATE_UPDATER.compareAndSet(this, current, CANCELED)) {
+					
+					getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, CANCELED, null);
+					
+					return;
+				}
+				// else: fall through the loop
+			}
+			else {
+				throw new IllegalStateException(current.name());
+			}
+		}
+	}
+	
+	public void fail(Throwable t) {
+		
+		// damn, we failed. This means only that we keep our books and notify our parent JobExecutionVertex
+		// the actual computation on the task manager is cleaned up by the taskmanager that noticed the failure
+		
+		// we may need to loop multiple times (in the presence of concurrent calls) in order to
+		// atomically switch to failed 
+		while (true) {
+			ExecutionState2 current = this.state;
+			
+			if (current == FAILED) {
+				// concurrently set to failed. It is enough to remember once that we failed (its sad enough)
+				return;
+			}
+			
+			if (current == CANCELED) {
+				// we already aborting
+				if (LOG.isDebugEnabled()) {
+					LOG.debug(String.format("Ignoring transition of vertex %s to %s while being %s",
+							getSimpleName(), FAILED, current));
+				}
+				return;
+			}
+			
+			// we should be in DEPLOYING or RUNNING when a regular failure happens
+			if (current != DEPLOYING && current != RUNNING && current != CANCELING) {
+				// this should not happen. still, what else to do but to comply and go to the FAILED state
+				// at least we should complain loudly to the log
+				LOG.error(String.format("Vertex %s unexpectedly went from state %s to %s with error: %s",
+						getSimpleName(), CREATED, FAILED, t.getMessage()), t);
+			}
+			
+			if (STATE_UPDATER.compareAndSet(this, current, FAILED)) {
+				// success (in a manner of speaking)
+				this.failureCause = t;
+				
+				getExecutionGraph().notifyExecutionChange(getJobvertexId(), subTaskIndex, FAILED, StringUtils.stringifyException(t));
+				
+				// release the slot (concurrency safe)
+				setAssignedSlot(null);
+				
+				this.jobVertex.vertexFailed(subTaskIndex);
+				
+				// leave the loop
+				return;
+			}
+		}
+	}
+	
+	private void sendCancelRpcCall() {
+		// first of all, copy a reference to the stack. any concurrent change to the
+		// field does not affect us now
+		final AllocatedSlot slot = this.assignedSlot;
+		if (slot == null) {
+			throw new IllegalStateException("Cannot cancel when task was not running or deployed.");
+		}
+		
+		Runnable cancelAction = new Runnable() {
+			
+			@Override
+			public void run() {
+				Throwable exception = null;
+				
+				for (int triesLeft = NUM_CANCEL_CALL_TRIES; triesLeft > 0; --triesLeft) {
+					
+					try {
+						// send the call. it may be that the task is not really there (asynchronous / overtaking messages)
+						// in which case it is fine (the deployer catches it)
+						TaskOperationResult result = slot.getInstance().getTaskManagerProxy().cancelTask(getJobvertexId(), subTaskIndex);
+						
+						if (result.isSuccess()) {
+							
+							// make sure that we release the slot
+							try {
+								// found and canceled
+								if (STATE_UPDATER.compareAndSet(ExecutionVertex2.this, CANCELING, CANCELED)) {
+									// we completed the call. 
+									// release the slot resource and let the parent know we have cancelled
+									ExecutionVertex2.this.jobVertex.vertexCancelled(ExecutionVertex2.this.subTaskIndex);
+								}
+								else {
+									ExecutionState2 foundState = ExecutionVertex2.this.state;
+									// failing in the meantime may happen and is no problem
+									if (foundState != FAILED) {
+										// corner case? log at least
+										LOG.error(String.format("Asynchronous race: Found state %s after successful cancel call.", foundState));
+									}
+									
+								}
+							} finally {
+								slot.releaseSlot();
+							}
+						}
+						else {
+							// the task was not found, which may be when the task concurrently finishes or fails, or
+							// when the cancel call overtakes the deployment call
+							if (LOG.isDebugEnabled()) {
+								LOG.debug("Cancel task call did not find task. Probably cause: Acceptable asynchronous race.");
+							}
+						}
+						
+						// in any case, we need not call multiple times, so we quit
+						return;
+					}
+					catch (Throwable t) {
+						if (exception == null) {
+							exception = t;
+						}
+						LOG.error("Canceling vertex " + getSimpleName() + " failed (" + triesLeft + " tries left): " + t.getMessage() , t);
+					}
+				}
+				
+				// dang, utterly unsuccessful - the target node must be down, in which case the tasks are lost anyways
+				fail(new Exception("Task could not be canceled.", exception));
+			}
+		};
+		
+		execute(cancelAction);
+	}
+	
 	public Iterable<Instance> getPreferredLocations() {
 		return null;
 	}
 	
+	private void setAssignedSlot(AllocatedSlot slot) {
+		
+		while (true) {
+			AllocatedSlot previous = this.assignedSlot;
+			if (ASSIGNED_SLOT_UPDATER.compareAndSet(this, previous, slot)) {
+				// successfully swapped
+				// release the predecessor, if it was not null. this call is idempotent, so it does not matter if it is
+				// called more than once
+				try {
+					if (previous != null) {
+						previous.releaseSlot();
+					}
+				} catch (Throwable t) {
+					LOG.debug("Error releasing slot " + slot, t);
+				}
+				return;
+			}
+		}
+	}
+	
+	
+	private TaskDeploymentDescriptor createDeploymentDescriptor() {
+		//  create the input gate deployment descriptors
+		List<GateDeploymentDescriptor> inputGates = new ArrayList<GateDeploymentDescriptor>(inputEdges.length);
+		for (ExecutionEdge2[] channels : inputEdges) {
+			inputGates.add(GateDeploymentDescriptor.fromEdges(channels));
+		}
+		
+		// create the output gate deployment descriptors
+		List<GateDeploymentDescriptor> outputGates = new ArrayList<GateDeploymentDescriptor>(resultPartitions.length);
+		for (IntermediateResultPartition partition : resultPartitions) {
+			for (List<ExecutionEdge2> channels : partition.getConsumers()) {
+				outputGates.add(GateDeploymentDescriptor.fromEdges(channels));
+			}
+		}
+		
+		String[] jarFiles = getExecutionGraph().getUserCodeJarFiles();
+		
+		return new TaskDeploymentDescriptor(getJobId(), getJobvertexId(), getTaskName(), 
+				subTaskIndex, getTotalNumberOfParallelSubtasks(), 
+				getExecutionGraph().getJobConfiguration(), jobVertex.getJobVertex().getConfiguration(),
+				jobVertex.getJobVertex().getInvokableClassName(), outputGates, inputGates, jarFiles);
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	//  Utilities
 	// --------------------------------------------------------------------------------------------
 	
+	public void execute(Runnable action) {
+		this.jobVertex.execute(action);
+	}
+	
 	/**
 	 * Creates a simple name representation in the style 'taskname (x/y)', where
 	 * 'taskname' is the name as returned by {@link #getTaskName()}, 'x' is the parallel
@@ -81,4 +664,9 @@ public class ExecutionVertex2 {
 	public String getSimpleName() {
 		return getTaskName() + " (" + (getParallelSubtaskIndex()+1) + '/' + getTotalNumberOfParallelSubtasks() + ')';
 	}
+	
+	@Override
+	public String toString() {
+		return getSimpleName() + " [" + state + ']';
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java
deleted file mode 100644
index 87e2120..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/**
- * 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 org.apache.flink.runtime.AbstractID;
-import org.apache.flink.runtime.managementgraph.ManagementVertexID;
-
-/**
- * A class for statistically unique execution vertex IDs.
- * 
- */
-public class ExecutionVertexID extends AbstractID {
-
-	private static final long serialVersionUID = 1L;
-	
-	/**
-	 * Creates a new random execution vertex id.
-	 */
-	public ExecutionVertexID() {
-		super();
-	}
-	
-	/**
-	 * Creates a new execution vertex id, equal to the given id.
-	 * 
-	 * @param from The id to copy.
-	 */
-	public ExecutionVertexID(AbstractID from) {
-		super(from);
-	}
-	
-	/**
-	 * Converts the execution vertex ID into a
-	 * management vertex ID. The new management vertex ID
-	 * will be equal to the execution vertex ID in the sense
-	 * that the <code>equals</code> method will return <code>
-	 * true</code> when both IDs are compared.
-	 * 
-	 * @return the new management vertex ID
-	 */
-	public ManagementVertexID toManagementVertexID() {
-		return new ManagementVertexID(this);
-	}
-
-	/**
-	 * Converts the given management vertex ID into the corresponding execution vertex ID. The new execution vertex ID
-	 * will be equals to the management vertex ID in the sense that the <code>equals</code> method will return
-	 * <code>true</code> when both IDs are compared.
-	 * 
-	 * @param vertexID
-	 *        the management vertex ID to be converted
-	 * @return the resulting execution vertex ID
-	 */
-	public static ExecutionVertexID fromManagementVertexID(ManagementVertexID vertexID) {
-		return new ExecutionVertexID(vertexID);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/GraphConversionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/GraphConversionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/GraphConversionException.java
deleted file mode 100644
index 75e2e95..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/GraphConversionException.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * 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;
-
-/**
- * A graph conversion exception is thrown if the creation of transformation
- * of an {@link ExecutionGraph} fails.
- * 
- */
-public class GraphConversionException extends Exception {
-
-	/**
-	 * Generated serial version UID.
-	 */
-	private static final long serialVersionUID = -7623370680208569211L;
-
-	/**
-	 * Creates a new exception with the given error message.
-	 * 
-	 * @param msg
-	 *        the error message to be transported through this exception
-	 */
-	public GraphConversionException(String msg) {
-		super(msg);
-	}
-
-	public GraphConversionException(String message, Throwable cause) {
-		super(message, cause);
-	}
-
-	public GraphConversionException(Throwable cause) {
-		super(cause);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
new file mode 100644
index 0000000..540996f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResult.java
@@ -0,0 +1,82 @@
+/**
+ * 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 org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+
+public class IntermediateResult {
+
+	private final IntermediateDataSetID id;
+	
+	private final ExecutionJobVertex producer;
+	
+	private final IntermediateResultPartition[] partitions;
+	
+	private final int numParallelProducers;
+	
+	private int partitionsAssigned;
+	
+	private int numConsumers;
+	
+	
+	public IntermediateResult(IntermediateDataSetID id, ExecutionJobVertex producer, int numParallelProducers) {
+		this.id = id;
+		this.producer = producer;
+		this.partitions = new IntermediateResultPartition[numParallelProducers];
+		this.numParallelProducers = numParallelProducers;
+		
+		// we do not set the intermediate result partitions here, because we let them be initialized by
+		// the execution vertex that produces them
+	}
+	
+	public void setPartition(int partitionNumber, IntermediateResultPartition partition) {
+		if (partition == null || partitionNumber < 0 || partitionNumber >= numParallelProducers) {
+			throw new IllegalArgumentException();
+		}
+		
+		if (partitions[partitionNumber] != null) {
+			throw new IllegalStateException("Partition #" + partitionNumber + " has already been assigned.");
+		}
+		
+		partitions[partitionNumber] = partition;
+		partitionsAssigned++;
+	}
+	
+	
+	
+	public IntermediateDataSetID getId() {
+		return id;
+	}
+	
+	public IntermediateResultPartition[] getPartitions() {
+		return partitions;
+	}
+	
+	public int registerConsumer() {
+		final int index = numConsumers;
+		numConsumers++;
+		
+		for (IntermediateResultPartition p : partitions) {
+			if (p.addConsumerGroup() != index) {
+				throw new RuntimeException("Inconsistent consumer mapping between intermediate result partitions.");
+			}
+		}
+		return index;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
new file mode 100644
index 0000000..13bb930
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/IntermediateResultPartition.java
@@ -0,0 +1,68 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+public class IntermediateResultPartition {
+	
+	private final IntermediateResult totalResut;
+	
+	private final ExecutionVertex2 producer;
+	
+	private final int partition;
+	
+	private List<List<ExecutionEdge2>> consumers;
+	
+	
+	public IntermediateResultPartition(IntermediateResult totalResut, ExecutionVertex2 producer, int partition) {
+		this.totalResut = totalResut;
+		this.producer = producer;
+		this.partition = partition;
+		this.consumers = new ArrayList<List<ExecutionEdge2>>(0);
+	}
+	
+	public ExecutionVertex2 getProducer() {
+		return producer;
+	}
+	
+	public int getPartition() {
+		return partition;
+	}
+	
+	public IntermediateResult getIntermediateResult() {
+		return totalResut;
+	}
+	
+	public List<List<ExecutionEdge2>> getConsumers() {
+		return consumers;
+	}
+	
+	int addConsumerGroup() {
+		int pos = consumers.size();
+		consumers.add(new ArrayList<ExecutionEdge2>());
+		return pos;
+	}
+	
+	public void addConsumer(ExecutionEdge2 edge, int consumerNumber) {
+		consumers.get(consumerNumber).add(edge);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/InternalJobStatus.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/InternalJobStatus.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/InternalJobStatus.java
deleted file mode 100644
index f7ccda1..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/InternalJobStatus.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * 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 org.apache.flink.runtime.jobgraph.JobStatus;
-
-/**
- * This enumeration contains all states a job represented by an {@link ExecutionGraph} can have during its lifetime. It
- * contains all states from {@link JobStatus} but also internal states to keep track of shutdown processes.
- * <p>
- * This class is thread-safe.
- * 
- */
-public enum InternalJobStatus {
-
-	/**
-	 * All tasks of the job are in the execution state CREATED.
-	 */
-	CREATED,
-
-	/**
-	 * All tasks of the job have been accepted by the scheduler, resources have been requested
-	 */
-	SCHEDULED,
-
-	/**
-	 * At least one task of the job is running, none has definitely failed.
-	 */
-	RUNNING,
-
-	/**
-	 * At least one task of the job has definitely failed and cannot be recovered. The job is in the process of being
-	 * terminated.
-	 */
-	FAILING,
-
-	/**
-	 * At least one task of the job has definitively failed and cannot
-	 * be recovered anymore. As a result, the job has been terminated.
-	 */
-	FAILED,
-
-	/**
-	 * At least one task has been canceled as a result of a user request. The job is in the process of being canceled
-	 * completely.
-	 */
-	CANCELING,
-
-	/**
-	 * All tasks of the job are canceled as a result of a user request. The job has been terminated.
-	 */
-	CANCELED,
-
-	/**
-	 * All of the job's tasks have successfully finished.
-	 */
-	FINISHED;
-
-	/**
-	 * Converts an internal job status in a {@link JobStatus} state.
-	 * 
-	 * @param status
-	 *        the internal job status to converted.
-	 * @return the corresponding job status or <code>null</code> if no corresponding job status exists
-	 */
-	@SuppressWarnings("incomplete-switch")
-	public static JobStatus toJobStatus(InternalJobStatus status) {
-
-		switch (status) {
-
-		case CREATED:
-			return JobStatus.CREATED;
-		case SCHEDULED:
-			return JobStatus.SCHEDULED;
-		case RUNNING:
-			return JobStatus.RUNNING;
-		case FAILED:
-			return JobStatus.FAILED;
-		case CANCELED:
-			return JobStatus.CANCELED;
-		case FINISHED:
-			return JobStatus.FINISHED;
-		}
-
-		return null;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java
index aabed5c..512a381 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/JobStatusListener.java
@@ -16,25 +16,21 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.executiongraph;
 
+import org.apache.flink.runtime.jobgraph.JobStatus;
+
 /**
- * This interface allows objects to receive notifications
- * when the status of an observed job has changed.
- * 
+ * This interface allows objects to receive notifications when the status of an observed job has changed.
  */
 public interface JobStatusListener {
 
-/**
-	 * Called when the status of the job with the given {@li
+	/**
+	 * Called when the status of the job changed.
 	 * 
-	 * @param executionGraph
-	 *        the executionGraph representing the job the event belongs to
-	 * @param newJobStatus
-	 *        the new job status
-	 * @param optionalMessage
-	 *        an optional message (possibly <code>null</code>) that can be attached to the state change
+	 * @param executionGraph   The executionGraph representing the job.
+	 * @param newJobStatus     The new job status.
+	 * @param optionalMessage  An optional message (possibly <code>null</code>) that can be attached to the state change.
 	 */
-	void jobStatusHasChanged(ExecutionGraph executionGraph, InternalJobStatus newJobStatus, String optionalMessage);
+	void jobStatusHasChanged(ExecutionGraph executionGraph, JobStatus newJobStatus, String optionalMessage);
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
index cb7e658..f3f489b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.instance;
 
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
 import org.apache.flink.runtime.jobgraph.JobID;
@@ -30,7 +31,10 @@ public class AllocatedSlot {
 	
 	private static final AtomicIntegerFieldUpdater<AllocatedSlot> STATUS_UPDATER = 
 			AtomicIntegerFieldUpdater.newUpdater(AllocatedSlot.class, "status");
-	 
+	
+	private static final AtomicReferenceFieldUpdater<AllocatedSlot, ExecutionVertex2> VERTEX_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(AllocatedSlot.class, ExecutionVertex2.class, "executedVertex");
+	
 	private static final int ALLOCATED_AND_ALIVE = 0;		// tasks may be added and might be running
 	private static final int CANCELLED = 1;					// no more tasks may run
 	private static final int RELEASED = 2;					// has been given back to the instance
@@ -45,10 +49,13 @@ public class AllocatedSlot {
 	/** The number of the slot on which the task is deployed */
 	private final int slotNumber;
 	
+	/** Vertex being executed in the slot. Volatile to force a memory barrier and allow for correct double-checking */
+	private volatile ExecutionVertex2 executedVertex;
+	
+	/** The state of the vertex, only atomically updated */
 	private volatile int status = ALLOCATED_AND_ALIVE;
 	
 
-
 	public AllocatedSlot(JobID jobID, Instance instance, int slotNumber) {
 		if (jobID == null || instance == null || slotNumber < 0) {
 			throw new IllegalArgumentException();
@@ -78,18 +85,34 @@ public class AllocatedSlot {
 		return slotNumber;
 	}
 	
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * @param vertex
-	 * 
-	 * @return True, if the task was scheduled correctly, false if the slot was asynchronously deallocated
-	 *         in the meantime.
-	 */
-	public boolean runTask(ExecutionVertex2 vertex) {
+	public boolean setExecutedVertex(ExecutionVertex2 executedVertex) {
+		if (executedVertex == null) {
+			throw new NullPointerException();
+		}
+		
+		// check that we can actually run in this slot
+		if (status != ALLOCATED_AND_ALIVE) {
+			return false;
+		}
+		
+		// atomically assign the vertex
+		if (!VERTEX_UPDATER.compareAndSet(this, null, executedVertex)) {
+			return false;
+		}
+
+		// we need to do a double check that we were not cancelled in the meantime
+		if (status != ALLOCATED_AND_ALIVE) {
+			this.executedVertex = null;
+			return false;
+		}
+		
 		return true;
 	}
 	
+	public ExecutionVertex2 getExecutedVertex() {
+		return executedVertex;
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	//  Status and life cycle
 	// --------------------------------------------------------------------------------------------
@@ -110,6 +133,9 @@ public class AllocatedSlot {
 	public void cancel() {
 		if (STATUS_UPDATER.compareAndSet(this, ALLOCATED_AND_ALIVE, CANCELLED)) {
 			// kill all tasks currently running in this slot
+			if (this.executedVertex != null) {
+				this.executedVertex.fail(new Exception("The slot in which the task was scheduled has been cancelled."));
+			}
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java
deleted file mode 100644
index eca23c4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/**
- * 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.instance;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
-import org.apache.flink.configuration.ConfigConstants;
-import org.apache.flink.configuration.GlobalConfiguration;
-
-/**
- * A simple implementation of an {@link InstanceManager}.
- */
-public class DefaultInstanceManager implements InstanceManager {
-
-	private static final Logger LOG = LoggerFactory.getLogger(DefaultInstanceManager.class);
-
-	// ------------------------------------------------------------------------
-	// Fields
-	// ------------------------------------------------------------------------
-
-	/** Global lock */
-	private final Object lock = new Object();
-
-	/** Set of hosts known to run a task manager that are thus able to execute tasks (by ID). */
-	private final Map<InstanceID, Instance> registeredHostsById;
-
-	/** Set of hosts known to run a task manager that are thus able to execute tasks (by connection). */
-	private final Map<InstanceConnectionInfo, Instance> registeredHostsByConnection;
-	
-	/** Set of hosts that were present once and have died */
-	private final Set<InstanceConnectionInfo> deadHosts;
-	
-	/** Listeners that want to be notified about availability and disappearance of instances */
-	private final List<InstanceListener> instanceListeners = new ArrayList<InstanceListener>();
-
-	/** Duration after which a task manager is considered dead if it did not send a heart-beat message. */
-	private final long heartbeatTimeout;
-	
-	/** The total number of task slots that the system has */
-	private int totalNumberOfAliveTaskSlots;
-
-	/** Flag marking the system as shut down */
-	private volatile boolean shutdown;
-
-	// ------------------------------------------------------------------------
-	// Constructor and set-up
-	// ------------------------------------------------------------------------
-	
-	/**
-	 * Creates an instance manager, using the global configuration value for maximum interval between heartbeats
-	 * where a task manager is still considered alive.
-	 */
-	public DefaultInstanceManager() {
-		this(1000 * GlobalConfiguration.getLong(
-				ConfigConstants.JOB_MANAGER_DEAD_TASKMANAGER_TIMEOUT_KEY,
-				ConfigConstants.DEFAULT_JOB_MANAGER_DEAD_TASKMANAGER_TIMEOUT));
-	}
-	
-	public DefaultInstanceManager(long heartbeatTimeout) {
-		this(heartbeatTimeout, heartbeatTimeout);
-	}
-	
-	public DefaultInstanceManager(long heartbeatTimeout, long cleanupInterval) {
-		if (heartbeatTimeout <= 0 || cleanupInterval <= 0) {
-			throw new IllegalArgumentException("Heartbeat timeout and cleanup interval must be positive.");
-		}
-		
-		this.registeredHostsById = new HashMap<InstanceID, Instance>();
-		this.registeredHostsByConnection = new HashMap<InstanceConnectionInfo, Instance>();
-		this.deadHosts = new HashSet<InstanceConnectionInfo>();
-		this.heartbeatTimeout = heartbeatTimeout;
-
-		new Timer(true).schedule(cleanupStaleMachines, cleanupInterval, cleanupInterval);
-	}
-
-	@Override
-	public void shutdown() {
-		synchronized (this.lock) {
-			if (this.shutdown) {
-				return;
-			}
-			this.shutdown = true;
-
-			this.cleanupStaleMachines.cancel();
-
-			for (Instance i : this.registeredHostsById.values()) {
-				i.markDead();
-			}
-			
-			this.registeredHostsById.clear();
-			this.registeredHostsByConnection.clear();
-			this.deadHosts.clear();
-			this.totalNumberOfAliveTaskSlots = 0;
-		}
-	}
-
-	@Override
-	public boolean reportHeartBeat(InstanceID instanceId) {
-		if (instanceId == null) {
-			throw new IllegalArgumentException("InstanceID may not be null.");
-		}
-		
-		synchronized (this.lock) {
-			if (this.shutdown) {
-				throw new IllegalStateException("InstanceManager is shut down.");
-			}
-			
-			Instance host = registeredHostsById.get(instanceId);
-
-			if (host == null){
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Received hearbeat from unknown TaskManager with instance ID " + instanceId.toString() + 
-							" Possibly TaskManager was maked as dead (timed-out) earlier. " +
-							"Reporting back that task manager is no longer known.");
-				}
-				return false;
-			}
-
-			host.reportHeartBeat();
-			return true;
-		}
-	}
-
-	@Override
-	public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription resources, int numberOfSlots){
-		synchronized(this.lock){
-			if (this.shutdown) {
-				throw new IllegalStateException("InstanceManager is shut down.");
-			}
-			
-			Instance prior = registeredHostsByConnection.get(instanceConnectionInfo);
-			if (prior != null) {
-				LOG.error("Registration attempt from TaskManager with connection info " + instanceConnectionInfo + 
-						". This connection is already registered under ID " + prior.getId());
-				return null;
-			}
-			
-			boolean wasDead = this.deadHosts.remove(instanceConnectionInfo);
-			if (wasDead) {
-				LOG.info("Registering TaskManager with connection info " + instanceConnectionInfo + 
-						" which was marked as dead earlier because of a heart-beat timeout.");
-			}
-
-			InstanceID id = null;
-			do {
-				id = new InstanceID();
-			} while (registeredHostsById.containsKey(id));
-			
-			
-			Instance host = new Instance(instanceConnectionInfo, id, resources, numberOfSlots);
-			
-			registeredHostsById.put(id, host);
-			registeredHostsByConnection.put(instanceConnectionInfo, host);
-			
-			totalNumberOfAliveTaskSlots += numberOfSlots;
-			
-			if (LOG.isInfoEnabled()) {
-				LOG.info(String.format("Registered TaskManager at %s as %s. Current number of registered hosts is %d.",
-						instanceConnectionInfo, id, registeredHostsById.size()));
-			}
-
-			host.reportHeartBeat();
-			
-			// notify all listeners (for example the scheduler)
-			notifyNewInstance(host);
-			
-			return id;
-		}
-	}
-
-	@Override
-	public int getNumberOfRegisteredTaskManagers() {
-		return this.registeredHostsById.size();
-	}
-
-	@Override
-	public int getTotalNumberOfSlots() {
-		return this.totalNumberOfAliveTaskSlots;
-	}
-	
-	@Override
-	public Map<InstanceID, Instance> getAllRegisteredInstances() {
-		return this.registeredHostsById;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	public void addInstanceListener(InstanceListener listener) {
-		synchronized (this.instanceListeners) {
-			this.instanceListeners.add(listener);
-		}
-	}
-	
-	public void removeInstanceListener(InstanceListener listener) {
-		synchronized (this.instanceListeners) {
-			this.instanceListeners.remove(listener);
-		}
-	}
-	
-	private void notifyNewInstance(Instance instance) {
-		synchronized (this.instanceListeners) {
-			for (InstanceListener listener : this.instanceListeners) {
-				try {
-					listener.newInstanceAvailable(instance);
-				}
-				catch (Throwable t) {
-					LOG.error("Notification of new instance availability failed.", t);
-				}
-			}
-		}
-	}
-	
-	private void notifyDeadInstance(Instance instance) {
-		synchronized (this.instanceListeners) {
-			for (InstanceListener listener : this.instanceListeners) {
-				try {
-					listener.instanceDied(instance);
-				}
-				catch (Throwable t) {
-					LOG.error("Notification of dead instance failed.", t);
-				}
-			}
-		}
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private void checkForDeadInstances() {
-		final long now = System.currentTimeMillis();
-		final long timeout = DefaultInstanceManager.this.heartbeatTimeout;
-		
-		synchronized (DefaultInstanceManager.this.lock) {
-			if (DefaultInstanceManager.this.shutdown) {
-				return;
-			}
-
-			final Iterator<Map.Entry<InstanceID, Instance>> entries = registeredHostsById.entrySet().iterator();
-			
-			// check all hosts whether they did not send heart-beat messages.
-			while (entries.hasNext()) {
-				
-				final Map.Entry<InstanceID, Instance> entry = entries.next();
-				final Instance host = entry.getValue();
-				
-				if (!host.isStillAlive(now, timeout)) {
-					
-					// remove from the living
-					entries.remove();
-					registeredHostsByConnection.remove(host.getInstanceConnectionInfo());
-					
-					// add to the dead
-					deadHosts.add(host.getInstanceConnectionInfo());
-					
-					host.markDead();
-					
-					totalNumberOfAliveTaskSlots -= host.getTotalNumberOfSlots();
-					
-					LOG.info(String.format("TaskManager %s at %s did not report a heartbeat for %d msecs - marking as dead. Current number of registered hosts is %d.",
-							host.getId(), host.getInstanceConnectionInfo(), heartbeatTimeout, registeredHostsById.size()));
-					
-					// report to all listeners
-					notifyDeadInstance(host);
-				}
-			}
-		}
-	}
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Periodic task that checks whether hosts have not sent their heart-beat
-	 * messages and purges the hosts in this case.
-	 */
-	private final TimerTask cleanupStaleMachines = new TimerTask() {
-		@Override
-		public void run() {
-			try {
-				checkForDeadInstances();
-			}
-			catch (Throwable t) {
-				LOG.error("Checking for dead instances failed.", t);
-			}
-		}
-	};
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
index a168b2c..543ae86 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
@@ -27,6 +27,10 @@ import java.util.List;
 import java.util.Queue;
 import java.util.Set;
 
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileRequest;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileResponse;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheUpdate;
 import org.apache.flink.runtime.ipc.RPC;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailablilityListener;
@@ -186,6 +190,28 @@ public class Instance {
 		}
 	}
 	
+	public void checkLibraryAvailability(JobID jobID) throws IOException {
+		String[] requiredLibraries = LibraryCacheManager.getRequiredJarFiles(jobID);
+
+		if (requiredLibraries == null) {
+			throw new IOException("No entry of required libraries for job " + jobID);
+		}
+
+		LibraryCacheProfileRequest request = new LibraryCacheProfileRequest();
+		request.setRequiredLibraries(requiredLibraries);
+
+		// Send the request
+		LibraryCacheProfileResponse response = getTaskManagerProxy().getLibraryCacheProfile(request);
+
+		// Check response and transfer libraries if necessary
+		for (int k = 0; k < requiredLibraries.length; k++) {
+			if (!response.isCached(k)) {
+				LibraryCacheUpdate update = new LibraryCacheUpdate(requiredLibraries[k]);
+				getTaskManagerProxy().updateLibraryCache(update);
+			}
+		}
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	// Heartbeats
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
index 09c384a..5e139f3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceManager.java
@@ -18,22 +18,289 @@
 
 package org.apache.flink.runtime.instance;
 
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.GlobalConfiguration;
 
 /**
  * Simple manager that keeps track of which TaskManager are available and alive.
  */
-public interface InstanceManager {
+public class InstanceManager {
+
+	private static final Log LOG = LogFactory.getLog(InstanceManager.class);
+
+	// ------------------------------------------------------------------------
+	// Fields
+	// ------------------------------------------------------------------------
+
+	/** Global lock */
+	private final Object lock = new Object();
+
+	/** Set of hosts known to run a task manager that are thus able to execute tasks (by ID). */
+	private final Map<InstanceID, Instance> registeredHostsById;
+
+	/** Set of hosts known to run a task manager that are thus able to execute tasks (by connection). */
+	private final Map<InstanceConnectionInfo, Instance> registeredHostsByConnection;
+	
+	/** Set of hosts that were present once and have died */
+	private final Set<InstanceConnectionInfo> deadHosts;
+	
+	/** Listeners that want to be notified about availability and disappearance of instances */
+	private final List<InstanceListener> instanceListeners = new ArrayList<InstanceListener>();
 
-	InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription resources, int numberOfTaskSlots);
+	/** Duration after which a task manager is considered dead if it did not send a heart-beat message. */
+	private final long heartbeatTimeout;
 	
-	boolean reportHeartBeat(InstanceID instance);
+	/** The total number of task slots that the system has */
+	private int totalNumberOfAliveTaskSlots;
+
+	/** Flag marking the system as shut down */
+	private volatile boolean shutdown;
+
+	// ------------------------------------------------------------------------
+	// Constructor and set-up
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Creates an instance manager, using the global configuration value for maximum interval between heartbeats
+	 * where a task manager is still considered alive.
+	 */
+	public InstanceManager() {
+		this(1000 * GlobalConfiguration.getLong(
+				ConfigConstants.JOB_MANAGER_DEAD_TASKMANAGER_TIMEOUT_KEY,
+				ConfigConstants.DEFAULT_JOB_MANAGER_DEAD_TASKMANAGER_TIMEOUT));
+	}
+	
+	public InstanceManager(long heartbeatTimeout) {
+		this(heartbeatTimeout, heartbeatTimeout);
+	}
+	
+	public InstanceManager(long heartbeatTimeout, long cleanupInterval) {
+		if (heartbeatTimeout <= 0 || cleanupInterval <= 0) {
+			throw new IllegalArgumentException("Heartbeat timeout and cleanup interval must be positive.");
+		}
+		
+		this.registeredHostsById = new HashMap<InstanceID, Instance>();
+		this.registeredHostsByConnection = new HashMap<InstanceConnectionInfo, Instance>();
+		this.deadHosts = new HashSet<InstanceConnectionInfo>();
+		this.heartbeatTimeout = heartbeatTimeout;
+
+		new Timer(true).schedule(cleanupStaleMachines, cleanupInterval, cleanupInterval);
+	}
+
+	public void shutdown() {
+		synchronized (this.lock) {
+			if (this.shutdown) {
+				return;
+			}
+			this.shutdown = true;
+
+			this.cleanupStaleMachines.cancel();
 
-	void shutdown();
+			for (Instance i : this.registeredHostsById.values()) {
+				i.markDead();
+			}
+			
+			this.registeredHostsById.clear();
+			this.registeredHostsByConnection.clear();
+			this.deadHosts.clear();
+			this.totalNumberOfAliveTaskSlots = 0;
+		}
+	}
 
-	Map<InstanceID, Instance> getAllRegisteredInstances();
+	public boolean reportHeartBeat(InstanceID instanceId) {
+		if (instanceId == null) {
+			throw new IllegalArgumentException("InstanceID may not be null.");
+		}
+		
+		synchronized (this.lock) {
+			if (this.shutdown) {
+				throw new IllegalStateException("InstanceManager is shut down.");
+			}
+			
+			Instance host = registeredHostsById.get(instanceId);
 
-	int getNumberOfRegisteredTaskManagers();
+			if (host == null){
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Received hearbeat from unknown TaskManager with instance ID " + instanceId.toString() + 
+							" Possibly TaskManager was maked as dead (timed-out) earlier. " +
+							"Reporting back that task manager is no longer known.");
+				}
+				return false;
+			}
 
-	int getTotalNumberOfSlots();
+			host.reportHeartBeat();
+			return true;
+		}
+	}
+
+	public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription resources, int numberOfSlots){
+		synchronized(this.lock){
+			if (this.shutdown) {
+				throw new IllegalStateException("InstanceManager is shut down.");
+			}
+			
+			Instance prior = registeredHostsByConnection.get(instanceConnectionInfo);
+			if (prior != null) {
+				LOG.error("Registration attempt from TaskManager with connection info " + instanceConnectionInfo + 
+						". This connection is already registered under ID " + prior.getId());
+				return null;
+			}
+			
+			boolean wasDead = this.deadHosts.remove(instanceConnectionInfo);
+			if (wasDead) {
+				LOG.info("Registering TaskManager with connection info " + instanceConnectionInfo + 
+						" which was marked as dead earlier because of a heart-beat timeout.");
+			}
+
+			InstanceID id = null;
+			do {
+				id = new InstanceID();
+			} while (registeredHostsById.containsKey(id));
+			
+			
+			Instance host = new Instance(instanceConnectionInfo, id, resources, numberOfSlots);
+			
+			registeredHostsById.put(id, host);
+			registeredHostsByConnection.put(instanceConnectionInfo, host);
+			
+			totalNumberOfAliveTaskSlots += numberOfSlots;
+			
+			if (LOG.isInfoEnabled()) {
+				LOG.info(String.format("Registered TaskManager at %s as %s. Current number of registered hosts is %d.",
+						instanceConnectionInfo, id, registeredHostsById.size()));
+			}
+
+			host.reportHeartBeat();
+			
+			// notify all listeners (for example the scheduler)
+			notifyNewInstance(host);
+			
+			return id;
+		}
+	}
+
+	public int getNumberOfRegisteredTaskManagers() {
+		return this.registeredHostsById.size();
+	}
+
+	public int getTotalNumberOfSlots() {
+		return this.totalNumberOfAliveTaskSlots;
+	}
+	
+	public Map<InstanceID, Instance> getAllRegisteredInstances() {
+		synchronized (this.lock) {
+			return new HashMap<InstanceID, Instance>(this.registeredHostsById);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void addInstanceListener(InstanceListener listener) {
+		synchronized (this.instanceListeners) {
+			this.instanceListeners.add(listener);
+		}
+	}
+	
+	public void removeInstanceListener(InstanceListener listener) {
+		synchronized (this.instanceListeners) {
+			this.instanceListeners.remove(listener);
+		}
+	}
+	
+	private void notifyNewInstance(Instance instance) {
+		synchronized (this.instanceListeners) {
+			for (InstanceListener listener : this.instanceListeners) {
+				try {
+					listener.newInstanceAvailable(instance);
+				}
+				catch (Throwable t) {
+					LOG.error("Notification of new instance availability failed.", t);
+				}
+			}
+		}
+	}
+	
+	private void notifyDeadInstance(Instance instance) {
+		synchronized (this.instanceListeners) {
+			for (InstanceListener listener : this.instanceListeners) {
+				try {
+					listener.instanceDied(instance);
+				}
+				catch (Throwable t) {
+					LOG.error("Notification of dead instance failed.", t);
+				}
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private void checkForDeadInstances() {
+		final long now = System.currentTimeMillis();
+		final long timeout = InstanceManager.this.heartbeatTimeout;
+		
+		synchronized (InstanceManager.this.lock) {
+			if (InstanceManager.this.shutdown) {
+				return;
+			}
+
+			final Iterator<Map.Entry<InstanceID, Instance>> entries = registeredHostsById.entrySet().iterator();
+			
+			// check all hosts whether they did not send heart-beat messages.
+			while (entries.hasNext()) {
+				
+				final Map.Entry<InstanceID, Instance> entry = entries.next();
+				final Instance host = entry.getValue();
+				
+				if (!host.isStillAlive(now, timeout)) {
+					
+					// remove from the living
+					entries.remove();
+					registeredHostsByConnection.remove(host.getInstanceConnectionInfo());
+					
+					// add to the dead
+					deadHosts.add(host.getInstanceConnectionInfo());
+					
+					host.markDead();
+					
+					totalNumberOfAliveTaskSlots -= host.getTotalNumberOfSlots();
+					
+					LOG.info(String.format("TaskManager %s at %s did not report a heartbeat for %d msecs - marking as dead. Current number of registered hosts is %d.",
+							host.getId(), host.getInstanceConnectionInfo(), heartbeatTimeout, registeredHostsById.size()));
+					
+					// report to all listeners
+					notifyDeadInstance(host);
+				}
+			}
+		}
+	}
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Periodic task that checks whether hosts have not sent their heart-beat
+	 * messages and purges the hosts in this case.
+	 */
+	private final TimerTask cleanupStaleMachines = new TimerTask() {
+		@Override
+		public void run() {
+			try {
+				checkForDeadInstances();
+			}
+			catch (Throwable t) {
+				LOG.error("Checking for dead instances failed.", t);
+			}
+		}
+	};
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java
index f8f41ae..832b8cd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/LocalInstanceManager.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.runtime.instance;
 
-
 import java.util.ArrayList;
 import java.util.List;
 
@@ -29,7 +28,11 @@ import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.taskmanager.TaskManager;
 
-public class LocalInstanceManager extends DefaultInstanceManager {
+/**
+ * A variant of the {@link InstanceManager} that internally spawn task managers as instances, rather than waiting for external
+ * TaskManagers to register.
+ */
+public class LocalInstanceManager extends InstanceManager {
 	
 	private final List<TaskManager> taskManagers = new ArrayList<TaskManager>();
 
@@ -55,7 +58,7 @@ public class LocalInstanceManager extends DefaultInstanceManager {
 				GlobalConfiguration.includeConfiguration(tm);
 			}
 
-			taskManagers.add(new TaskManager(execMode));
+			taskManagers.add(TaskManager.createTaskManager(execMode));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
index cd2e9ca..772a4f0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import java.io.EOFException;
@@ -28,9 +27,9 @@ import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
 
-
 /**
- * A {@link DataInputView} that is backed by a {@link BlockChannelReader}, making it effectively a data input
+ * A {@link org.apache.flink.core.memory.DataInputView} that is backed by a
+ * {@link BlockChannelReader}, making it effectively a data input
  * stream. The view reads it data in blocks from the underlying channel. The view can only read data that
  * has been written by a {@link ChannelWriterOutputView}, due to block formatting.
  */
@@ -245,8 +244,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
 	
 
 	@Override
-	protected int getLimitForSegment(MemorySegment segment)
-	{
+	protected int getLimitForSegment(MemorySegment segment) {
 		return segment.getInt(ChannelWriterOutputView.HEAD_BLOCK_LENGTH_OFFSET);
 	}
 	
@@ -257,8 +255,7 @@ public class ChannelReaderInputView extends AbstractPagedInputView {
 	 * @param seg The segment to use for the read request.
 	 * @throws IOException Thrown, if the reader is in error.
 	 */
-	protected void sendReadRequest(MemorySegment seg) throws IOException
-	{
+	protected void sendReadRequest(MemorySegment seg) throws IOException {
 		if (this.numRequestsRemaining != 0) {
 			this.reader.readBlock(seg);
 			if (this.numRequestsRemaining != -1) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
index 06b49ae..c87c308 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import java.io.IOException;
@@ -27,15 +26,14 @@ import java.util.concurrent.LinkedBlockingQueue;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
 
-
 /**
- * A {@link DataOutputView} that is backed by a {@link BlockChannelWriter}, making it effectively a data output
+ * A {@link org.apache.flink.core.memory.DataOutputView} that is backed by a
+ * {@link BlockChannelWriter}, making it effectively a data output
  * stream. The view writes it data in blocks to the underlying channel, adding a minimal header to each block.
  * The data can be re-read by a {@link ChannelReaderInputView}, if it uses the same block size.
- *
  */
-public final class ChannelWriterOutputView extends AbstractPagedOutputView
-{
+public final class ChannelWriterOutputView extends AbstractPagedOutputView {
+	
 	/**
 	 * The magic number that identifies blocks as blocks from a ChannelWriterOutputView.
 	 */

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/Deserializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/Deserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/Deserializer.java
deleted file mode 100644
index c6a2ebe..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/Deserializer.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * 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.io.disk.iomanager;
-
-import java.io.DataInput;
-import java.io.IOException;
-
-/**
- * <p>
- * Provides a facility for deserializing objects of type <T> from an {@link InputStream}.
- * </p>
- * <p>
- * Deserializers are stateful, but must not buffer the input since other producers may read from the input between calls
- * to {@link #deserialize(Object)}.
- * </p>
- * 
- * @param <T>
- */
-public interface Deserializer<T> {
-	/**
-	 * <p>
-	 * Prepare the deserializer for reading.
-	 * </p>
-	 */
-	void open(DataInput in) throws IOException;
-
-	/**
-	 * <p>
-	 * Deserialize the next object from the underlying input stream. If the object <code>t</code> is non-null then this
-	 * deserializer <i>may</i> set its internal state to the next object read from the input stream. Otherwise, if the
-	 * object <code>t</code> is null a new deserialized object will be created.
-	 * </p>
-	 * 
-	 * @return the deserialized object
-	 */
-	T deserialize(T t) throws IOException;
-
-	/**
-	 * <p>
-	 * Clear up any resources.
-	 * </p>
-	 */
-	void close() throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
index bd3665f..0147f73 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/HeaderlessChannelReaderInputView.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import java.io.EOFException;
@@ -25,9 +24,9 @@ import java.util.List;
 
 import org.apache.flink.core.memory.MemorySegment;
 
-
 /**
- * A {@link DataInputView} that is backed by a {@link BlockChannelReader}, making it effectively a data input
+ * A {@link org.apache.flink.core.memory.DataInputView} that is backed by a
+ * {@link BlockChannelReader}, making it effectively a data input
  * stream. This view is similar to the {@link ChannelReaderInputView}, but does not expect
  * a header for each block, giving a direct stream abstraction over sequence of written
  * blocks. It therefore requires specification of the number of blocks and the number of
@@ -73,8 +72,7 @@ public class HeaderlessChannelReaderInputView extends ChannelReaderInputView
 	
 
 	@Override
-	protected MemorySegment nextSegment(MemorySegment current) throws IOException
-	{
+	protected MemorySegment nextSegment(MemorySegment current) throws IOException {
 		// check for end-of-stream
 		if (this.numBlocksRemaining <= 0) {
 			this.reader.close();
@@ -94,8 +92,7 @@ public class HeaderlessChannelReaderInputView extends ChannelReaderInputView
 	
 
 	@Override
-	protected int getLimitForSegment(MemorySegment segment)
-	{
+	protected int getLimitForSegment(MemorySegment segment) {
 		return this.numBlocksRemaining > 0 ? segment.size() : this.lastBlockBytes;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
index 8fced32..fc39db2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/IOManager.java
@@ -31,13 +31,10 @@ import org.apache.flink.core.memory.MemorySegment;
 
 /**
  * The facade for the provided I/O manager services.
- * 
  */
-public final class IOManager implements UncaughtExceptionHandler
-{
-	/**
-	 * Logging.
-	 */
+public class IOManager implements UncaughtExceptionHandler {
+	
+	/** Logging */
 	private static final Logger LOG = LoggerFactory.getLogger(IOManager.class);
 
 	/**
@@ -85,7 +82,7 @@ public final class IOManager implements UncaughtExceptionHandler
 	/**
 	 * Constructs a new IOManager.
 	 * 
-	 * @param path The base directory path for files underlying channels.
+	 * @param tempDir The base directory path for files underlying channels.
 	 */
 	public IOManager(String tempDir) {
 		this(new String[] {tempDir});
@@ -94,12 +91,10 @@ public final class IOManager implements UncaughtExceptionHandler
 	/**
 	 * Constructs a new IOManager.
 	 * 
-	 * @param path
-	 *        the basic directory path for files underlying anonymous
-	 *        channels.
+	 * @param paths
+	 *        the basic directory paths for files underlying anonymous channels.
 	 */
-	public IOManager(String[] paths)
-	{
+	public IOManager(String[] paths) {
 		this.paths = paths;
 		this.random = new Random();
 		this.nextPath = 0;
@@ -199,7 +194,7 @@ public final class IOManager implements UncaughtExceptionHandler
 	@Override
 	public void uncaughtException(Thread t, Throwable e)
 	{
-		LOG.error("IO Thread '" + t.getName() + "' terminated due to an exception. Closing I/O Manager.", e);
+		LOG.fatal("IO Thread '" + t.getName() + "' terminated due to an exception. Closing I/O Manager.", e);
 		shutdown();	
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
index 3aae114..5571ccb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ChannelManager.java
@@ -26,7 +26,7 @@ import org.apache.flink.runtime.AbstractID;
 import org.apache.flink.runtime.execution.CancelTaskException;
 import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.io.network.bufferprovider.BufferProvider;
 import org.apache.flink.runtime.io.network.bufferprovider.BufferProviderBroker;
@@ -124,7 +124,7 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 		// Check if we can safely run this task with the given buffers
 		ensureBufferAvailability(task);
 
-		RuntimeEnvironment environment = task.getRuntimeEnvironment();
+		RuntimeEnvironment environment = task.getEnvironment();
 
 		// -------------------------------------------------------------------------------------------------------------
 		//                                       Register output channels
@@ -132,8 +132,8 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 
 		environment.registerGlobalBufferPool(this.globalBufferPool);
 
-		if (this.localBuffersPools.containsKey(task.getVertexID())) {
-			throw new IllegalStateException("Vertex " + task.getVertexID() + " has a previous buffer pool owner");
+		if (this.localBuffersPools.containsKey(task.getExecutionId())) {
+			throw new IllegalStateException("Execution " + task.getExecutionId() + " has a previous buffer pool owner");
 		}
 
 		for (OutputGate gate : environment.outputGates()) {
@@ -155,7 +155,7 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 			}
 		}
 
-		this.localBuffersPools.put(task.getVertexID(), environment);
+		this.localBuffersPools.put(task.getExecutionId(), environment);
 
 		// -------------------------------------------------------------------------------------------------------------
 		//                                       Register input channels
@@ -187,10 +187,10 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 	/**
 	 * Unregisters the given task from the channel manager.
 	 *
-	 * @param vertexId the ID of the task to be unregistered
+	 * @param executionId the ID of the task to be unregistered
 	 * @param task the task to be unregistered
 	 */
-	public void unregister(ExecutionVertexID vertexId, Task task) {
+	public void unregister(ExecutionAttemptID executionId, Task task) {
 		final Environment environment = task.getEnvironment();
 
 		// destroy and remove OUTPUT channels from registered channels and cache
@@ -222,7 +222,7 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 		}
 
 		// clear and remove OUTPUT side buffer pool
-		LocalBufferPoolOwner bufferPool = this.localBuffersPools.remove(vertexId);
+		LocalBufferPoolOwner bufferPool = this.localBuffersPools.remove(executionId);
 		if (bufferPool != null) {
 			bufferPool.clearLocalBufferPool();
 		}
@@ -249,7 +249,7 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 		int numChannels = this.channels.size() + env.getNumberOfOutputChannels() + env.getNumberOfInputChannels();
 
 		// need at least one buffer per channel
-		if (numBuffers / numChannels < 1) {
+		if (numChannels > 0 && numBuffers / numChannels < 1) {
 			String msg = String.format("%s has not enough buffers to safely execute %s (%d buffers missing)",
 					this.connectionInfo.hostname(), env.getTaskName(), numChannels - numBuffers);
 
@@ -582,10 +582,6 @@ public class ChannelManager implements EnvelopeDispatcher, BufferProviderBroker
 	/**
 	 * 
 	 * Upon an exception, this method frees the envelope.
-	 * 
-	 * @param envelope
-	 * @return
-	 * @throws IOException
 	 */
 	private final EnvelopeReceiverList getReceiverListForEnvelope(Envelope envelope, boolean reportException) throws IOException {
 		try {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/InputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/InputGate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/InputGate.java
index 238d1e7..c167093 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/InputGate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/InputGate.java
@@ -16,10 +16,10 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.network.gates;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicReference;
@@ -43,7 +43,7 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobID;
 
 /**
- * In Nephele input gates are a specialization of general gates and connect input channels and record readers. As
+ * Input gates are a specialization of general gates and connect input channels and record readers. As
  * channels, input gates are always parameterized to a specific type of record which they can transport. In contrast to
  * output gates input gates can be associated with a {@link DistributionPattern} object which dictates the concrete
  * wiring between two groups of vertices.
@@ -103,14 +103,12 @@ public class InputGate<T extends IOReadableWritable> extends Gate<T> implements
 
 	@SuppressWarnings("unchecked")
 	public void initializeChannels(GateDeploymentDescriptor inputGateDescriptor){
-		channels = new InputChannel[inputGateDescriptor.getNumberOfChannelDescriptors()];
-
-		setChannelType(inputGateDescriptor.getChannelType());
-
-		final int nicdd = inputGateDescriptor.getNumberOfChannelDescriptors();
+		List<ChannelDeploymentDescriptor> channelDescr = inputGateDescriptor.getChannels();
+		
+		channels = new InputChannel[channelDescr.size()];
 
-		for(int i = 0; i < nicdd; i++){
-			final ChannelDeploymentDescriptor cdd = inputGateDescriptor.getChannelDescriptor(i);
+		for(int i = 0; i < channelDescr.size(); i++){
+			ChannelDeploymentDescriptor cdd = channelDescr.get(i);
 			channels[i] = new InputChannel<T>(this, i, cdd.getInputChannelID(),
 					cdd.getOutputChannelID(), getChannelType());
 		}


[57/63] [abbrv] git commit: Fix failure exception message report back to client

Posted by se...@apache.org.
Fix failure exception message report back to client


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/8e7216a0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/8e7216a0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/8e7216a0

Branch: refs/heads/master
Commit: 8e7216a070f2de6923342154e4c53816dc058fcf
Parents: 2876455
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 00:04:28 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:01 2014 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/executiongraph/Execution.java | 13 ++++++++++++-
 .../flink/runtime/executiongraph/ExecutionGraph.java   | 12 +++++++++---
 2 files changed, 21 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8e7216a0/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index f6b6a9b..b002d8d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -341,7 +341,16 @@ public class Execution {
 					
 					// we skip the canceling state. set the timestamp, for a consistent appearance
 					markTimestamp(CANCELING, getStateTimestamp(CANCELED));
-					vertex.executionCanceled();
+					
+					try {
+						vertex.executionCanceled();
+					}
+					finally {
+						vertex.getExecutionGraph().deregisterExecution(this);
+						if (assignedResource != null) {
+							assignedResource.releaseSlot();
+						}
+					}
 					return;
 				}
 				// else: fall through the loop
@@ -440,6 +449,8 @@ public class Execution {
 					}
 					return;
 				}
+				
+				// else fall through the loop
 			} 
 			else {
 				// failing in the meantime may happen and is no problem.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8e7216a0/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index d916f74..3ac3386 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -107,6 +107,8 @@ public class ExecutionGraph {
 	
 	private volatile JobStatus state = JobStatus.CREATED;
 	
+	private volatile Throwable failureCause;
+	
 	
 	private Scheduler scheduler;
 	
@@ -322,6 +324,8 @@ public class ExecutionGraph {
 				return;
 			}
 			else if (transitionState(current, JobStatus.FAILING, t)) {
+				this.failureCause = t;
+				
 				// cancel all. what is failed will not cancel but stay failed
 				for (ExecutionJobVertex ejv : verticesInCreationOrder) {
 					ejv.cancel();
@@ -366,7 +370,9 @@ public class ExecutionGraph {
 		synchronized (progressLock) {
 			int nextPos = nextVertexToFinish;
 			if (nextPos >= verticesInCreationOrder.size()) {
-				throw new RuntimeException("Inconsistency in job progress detection.");
+				// already done, and we still get a report?
+				LOG.error("Job entered finished state a repeated time.");
+				return;
 			}
 			
 			// see if we are the next to finish and then progress until the next unfinished one
@@ -390,7 +396,7 @@ public class ExecutionGraph {
 						if (current == JobStatus.CANCELLING && transitionState(current, JobStatus.CANCELED)) {
 							break;
 						}
-						if (current == JobStatus.FAILING && transitionState(current, JobStatus.FAILED)) {
+						if (current == JobStatus.FAILING && transitionState(current, JobStatus.FAILED, failureCause)) {
 							break;
 						}
 						if (current == JobStatus.CANCELED || current == JobStatus.CREATED || current == JobStatus.FINISHED) {
@@ -526,7 +532,7 @@ public class ExecutionGraph {
 				return ConnectionInfoLookupResponse.createReceiverNotReady();
 			}
 			catch (JobException e) {
-				fail(new Exception("Cannot schedule the receivers, not enough resources."));
+				fail(new Exception("Cannot schedule the receivers, not enough resources", e));
 				return ConnectionInfoLookupResponse.createJobIsAborting();
 			}
 		}


[34/63] [abbrv] git commit: Fix race at TaskManager registration during startup

Posted by se...@apache.org.
Fix race at TaskManager registration during startup


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/ef7957f0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/ef7957f0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/ef7957f0

Branch: refs/heads/master
Commit: ef7957f008f98491c3c525a224fec9107dca9179
Parents: 8998a30
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 16:52:48 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../main/java/org/apache/flink/runtime/jobmanager/JobManager.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ef7957f0/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index 113f8fd..f79fecb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -661,7 +661,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 
 	@Override
 	public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) {
-		if (this.instanceManager != null) {
+		if (this.instanceManager != null && this.scheduler != null) {
 			return this.instanceManager.registerTaskManager(instanceConnectionInfo, hardwareDescription, numberOfSlots);
 		} else {
 			return null;


[42/63] [abbrv] Adjust ExecutionGraph state machine to TaskManager's failing model (direct transitions to canceled)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
index fd9f10d..e8f8b72 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
@@ -69,6 +69,7 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileRespon
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheUpdate;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.filecache.FileCache;
+import org.apache.flink.runtime.instance.Hardware;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.InstanceID;
@@ -218,7 +219,10 @@ public class TaskManager implements TaskOperationProtocol {
 
 		// Start local RPC server, give it the number of threads as we have slots
 		try {
-			this.taskManagerServer = RPC.getServer(this, taskManagerBindAddress.getHostAddress(), ipcPort, numberOfSlots);
+			// some magic number for the handler threads
+			final int numHandlers = Math.min(numberOfSlots, 2*Hardware.getNumberCPUCores());
+			
+			this.taskManagerServer = RPC.getServer(this, taskManagerBindAddress.getHostAddress(), ipcPort, numHandlers);
 			this.taskManagerServer.start();
 		} catch (IOException e) {
 			LOG.error("Failed to start TaskManager server. " + e.getMessage(), e);
@@ -396,6 +400,8 @@ public class TaskManager implements TaskOperationProtocol {
 
 		LOG.info("Shutting down TaskManager");
 		
+		cancelAndClearEverything();
+		
 		// first, stop the heartbeat thread and wait for it to terminate
 		this.heartbeatThread.interrupt();
 		try {
@@ -540,7 +546,13 @@ public class TaskManager implements TaskOperationProtocol {
 		final int taskIndex = tdd.getIndexInSubtaskGroup();
 		final int numSubtasks = tdd.getCurrentNumberOfSubtasks();
 		
+		boolean jarsRegistered = false;
+		
 		try {
+			// library and classloader issues first
+			LibraryCacheManager.register(jobID, tdd.getRequiredJarFiles());
+			jarsRegistered = true;
+			
 			final ClassLoader userCodeClassLoader = LibraryCacheManager.getClassLoader(jobID);
 			if (userCodeClassLoader == null) {
 				throw new Exception("No user code ClassLoader available.");
@@ -578,11 +590,17 @@ public class TaskManager implements TaskOperationProtocol {
 					cpTasks.put(e.getKey(), cp);
 				}
 				env.addCopyTasksForCacheFile(cpTasks);
-			
+				
 				if (!task.startExecution()) {
 					throw new Exception("Cannot start task. Task was canceled or failed.");
 				}
 			
+				// final check that we can go (we do this after the registration, so the the "happen's before"
+				// relationship ensures that either the shutdown removes this task, or we are aware of the shutdown
+				if (shutdownStarted.get()) {
+					throw new Exception("Task Manager is shut down.");
+				}
+				
 				success = true;
 				return new TaskOperationResult(executionId, true);
 			}
@@ -590,6 +608,7 @@ public class TaskManager implements TaskOperationProtocol {
 				if (!success) {
 					// remove task 
 					this.runningTasks.remove(executionId);
+					
 					// delete distributed cache files
 					for (Entry<String, DistributedCacheEntry> e : DistributedCache.readFileInfoFromConfig(tdd.getJobConfiguration())) {
 						this.fileCache.deleteTmpFile(e.getKey(), e.getValue(), jobID);
@@ -600,11 +619,13 @@ public class TaskManager implements TaskOperationProtocol {
 		catch (Throwable t) {
 			LOG.error("Could not instantiate task", t);
 			
-			try {
-				LibraryCacheManager.unregister(jobID);
-			} catch (IOException e) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Unregistering the execution " + executionId + " caused an IOException");
+			if (jarsRegistered) {
+				try {
+					LibraryCacheManager.unregister(jobID);
+				} catch (IOException e) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Unregistering the execution " + executionId + " caused an IOException");
+					}
 				}
 			}
 			
@@ -623,7 +644,9 @@ public class TaskManager implements TaskOperationProtocol {
 		// Task de-registration must be atomic
 		final Task task = this.runningTasks.remove(executionId);
 		if (task == null) {
-			LOG.error("Cannot find task with ID " + executionId + " to unregister");
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Cannot find task with ID " + executionId + " to unregister");
+			}
 			return;
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
index 91d9973..8e276e1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
@@ -58,7 +58,9 @@ public class ExecutorThreadFactory implements ThreadFactory {
 
 		@Override
 		public void uncaughtException(Thread t, Throwable e) {
-			LOG.error("Thread '" + t.getName() + "' produced an uncaught exception.", e);
+			if (LOG.isErrorEnabled()) {
+				LOG.error("Thread '" + t.getName() + "' produced an uncaught exception.", e);
+			}
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
index ac76623..41ca7da 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -42,7 +42,7 @@ import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.operators.RegularPactTask;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
@@ -257,7 +257,7 @@ public class ExecutionGraphDeploymentTest {
 			}
 		});
 		
-		DefaultScheduler scheduler = new DefaultScheduler();
+		Scheduler scheduler = new Scheduler();
 		for (int i = 0; i < dop1 + dop2; i++) {
 			scheduler.newInstanceAvailable(getInstance(taskManager));
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
index ce1ab30..a351209 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
@@ -34,7 +34,7 @@ import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
 
@@ -417,6 +417,45 @@ public class ExecutionVertexCancelTest {
 		}
 	}
 	
+	@Test
+	public void testSendCancelAndReceiveFail() {
+		
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
+			
+			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
+			final ExecutionAttemptID execId = vertex.getCurrentExecutionAttempt().getAttemptId();
+
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.cancelTask(execId)).thenThrow(new IOException("RPC call failed"));
+			
+			Instance instance = getInstance(taskManager);
+			AllocatedSlot slot = instance.allocateSlot(new JobID());
+
+			setVertexState(vertex, ExecutionState.RUNNING);
+			setVertexResource(vertex, slot);
+			
+			assertEquals(ExecutionState.RUNNING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState.CANCELING, vertex.getExecutionState());
+			
+			vertex.getCurrentExecutionAttempt().markFailed(new Throwable("test"));
+			
+			assertTrue(vertex.getExecutionState() == ExecutionState.CANCELED || vertex.getExecutionState() == ExecutionState.FAILED);
+			
+			assertTrue(slot.isReleased());
+			
+			assertEquals(0, vertex.getExecutionGraph().getRegisteredExecutions().size());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	//  Actions after a vertex has been canceled or while canceling
 	// --------------------------------------------------------------------------------------------
@@ -436,7 +475,7 @@ public class ExecutionVertexCancelTest {
 			// scheduling after being created should be tolerated (no exception) because
 			// it can occur as the result of races
 			{
-				DefaultScheduler scheduler = mock(DefaultScheduler.class);
+				Scheduler scheduler = mock(Scheduler.class);
 				vertex.scheduleForExecution(scheduler, false);
 				
 				assertEquals(ExecutionState.CANCELED, vertex.getExecutionState());
@@ -475,7 +514,7 @@ public class ExecutionVertexCancelTest {
 				ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 				setVertexState(vertex, ExecutionState.CANCELING);
 				
-				DefaultScheduler scheduler = mock(DefaultScheduler.class);
+				Scheduler scheduler = mock(Scheduler.class);
 				vertex.scheduleForExecution(scheduler, false);
 				fail("Method should throw an exception");
 			}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
index 43d6547..f59b326 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
@@ -29,7 +29,7 @@ import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
@@ -54,7 +54,7 @@ public class ExecutionVertexSchedulingTest {
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(new JobVertexID());
 			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 			
-			DefaultScheduler scheduler = mock(DefaultScheduler.class);
+			Scheduler scheduler = mock(Scheduler.class);
 			when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot);
 			
 			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
@@ -89,7 +89,7 @@ public class ExecutionVertexSchedulingTest {
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(new JobVertexID());
 			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 			
-			DefaultScheduler scheduler = mock(DefaultScheduler.class);
+			Scheduler scheduler = mock(Scheduler.class);
 			when(scheduler.scheduleQueued(Matchers.any(ScheduledUnit.class))).thenReturn(future);
 			
 			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());
@@ -124,7 +124,7 @@ public class ExecutionVertexSchedulingTest {
 			final ExecutionJobVertex ejv = getJobVertexNotExecuting(new JobVertexID());
 			final ExecutionVertex vertex = new ExecutionVertex(ejv, 0, new IntermediateResult[0]);
 			
-			DefaultScheduler scheduler = mock(DefaultScheduler.class);
+			Scheduler scheduler = mock(Scheduler.class);
 			when(scheduler.scheduleImmediately(Matchers.any(ScheduledUnit.class))).thenReturn(slot);
 			
 			assertEquals(ExecutionState.CREATED, vertex.getExecutionState());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index c7e3463..8e87e7b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -32,35 +32,51 @@ import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.client.AbstractJobResult;
 import org.apache.flink.runtime.client.JobSubmissionResult;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
+import org.apache.flink.runtime.io.network.api.RecordReader;
+import org.apache.flink.runtime.io.network.api.RecordWriter;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobmanager.tasks.NoOpInvokable;
+import org.apache.flink.runtime.types.IntegerRecord;
+import org.apache.flink.util.StringUtils;
+
 import org.junit.Test;
 
 /**
  * This test is intended to cover the basic functionality of the {@link JobManager}.
  */
 public class JobManagerITCase {
-
+	
 	@Test
-	public void testSingleVertexJob() {
+	public void testSingleVertexJobImmediately() {
+		
+		final int NUM_TASKS = 133;
+		
 		try {
 			final AbstractJobVertex vertex = new AbstractJobVertex("Test Vertex");
-			vertex.setParallelism(3);
+			vertex.setParallelism(NUM_TASKS);
 			vertex.setInvokableClass(NoOpInvokable.class);
 			
 			final JobGraph jobGraph = new JobGraph("Test Job", vertex);
 			
-			JobManager jm = startJobManager(3);
+			JobManager jm = startJobManager(NUM_TASKS);
 			try {
 				
+				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 				
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -77,7 +93,6 @@ public class JobManagerITCase {
 							success = true;
 							break;
 						}
-						
 						else if (state == JobStatus.FAILED || state == JobStatus.CANCELED) {
 							break;
 						}
@@ -102,6 +117,475 @@ public class JobManagerITCase {
 		}
 	}
 	
+	@Test
+	public void testSingleVertexJobQueued() {
+		
+		final int NUM_TASKS = 111;
+		
+		try {
+			final AbstractJobVertex vertex = new AbstractJobVertex("Test Vertex");
+			vertex.setParallelism(NUM_TASKS);
+			vertex.setInvokableClass(NoOpInvokable.class);
+			
+			final JobGraph jobGraph = new JobGraph("Test Job", vertex);
+			jobGraph.setAllowQueuedScheduling(true);
+			
+			JobManager jm = startJobManager(10);
+			try {
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+				
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FINISHED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testForwardJob() {
+		
+		final int NUM_TASKS = 31;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(Receiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			JobManager jm = startJobManager(2 * NUM_TASKS);
+			try {
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FINISHED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testBipartiteJob() {
+		
+		final int NUM_TASKS = 31;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(AgnosticReceiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			final JobGraph jobGraph = new JobGraph("Bipartite Job", sender, receiver);
+			
+			JobManager jm = startJobManager(2 * NUM_TASKS);
+			try {
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FINISHED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTwoInputJob() {
+		
+		final int NUM_TASKS = 13;
+		
+		try {
+			final AbstractJobVertex sender1 = new AbstractJobVertex("Sender1");
+			final AbstractJobVertex sender2 = new AbstractJobVertex("Sender2");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender1.setInvokableClass(Sender.class);
+			sender2.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(AgnosticReceiver.class);
+			
+			sender1.setParallelism(NUM_TASKS);
+			sender2.setParallelism(2*NUM_TASKS);
+			receiver.setParallelism(3*NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE);
+			receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE);
+			
+			final JobGraph jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2);
+			
+			JobManager jm = startJobManager(6 * NUM_TASKS);
+			try {
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FAILED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testJobFailingSender() {
+		
+		final int NUM_TASKS = 100;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(ExceptionSender.class);
+			receiver.setInvokableClass(Receiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			JobManager jm = startJobManager(NUM_TASKS);
+			try {
+				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FAILED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testJobSometimesFailingSender() {
+		
+		final int NUM_TASKS = 100;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(SometimesExceptionSender.class);
+			receiver.setInvokableClass(Receiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			JobManager jm = startJobManager(NUM_TASKS);
+			try {
+				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FAILED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testJobFailingReceiver() {
+		
+		final int NUM_TASKS = 200;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(Sender.class);
+			receiver.setInvokableClass(ExceptionReceiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			JobManager jm = startJobManager(2 * NUM_TASKS);
+			try {
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+				
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FAILED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	/**
+	 * Test failure in instantiation, where all fail by themselves
+	 */
+	@Test
+	public void testJobFailingInstantiation() {
+		
+		final int NUM_TASKS = 200;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(InstantiationErrorSender.class);
+			receiver.setInvokableClass(Receiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			JobManager jm = startJobManager(NUM_TASKS);
+			try {
+				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FAILED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	/**
+	 * Test failure in instantiation, where some have to be canceled (not all fail by themselves)
+	 */
+	@Test
+	public void testJobFailingSomeInstantiations() {
+		
+		final int NUM_TASKS = 200;
+		
+		try {
+			final AbstractJobVertex sender = new AbstractJobVertex("Sender");
+			final AbstractJobVertex receiver = new AbstractJobVertex("Receiver");
+			
+			sender.setInvokableClass(SometimesInstantiationErrorSender.class);
+			receiver.setInvokableClass(Receiver.class);
+			
+			sender.setParallelism(NUM_TASKS);
+			receiver.setParallelism(NUM_TASKS);
+			
+			receiver.connectNewDataSetAsInput(sender, DistributionPattern.POINTWISE);
+			
+			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
+			
+			JobManager jm = startJobManager(NUM_TASKS);
+			try {
+				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				
+				// we need to register the job at the library cache manager (with no libraries)
+				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
+				
+				JobSubmissionResult result = jm.submitJob(jobGraph);
+
+				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
+				
+				// monitor the execution
+				ExecutionGraph eg = jm.getCurrentJobs().get(jobGraph.getJobID());
+				
+				if (eg != null) {
+					eg.waitForJobEnd();
+					assertEquals(JobStatus.FAILED, eg.getState());
+				}
+				else {
+					// already done, that was fast;
+				}
+				
+				for (Execution e : eg.getRegisteredExecutions().values()) {
+					System.out.println(e + StringUtils.arrayAwareToString(e.getStateTimestamps()));
+				}
+				
+				assertEquals(0, eg.getRegisteredExecutions().size());
+			}
+			finally {
+				jm.shutdown();
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
 	// --------------------------------------------------------------------------------------------
 	
 	private static final JobManager startJobManager(int numSlots) throws Exception {
@@ -114,6 +598,17 @@ public class JobManagerITCase {
 		GlobalConfiguration.includeConfiguration(cfg);
 		
 		JobManager jm = new JobManager(ExecutionMode.LOCAL);
+		
+		// we need to wait until the taskmanager is registered
+		// max time is 5 seconds
+		long deadline = System.currentTimeMillis() + 5000;
+		
+		while (jm.getAvailableSlots() < numSlots && System.currentTimeMillis() < deadline) {
+			Thread.sleep(10);
+		}
+		
+		assertEquals(numSlots, jm.getAvailableSlots());
+		
 		return jm;
 	}
 	
@@ -133,4 +628,169 @@ public class JobManagerITCase {
 		
 		throw new IOException("could not find free port");
 	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Simple test tasks
+	// --------------------------------------------------------------------------------------------
+	
+	public static final class Sender extends AbstractInvokable {
+
+		private RecordWriter<IntegerRecord> writer;
+		
+		@Override
+		public void registerInputOutput() {
+			writer = new RecordWriter<IntegerRecord>(this);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			writer.initializeSerializers();
+			writer.emit(new IntegerRecord(42));
+			writer.emit(new IntegerRecord(1337));
+			writer.flush();
+		}
+	}
+	
+	public static final class Receiver extends AbstractInvokable {
+
+		private RecordReader<IntegerRecord> reader;
+		
+		@Override
+		public void registerInputOutput() {
+			reader = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			IntegerRecord i1 = reader.next();
+			IntegerRecord i2 = reader.next();
+			IntegerRecord i3 = reader.next();
+			
+			if (i1.getValue() != 42 || i2.getValue() != 1337 || i3 != null) {
+				throw new Exception("Wrong Data Received");
+			}
+		}
+	}
+	
+	public static final class AgnosticReceiver extends AbstractInvokable {
+
+		private RecordReader<IntegerRecord> reader;
+		
+		@Override
+		public void registerInputOutput() {
+			reader = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			while (reader.next() != null);
+		}
+	}
+	
+	public static final class AgnosticBinaryReceiver extends AbstractInvokable {
+
+		private RecordReader<IntegerRecord> reader1;
+		private RecordReader<IntegerRecord> reader2;
+		
+		@Override
+		public void registerInputOutput() {
+			reader1 = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+			reader2 = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			while (reader1.next() != null);
+			while (reader2.next() != null);
+		}
+	}
+	
+	public static final class ExceptionSender extends AbstractInvokable {
+
+		private RecordWriter<IntegerRecord> writer;
+		
+		@Override
+		public void registerInputOutput() {
+			writer = new RecordWriter<IntegerRecord>(this);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			writer.initializeSerializers();
+			
+			throw new Exception("Test Exception");
+		}
+	}
+	
+	public static final class ExceptionReceiver extends AbstractInvokable {
+		
+		@Override
+		public void registerInputOutput() {
+			new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			throw new Exception("Expected Test Exception");
+		}
+	}
+	
+	public static final class SometimesExceptionSender extends AbstractInvokable {
+
+		private RecordWriter<IntegerRecord> writer;
+		
+		@Override
+		public void registerInputOutput() {
+			writer = new RecordWriter<IntegerRecord>(this);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			writer.initializeSerializers();
+			
+			if (Math.random() < 0.05) {
+				throw new Exception("Test Exception");
+			} else {
+				Object o = new Object();
+				synchronized (o) {
+					o.wait();
+				}
+			}
+		}
+	}
+	
+	public static final class InstantiationErrorSender extends AbstractInvokable {
+
+		public InstantiationErrorSender() {
+			throw new RuntimeException("Test Exception in Constructior");
+		}
+		
+		@Override
+		public void registerInputOutput() {}
+
+		@Override
+		public void invoke() {}
+	}
+	
+	public static final class SometimesInstantiationErrorSender extends AbstractInvokable {
+		
+		public SometimesInstantiationErrorSender() {
+			if (Math.random() < 0.05) {
+				throw new RuntimeException("Test Exception in Constructior");
+			}
+		}
+		
+		@Override
+		public void registerInputOutput() {
+			new RecordWriter<IntegerRecord>(this);
+		}
+		
+		@Override
+		public void invoke() throws Exception {
+			Object o = new Object();
+			synchronized (o) {
+				o.wait();
+			}
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
index b092312..2892384 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
@@ -40,14 +40,14 @@ import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 
 /**
- * Tests for the {@link DefaultScheduler} when scheduling individual tasks.
+ * Tests for the {@link Scheduler} when scheduling individual tasks.
  */
 public class SchedulerIsolatedTasksTest {
 	
 	@Test
 	public void testAddAndRemoveInstance() {
 		try {
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			
 			Instance i1 = getRandomInstance(2);
 			Instance i2 = getRandomInstance(2);
@@ -111,7 +111,7 @@ public class SchedulerIsolatedTasksTest {
 	@Test
 	public void testScheduleImmediately() {
 		try {
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			assertEquals(0, scheduler.getNumberOfAvailableSlots());
 			
 			scheduler.newInstanceAvailable(getRandomInstance(2));
@@ -181,7 +181,7 @@ public class SchedulerIsolatedTasksTest {
 		final int NUM_TASKS_TO_SCHEDULE = 2000;
 		
 		try {
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			
 			for (int i = 0;i < NUM_INSTANCES; i++) {
 				scheduler.newInstanceAvailable(getRandomInstance((int) (Math.random() * NUM_SLOTS_PER_INSTANCE) + 1));
@@ -270,7 +270,7 @@ public class SchedulerIsolatedTasksTest {
 	@Test
 	public void testScheduleWithDyingInstances() {
 		try {
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			
 			Instance i1 = getRandomInstance(2);
 			Instance i2 = getRandomInstance(2);
@@ -330,7 +330,7 @@ public class SchedulerIsolatedTasksTest {
 	@Test
 	public void testSchedulingLocation() {
 		try {
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			
 			Instance i1 = getRandomInstance(2);
 			Instance i2 = getRandomInstance(2);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
index afc0db9..c641524 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
@@ -42,7 +42,7 @@ public class SchedulerSlotSharingTest {
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1);
 			
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			Instance i1 = getRandomInstance(2);
 			Instance i2 = getRandomInstance(2);
 			scheduler.newInstanceAvailable(i1);
@@ -124,7 +124,7 @@ public class SchedulerSlotSharingTest {
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
 			
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
@@ -240,7 +240,7 @@ public class SchedulerSlotSharingTest {
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2, jid3);
 			
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
@@ -346,7 +346,7 @@ public class SchedulerSlotSharingTest {
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
 			
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
 			// schedule 1 tasks from the first vertex group and 2 from the second
@@ -393,7 +393,7 @@ public class SchedulerSlotSharingTest {
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
 			
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			scheduler.newInstanceAvailable(getRandomInstance(3));
 			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
@@ -535,7 +535,7 @@ public class SchedulerSlotSharingTest {
 			Instance i1 = getRandomInstance(2);
 			Instance i2 = getRandomInstance(2);
 			
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			scheduler.newInstanceAvailable(i1);
 			scheduler.newInstanceAvailable(i2);
 			
@@ -583,7 +583,7 @@ public class SchedulerSlotSharingTest {
 			Instance i1 = getRandomInstance(2);
 			Instance i2 = getRandomInstance(2);
 			
-			DefaultScheduler scheduler = new DefaultScheduler();
+			Scheduler scheduler = new Scheduler();
 			scheduler.newInstanceAvailable(i1);
 			scheduler.newInstanceAvailable(i2);
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index 13261d7..ba08a9f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -114,11 +114,6 @@ public class TaskManagerTest {
 					Collections.<GateDeploymentDescriptor>emptyList(),
 					new String[0], 0);
 			
-			LibraryCacheManager.register(jid1, new String[0]);
-			LibraryCacheManager.register(jid2, new String[0]);
-			assertNotNull(LibraryCacheManager.getClassLoader(jid1));
-			assertNotNull(LibraryCacheManager.getClassLoader(jid2));
-			
 			TaskOperationResult result1 = tm.submitTask(tdd1);
 			TaskOperationResult result2 = tm.submitTask(tdd2);
 			
@@ -196,10 +191,6 @@ public class TaskManagerTest {
 					Collections.<GateDeploymentDescriptor>emptyList(),
 					new String[0], 0);
 			
-			LibraryCacheManager.register(jid, new String[0]);
-			LibraryCacheManager.register(jid, new String[0]);
-			assertNotNull(LibraryCacheManager.getClassLoader(jid));
-			
 			assertFalse(tm.submitTask(tdd1).isSuccess());
 			assertFalse(tm.submitTask(tdd2).isSuccess());
 			
@@ -250,11 +241,6 @@ public class TaskManagerTest {
 					Collections.singletonList(new GateDeploymentDescriptor(Collections.singletonList(cdd))),
 					new String[0], 0);
 			
-			// register the job twice (for two tasks) at the lib cache
-			LibraryCacheManager.register(jid, new String[0]);
-			LibraryCacheManager.register(jid, new String[0]);
-			assertNotNull(LibraryCacheManager.getClassLoader(jid));
-			
 			// deploy sender before receiver, so the target is online when the sender requests the connection info
 			TaskOperationResult result2 = tm.submitTask(tdd2);
 			TaskOperationResult result1 = tm.submitTask(tdd1);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java
index 205ad00..c6379eb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/EnvironmentInformationTest.java
@@ -1,17 +1,20 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.util;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/25acb6ba/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java
index 27fa540..e2312ec 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/KeyGroupedIteratorTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.util;
 
 import java.io.IOException;


[62/63] [abbrv] git commit: Improve shutdown of JobManager (graceful exit of runing jobs and RPC service)

Posted by se...@apache.org.
Improve shutdown of JobManager (graceful exit of runing jobs and RPC service)


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/8411195b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/8411195b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/8411195b

Branch: refs/heads/master
Commit: 8411195b08bac461bc76073de71c425c219638de
Parents: 3e64400
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Sep 18 17:07:09 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:02 2014 +0200

----------------------------------------------------------------------
 .../flink/runtime/jobmanager/JobManager.java    | 30 +++++++++++---------
 1 file changed, 17 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8411195b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index 1c02127..90bfdc6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -40,7 +40,6 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
-
 import org.apache.flink.api.common.accumulators.Accumulator;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
@@ -61,6 +60,7 @@ import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.JobStatusListener;
+import org.apache.flink.runtime.instance.Hardware;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
@@ -92,7 +92,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.util.SerializableArrayList;
 import org.apache.flink.util.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -112,7 +111,7 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 	
 	
 	/** Executor service for asynchronous commands (to relieve the RPC threads of work) */
-	private final ExecutorService executorService = Executors.newCachedThreadPool(ExecutorThreadFactory.INSTANCE);
+	private final ExecutorService executorService = Executors.newFixedThreadPool(2 * Hardware.getNumberCPUCores(), ExecutorThreadFactory.INSTANCE);
 	
 
 	/** The RPC end point through which the JobManager gets its calls */
@@ -230,18 +229,13 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 		if (!this.isShutdownInProgress.compareAndSet(false, true)) {
 			return;
 		}
-
-		// Stop instance manager
-		if (this.instanceManager != null) {
-			this.instanceManager.shutdown();
-		}
-
-		// Stop RPC server
-		if (this.jobManagerServer != null) {
-			this.jobManagerServer.stop();
+		
+		for (ExecutionGraph e : this.currentJobs.values()) {
+			e.fail(new Exception("The JobManager is shutting down."));
 		}
-
+		
 		// Stop the executor service
+		// this waits for any pending calls to be done
 		if (this.executorService != null) {
 			this.executorService.shutdown();
 			try {
@@ -251,6 +245,16 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 			}
 		}
 
+		// Stop instance manager
+		if (this.instanceManager != null) {
+			this.instanceManager.shutdown();
+		}
+
+		// Stop RPC server
+		if (this.jobManagerServer != null) {
+			this.jobManagerServer.stop();
+		}
+
 		// Stop and clean up the job progress collector
 		if (this.eventCollector != null) {
 			this.eventCollector.shutdown();


[54/63] [abbrv] git commit: Improved distribution of IDs. Previous implementation lost bits due to double-to-long multiplication and rounding.

Posted by se...@apache.org.
Improved distribution of IDs. Previous implementation lost bits due to double-to-long multiplication and rounding.


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/28764555
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/28764555
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/28764555

Branch: refs/heads/master
Commit: 2876455503b64177279ecfdd7e00a22aab53e17e
Parents: 5d13ddb
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 00:03:00 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:01 2014 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/runtime/AbstractID.java  | 17 ++++++-----------
 1 file changed, 6 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/28764555/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
index 1fa9491..e651f6c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Random;
 
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
@@ -35,6 +36,9 @@ import io.netty.buffer.ByteBuf;
 public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, java.io.Serializable {
 
 	private static final long serialVersionUID = 1L;
+	
+	private static final Random RND = new Random();
+	
 
 	/** The size of a long in bytes */
 	private static final int SIZE_OF_LONG = 8;
@@ -94,8 +98,8 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 	 * Constructs a new random ID from a uniform distribution.
 	 */
 	public AbstractID() {
-		this.lowerPart = generateRandomLong();
-		this.upperPart = generateRandomLong();
+		this.lowerPart = RND.nextLong();
+		this.upperPart = RND.nextLong();
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -109,15 +113,6 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 	}
 
 	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Generates a uniformly distributed random positive long.
-	 *
-	 * @return a uniformly distributed random positive long
-	 */
-	protected static long generateRandomLong() {
-		return (long) (Math.random() * Long.MAX_VALUE);
-	}
 
 	/**
 	 * Converts the given byte array to a long.


[07/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 450732c..7e48d7c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -16,60 +16,72 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.taskmanager;
 
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
 import org.apache.flink.runtime.execution.ExecutionListener;
-import org.apache.flink.runtime.execution.ExecutionObserver;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.execution.ExecutionStateTransition;
+import org.apache.flink.runtime.execution.ExecutionState2;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.profiling.TaskManagerProfiler;
+import org.apache.flink.util.ExceptionUtils;
 
-import java.util.Iterator;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
+public final class Task {
 
-public final class Task implements ExecutionObserver {
-
-	/**
-	 * The log object used for debugging.
-	 */
+	/** For atomic state updates */
+	private static final AtomicReferenceFieldUpdater<Task, ExecutionState2> STATE_UPDATER = 
+			AtomicReferenceFieldUpdater.newUpdater(Task.class, ExecutionState2.class, "executionState");
+			
+	/** The log object used for debugging. */
 	private static final Logger LOG = LoggerFactory.getLogger(Task.class);
 
-	private final ExecutionVertexID vertexID;
-
-	private final RuntimeEnvironment environment;
+	// --------------------------------------------------------------------------------------------
+	
+	private final JobID jobId;
+	
+	private final JobVertexID vertexId;
+	
+	private final int subtaskIndex;
+	
+	private final int numberOfSubtasks;
+	
+	private final ExecutionAttemptID executionId;
+	
+	private final String taskName;
 
 	private final TaskManager taskManager;
+	
+	
+	private final List<ExecutionListener> executionListeners = new CopyOnWriteArrayList<ExecutionListener>();
 
-	/**
-	 * Stores whether the task has been canceled.
-	 */
-	private final AtomicBoolean canceled = new AtomicBoolean(false);
-
-	/**
-	 * The current execution state of the task
-	 */
-	private volatile ExecutionState executionState = ExecutionState.STARTING;
-
+	/** The environment (with the invokable) executed by this task */
+	private volatile RuntimeEnvironment environment;
 	
-	private Queue<ExecutionListener> registeredListeners = new ConcurrentLinkedQueue<ExecutionListener>();
+	/** The current execution state of the task */
+	private volatile ExecutionState2 executionState = ExecutionState2.DEPLOYING;
 
-	public Task(ExecutionVertexID vertexID, final RuntimeEnvironment environment, TaskManager taskManager) {
-		this.vertexID = vertexID;
-		this.environment = environment;
+	// --------------------------------------------------------------------------------------------	
+	
+	public Task(JobID jobId, JobVertexID vertexId, int taskIndex, int parallelism, 
+			ExecutionAttemptID executionId, String taskName, TaskManager taskManager)
+	{
+		this.jobId = jobId;
+		this.vertexId = vertexId;
+		this.subtaskIndex = taskIndex;
+		this.numberOfSubtasks = parallelism;
+		this.executionId = executionId;
+		this.taskName = taskName;
 		this.taskManager = taskManager;
-
-		this.environment.setExecutionObserver(this);
 	}
 
 
@@ -79,89 +91,186 @@ public final class Task implements ExecutionObserver {
 	 * @return the ID of the job this task belongs to
 	 */
 	public JobID getJobID() {
-		return this.environment.getJobID();
+		return this.jobId;
 	}
 
 	/**
-	 * Returns the ID of this task.
+	 * Returns the ID of this task vertex.
 	 * 
-	 * @return the ID of this task
+	 * @return the ID of this task vertex.
 	 */
-	public ExecutionVertexID getVertexID() {
-		return this.vertexID;
+	public JobVertexID getVertexID() {
+		return this.vertexId;
 	}
 
 	/**
-	 * Returns the environment associated with this task.
+	 * Gets the index of the parallel subtask [0, parallelism).
 	 * 
-	 * @return the environment associated with this task
+	 * @return The task index of the parallel subtask.
 	 */
-	public Environment getEnvironment() {
-		return this.environment;
+	public int getSubtaskIndex() {
+		return subtaskIndex;
 	}
-
+	
 	/**
-	 * Marks the task as failed and triggers the appropriate state changes.
+	 * Gets the total number of subtasks of the task that this subtask belongs to.
+	 * 
+	 * @return The total number of this task's subtasks.
 	 */
-	public void markAsFailed() {
-		executionStateChanged(ExecutionState.FAILED, "Execution thread died unexpectedly");
+	public int getNumberOfSubtasks() {
+		return numberOfSubtasks;
 	}
-
-	public void cancelExecution() {
-		cancelOrKillExecution();
+	
+	/**
+	 * Gets the ID of the execution attempt.
+	 * 
+	 * @return The ID of the execution attempt.
+	 */
+	public ExecutionAttemptID getExecutionId() {
+		return executionId;
 	}
-
-	public void killExecution() {
-		cancelOrKillExecution();
+	
+	/**
+	 * Returns the current execution state of the task.
+	 * 
+	 * @return the current execution state of the task
+	 */
+	public ExecutionState2 getExecutionState() {
+		return this.executionState;
 	}
-
+	
+	public void setEnvironment(RuntimeEnvironment environment) {
+		this.environment = environment;
+	}
+	
+	public RuntimeEnvironment getEnvironment() {
+		return environment;
+	}
+	
+	public boolean isCanceled() {
+		return executionState == ExecutionState2.CANCELING ||
+				executionState == ExecutionState2.CANCELED;
+	}
+	
+	public String getTaskName() {
+		return taskName;
+	}
+	
+	public String getTaskNameWithSubtasks() {
+		return this.taskName + " (" + (this.subtaskIndex + 1) + "/" + this.numberOfSubtasks + ")";
+	}
+	
+	// ----------------------------------------------------------------------------------------------------------------
+	//  States and Transitions
+	// ----------------------------------------------------------------------------------------------------------------
+	
 	/**
-	 * Cancels or kills the task.
-	 *
-	 * @param cancel <code>true/code> if the task shall be canceled, <code>false</code> if it shall be killed
+	 * Marks the task as finished. This succeeds, if the task was previously in the state
+	 * "RUNNING", otherwise it fails. Failure indicates that the task was either
+	 * canceled, or set to failed.
+	 * 
+	 * @return True, if the task correctly enters the state FINISHED.
 	 */
-	private void cancelOrKillExecution() {
-		if (!this.canceled.compareAndSet(false, true)) {
-			return;
+	public boolean markAsFinished() {
+		if (STATE_UPDATER.compareAndSet(this, ExecutionState2.RUNNING, ExecutionState2.FINISHED)) {
+			notifyObservers(ExecutionState2.FINISHED, null);
+			taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.FINISHED, null);
+			return true;
+		} else {
+			return false;
 		}
-
-		if (this.executionState != ExecutionState.RUNNING && this.executionState != ExecutionState.FINISHING) {
-			return;
+	}
+	
+	public void markFailed(Throwable error) {
+		while (true) {
+			ExecutionState2 current = this.executionState;
+			
+			if (current == ExecutionState2.CANCELED || current == ExecutionState2.CANCELING) {
+				return;
+			}
+			
+			if (STATE_UPDATER.compareAndSet(this, current, ExecutionState2.FAILED)) {
+				String message = ExceptionUtils.stringifyException(error);
+				notifyObservers(ExecutionState2.FAILED, message);
+				taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.FAILED, message);
+				return;
+			}
 		}
-
-		executionStateChanged(ExecutionState.CANCELING, null);
-
-		// Request user code to shut down
-		try {
-			this.environment.cancelExecution();
-		} catch (Throwable e) {
-			LOG.error("Error while cancelling the task.", e);
+	}
+	
+	public void cancelExecution() {
+		while (true) {
+			ExecutionState2 current = this.executionState;
+			
+			// if the task is already canceled (or canceling) or finished, then we
+			// need not do anything
+			if (current == ExecutionState2.FINISHED || current == ExecutionState2.CANCELED ||
+					current == ExecutionState2.CANCELING) {
+				return;
+			}
+			
+			if (current == ExecutionState2.DEPLOYING) {
+				// directly set to canceled
+				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState2.CANCELED)) {
+					notifyObservers(ExecutionState2.CANCELED, null);
+					taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.CANCELED, null);
+					return;
+				}
+			}
+			else if (current == ExecutionState2.RUNNING) {
+				// go to canceling and perform the actual task canceling
+				if (STATE_UPDATER.compareAndSet(this, current, ExecutionState2.CANCELING)) {
+					notifyObservers(ExecutionState2.CANCELING, null);
+					try {
+						this.environment.cancelExecution();
+					} catch (Throwable e) {
+						LOG.error("Error while cancelling the task.", e);
+					}
+					
+					return;
+				}
+			}
+			else {
+				throw new RuntimeException("unexpected state for cancelling: " + current);
+			}
+		}
+	}
+	
+	public void cancelingDone() {
+		if (STATE_UPDATER.compareAndSet(this, ExecutionState2.CANCELING, ExecutionState2.CANCELED)) {
+			notifyObservers(ExecutionState2.CANCELED, null);
+			taskManager.notifyExecutionStateChange(jobId, executionId, ExecutionState2.CANCELED, null);
 		}
 	}
 
 	/**
-	 * Checks if the state of the thread which is associated with this task is <code>TERMINATED</code>.
-	 * 
-	 * @return <code>true</code> if the state of this thread which is associated with this task is
-	 *         <code>TERMINATED</code>, <code>false</code> otherwise
+	 * Starts the execution of this task.
 	 */
-	public boolean isTerminated() {
-		final Thread executingThread = this.environment.getExecutingThread();
-		if (executingThread.getState() == Thread.State.TERMINATED) {
+	public boolean startExecution() {
+		if (STATE_UPDATER.compareAndSet(this, ExecutionState2.DEPLOYING, ExecutionState2.RUNNING)) {
+			final Thread thread = this.environment.getExecutingThread();
+			thread.start();
 			return true;
+		} else {
+			return false;
 		}
-
-		return false;
 	}
-
+	
 	/**
-	 * Starts the execution of this task.
+	 * Unregisters the task from the central memory manager.
+	 * 
+	 * @param memoryManager
+	 *        the central memory manager
 	 */
-	public void startExecution() {
-
-		final Thread thread = this.environment.getExecutingThread();
-		thread.start();
+	public void unregisterMemoryManager(MemoryManager memoryManager) {
+		if (memoryManager != null) {
+			memoryManager.releaseAll(this.environment.getInvokable());
+		}
 	}
+	
+	// -----------------------------------------------------------------------------------------------------------------
+	//                                        Task Profiling
+	// -----------------------------------------------------------------------------------------------------------------
 
 	/**
 	 * Registers the task manager profiler with the task.
@@ -171,138 +280,57 @@ public final class Task implements ExecutionObserver {
 	 * @param jobConfiguration
 	 *        the configuration attached to the job
 	 */
-	public void registerProfiler(final TaskManagerProfiler taskManagerProfiler, final Configuration jobConfiguration) {
+	public void registerProfiler(TaskManagerProfiler taskManagerProfiler, Configuration jobConfiguration) {
 		taskManagerProfiler.registerExecutionListener(this, jobConfiguration);
 	}
 
 	/**
-	 * Unregisters the task from the central memory manager.
-	 * 
-	 * @param memoryManager
-	 *        the central memory manager
-	 */
-	public void unregisterMemoryManager(final MemoryManager memoryManager) {
-		if (memoryManager != null) {
-			memoryManager.releaseAll(this.environment.getInvokable());
-		}
-	}
-
-	/**
 	 * Unregisters the task from the task manager profiler.
 	 * 
 	 * @param taskManagerProfiler
 	 *        the task manager profiler
 	 */
-	public void unregisterProfiler(final TaskManagerProfiler taskManagerProfiler) {
+	public void unregisterProfiler(TaskManagerProfiler taskManagerProfiler) {
 		if (taskManagerProfiler != null) {
-			taskManagerProfiler.unregisterExecutionListener(this.vertexID);
+			taskManagerProfiler.unregisterExecutionListener(this.executionId);
 		}
 	}
-
-	/**
-	 * Returns the current execution state of the task.
-	 * 
-	 * @return the current execution state of the task
-	 */
-	public ExecutionState getExecutionState() {
-		return this.executionState;
-	}
-
-	// -----------------------------------------------------------------------------------------------------------------
-	//                                        ExecutionObserver methods
-	// -----------------------------------------------------------------------------------------------------------------
-	@Override
-	public void executionStateChanged(final ExecutionState newExecutionState, final String optionalMessage) {
-
-		// Check the state transition
-		ExecutionStateTransition.checkTransition(false, getTaskName(), this.executionState, newExecutionState);
-
-		// Make sure the reason for a transition to FAILED appears in the log files
-		if (newExecutionState == ExecutionState.FAILED) {
-			LOG.error(optionalMessage);
-		}
-
-		// Notify all listener objects
-		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
-		while (it.hasNext()) {
-			it.next().executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
-					optionalMessage);
+	
+	// --------------------------------------------------------------------------------------------
+	//                                     State Listeners
+	// --------------------------------------------------------------------------------------------
+	
+	public void registerExecutionListener(ExecutionListener listener) {
+		if (listener == null) {
+			throw new IllegalArgumentException();
 		}
-
-		// Store the new execution state
-		this.executionState = newExecutionState;
-
-		// Finally propagate the state change to the job manager
-		this.taskManager.executionStateChanged(this.environment.getJobID(), this.vertexID, newExecutionState,
-				optionalMessage);
-	}
-
-	/**
-	 * Returns the name of the task associated with this observer object.
-	 *
-	 * @return the name of the task associated with this observer object
-	 */
-	private String getTaskName() {
-		return this.environment.getTaskName() + " (" + (this.environment.getIndexInSubtaskGroup() + 1) + "/"
-				+ this.environment.getCurrentNumberOfSubtasks() + ")";
+		this.executionListeners.add(listener);
 	}
 
-
-	@Override
-	public void userThreadStarted(final Thread userThread) {
-		// Notify the listeners
-		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
-		while (it.hasNext()) {
-			it.next().userThreadStarted(this.environment.getJobID(), this.vertexID, userThread);
+	public void unregisterExecutionListener(ExecutionListener listener) {
+		if (listener == null) {
+			throw new IllegalArgumentException();
 		}
+		this.executionListeners.remove(listener);
 	}
-
-
-	@Override
-	public void userThreadFinished(final Thread userThread) {
-		// Notify the listeners
-		final Iterator<ExecutionListener> it = this.registeredListeners.iterator();
-		while (it.hasNext()) {
-			it.next().userThreadFinished(this.environment.getJobID(), this.vertexID, userThread);
+	
+	private void notifyObservers(ExecutionState2 newState, String message) {
+		for (ExecutionListener listener : this.executionListeners) {
+			try {
+				listener.executionStateChanged(jobId, vertexId, subtaskIndex, executionId, newState, message);
+			}
+			catch (Throwable t) {
+				LOG.error("Error while calling execution listener.", t);
+			}
 		}
 	}
-
-	/**
-	 * Registers the {@link ExecutionListener} object for this task. This object
-	 * will be notified about important events during the task execution.
-	 *
-	 * @param executionListener
-	 *        the object to be notified for important events during the task execution
-	 */
-
-	public void registerExecutionListener(final ExecutionListener executionListener) {
-		this.registeredListeners.add(executionListener);
-	}
-
-	/**
-	 * Unregisters the {@link ExecutionListener} object for this environment. This object
-	 * will no longer be notified about important events during the task execution.
-	 *
-	 * @param executionListener
-	 *        the lister object to be unregistered
-	 */
-
-	public void unregisterExecutionListener(final ExecutionListener executionListener) {
-		this.registeredListeners.remove(executionListener);
-	}
-
-
+	
+	// --------------------------------------------------------------------------------------------
+	//                                       Utilities
+	// --------------------------------------------------------------------------------------------
+	
 	@Override
-	public boolean isCanceled() {
-		return this.canceled.get();
-	}
-
-	/**
-	 * Returns the runtime environment associated with this task.
-	 *
-	 * @return the runtime environment associated with this task
-	 */
-	public RuntimeEnvironment getRuntimeEnvironment() {
-		return this.environment;
+	public String toString() {
+		return getTaskNameWithSubtasks() + " [" + executionState + ']';
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskCancelResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskCancelResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskCancelResult.java
deleted file mode 100644
index 465d03d..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskCancelResult.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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.taskmanager;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-
-/**
- * A <code>TaskCancelResult</code> is used to report the results
- * of a task cancel attempt. It contains the ID of the task to be canceled, a return code and
- * a description. In case of an error during the cancel operation the description includes an error message.
- * 
- */
-public class TaskCancelResult extends AbstractTaskResult {
-
-	/**
-	 * Constructs a new task cancel result.
-	 * 
-	 * @param vertexID
-	 *        the task ID this result belongs to
-	 * @param returnCode
-	 *        the return code of the cancel
-	 */
-	public TaskCancelResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) {
-		super(vertexID, returnCode);
-	}
-
-	/**
-	 * Constructs an empty task cancel result.
-	 */
-	public TaskCancelResult() {
-		super();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
index e12012b..8ab67bc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskExecutionState.java
@@ -16,51 +16,52 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.taskmanager;
 
 import java.io.IOException;
 
 import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.util.EnumUtils;
+import org.apache.flink.types.StringValue;
 
 /**
- * This class can be used to propagate updates about a task's execution state from the
- * task manager to the job manager.
- * 
+ * This class represents an update about a task's execution state.
  */
-public class TaskExecutionState implements IOReadableWritable {
+public class TaskExecutionState implements IOReadableWritable , java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
 
-	private JobID jobID = null;
+	private JobID jobID;
 
-	private ExecutionVertexID executionVertexID = null;
+	private ExecutionAttemptID executionId;
 
-	private ExecutionState executionState = null;
+	private ExecutionState2 executionState;
 
-	private String description = null;
+	private String description;
 
 	/**
 	 * Creates a new task execution state.
 	 * 
 	 * @param jobID
 	 *        the ID of the job the task belongs to
-	 * @param id
-	 *        the ID of the task whose state is to be reported
+	 * @param executionId
+	 *        the ID of the task execution whose state is to be reported
 	 * @param executionState
 	 *        the execution state to be reported
 	 * @param description
 	 *        an optional description
 	 */
-	public TaskExecutionState(final JobID jobID, final ExecutionVertexID id, final ExecutionState executionState,
-			final String description) {
+	public TaskExecutionState(JobID jobID, ExecutionAttemptID executionId, ExecutionState2 executionState, String description) {
+		if (jobID == null || executionId == null || executionState == null) {
+			throw new NullPointerException();
+		}
+		
 		this.jobID = jobID;
-		this.executionVertexID = id;
+		this.executionId = executionId;
 		this.executionState = executionState;
 		this.description = description;
 	}
@@ -69,8 +70,12 @@ public class TaskExecutionState implements IOReadableWritable {
 	 * Creates an empty task execution state.
 	 */
 	public TaskExecutionState() {
+		this.jobID = new JobID();
+		this.executionId = new ExecutionAttemptID();
 	}
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Returns the description of this task execution state.
 	 * 
@@ -85,8 +90,8 @@ public class TaskExecutionState implements IOReadableWritable {
 	 * 
 	 * @return the ID of the task this result belongs to
 	 */
-	public ExecutionVertexID getID() {
-		return this.executionVertexID;
+	public ExecutionAttemptID getID() {
+		return this.executionId;
 	}
 
 	/**
@@ -94,7 +99,7 @@ public class TaskExecutionState implements IOReadableWritable {
 	 * 
 	 * @return the new execution state of the task
 	 */
-	public ExecutionState getExecutionState() {
+	public ExecutionState2 getExecutionState() {
 		return this.executionState;
 	}
 
@@ -107,61 +112,60 @@ public class TaskExecutionState implements IOReadableWritable {
 		return this.jobID;
 	}
 
+	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public void read(final DataInputView in) throws IOException {
-
-		boolean isNotNull = in.readBoolean();
+	public void read(DataInputView in) throws IOException {
+		this.jobID.read(in);
+		this.executionId.read(in);
+		this.executionState = ExecutionState2.values()[in.readInt()];
 
-		if (isNotNull) {
-			this.jobID = new JobID();
-			this.jobID.read(in);
+		if (in.readBoolean()) {
+			this.description = StringValue.readString(in);
 		} else {
-			this.jobID = null;
+			this.description = null;
 		}
+	}
 
-		isNotNull = in.readBoolean();
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		this.jobID.write(out);
+		this.executionId.write(out);
+		out.writeInt(this.executionState.ordinal());
 
-		// Read the execution vertex ID
-		if (isNotNull) {
-			this.executionVertexID = new ExecutionVertexID();
-			this.executionVertexID.read(in);
+		if (description != null) {
+			out.writeBoolean(true);
+			StringValue.writeString(description, out);
 		} else {
-			this.executionVertexID = null;
+			out.writeBoolean(false);
 		}
-
-		// Read execution state
-		this.executionState = EnumUtils.readEnum(in, ExecutionState.class);
-
-		// Read description
-		this.description = StringRecord.readString(in);
 	}
-
+	
+	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public void write(final DataOutputView out) throws IOException {
-
-		if (this.jobID == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			this.jobID.write(out);
+	public boolean equals(Object obj) {
+		if (obj instanceof TaskExecutionState) {
+			TaskExecutionState other = (TaskExecutionState) obj;
+			return other.jobID.equals(this.jobID) &&
+					other.executionId.equals(this.executionId) &&
+					other.executionState == this.executionState &&
+					(other.description == null ? this.description == null :
+						(this.description != null && other.description.equals(this.description)));
 		}
-
-		// Write the execution vertex ID
-		if (this.executionVertexID == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			this.executionVertexID.write(out);
+		else {
+			return false;
 		}
-
-		// Write execution state
-		EnumUtils.writeEnum(out, this.executionState);
-
-		// Write description
-		StringRecord.writeString(out, this.description);
-
 	}
-
+	
+	@Override
+	public int hashCode() {
+		return jobID.hashCode() + executionId.hashCode() + executionState.ordinal();
+	}
+	
+	@Override
+	public String toString() {
+		return String.format("TaskState jobId=%s, executionId=%s, state=%s, description=%s", 
+				jobID, executionId, executionState, description == null ? "(null)" : description);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
index 1adb3a7..5a38691 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskInputSplitProvider.java
@@ -16,63 +16,37 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.taskmanager;
 
 import java.io.IOException;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.jobmanager.splitassigner.InputSplitWrapper;
 import org.apache.flink.runtime.protocols.InputSplitProviderProtocol;
-import org.apache.flink.runtime.types.IntegerRecord;
-import org.apache.flink.util.StringUtils;
 
-/**
- * The task input split provider is a component of the task manager which implements the {@link InputSplitProvider}
- * interface. It is called by a task in order to acquire a new input split to consume. The task input split provider in
- * return will call the global input split provider to retrieve a new input split.
- * <p>
- * This class is thread-safe.
- * 
- */
 public class TaskInputSplitProvider implements InputSplitProvider {
 
-	private final JobID jobID;
-
-	private final ExecutionVertexID executionVertexID;
-
-	private final InputSplitProviderProtocol globalInputSplitProvider;
-
-	private final AtomicInteger sequenceNumber = new AtomicInteger(0);
-
-	TaskInputSplitProvider(final JobID jobID, final ExecutionVertexID executionVertexID,
-			final InputSplitProviderProtocol globalInputSplitProvider) {
-
-		this.jobID = jobID;
-		this.executionVertexID = executionVertexID;
-		this.globalInputSplitProvider = globalInputSplitProvider;
+	private final InputSplitProviderProtocol protocol;
+	
+	private final JobID jobId;
+	
+	private final JobVertexID vertexId;
+	
+	public TaskInputSplitProvider(InputSplitProviderProtocol protocol, JobID jobId, JobVertexID vertexId) {
+		this.protocol = protocol;
+		this.jobId = jobId;
+		this.vertexId = vertexId;
 	}
 
-
 	@Override
 	public InputSplit getNextInputSplit() {
-
 		try {
-
-			synchronized (this.globalInputSplitProvider) {
-				final InputSplitWrapper wrapper = this.globalInputSplitProvider.requestNextInputSplit(this.jobID,
-					this.executionVertexID, new IntegerRecord(this.sequenceNumber.getAndIncrement()));
-				return wrapper.getInputSplit();
-			}
-
-		} catch (IOException ioe) {
-			// Convert IOException into a RuntimException and let the regular fault tolerance routines take care of the
-			// rest
-			throw new RuntimeException(StringUtils.stringifyException(ioe));
+			return protocol.requestNextInputSplit(jobId, vertexId);
+		}
+		catch (IOException e) {
+			throw new RuntimeException("Requesting the next InputSplit failed.", e);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskKillResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskKillResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskKillResult.java
deleted file mode 100644
index 72c7e98..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskKillResult.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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.taskmanager;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-
-/**
- * A <code>TaskKillResult</code> is used to report the results
- * of a task kill attempt. It contains the ID of the task to be killed, a return code and
- * a description. In case of an error during the kill operation the description includes an error message.
- * 
- */
-public class TaskKillResult extends AbstractTaskResult {
-
-	/**
-	 * Constructs a new task kill result.
-	 * 
-	 * @param vertexID
-	 *        the task ID this result belongs to
-	 * @param returnCode
-	 *        the return code of the kill
-	 */
-	public TaskKillResult(final ExecutionVertexID vertexID, final ReturnCode returnCode) {
-		super(vertexID, returnCode);
-	}
-
-	/**
-	 * Constructs an empty task kill result.
-	 */
-	public TaskKillResult() {
-		super();
-	}
-}


[63/63] [abbrv] git commit: JobManager checks for empty job graphs

Posted by se...@apache.org.
JobManager checks for empty job graphs

Temporarily disabled streaming tests that validate jobgraphs that used to be illegal, but are now legal.


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/91cfbc5a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/91cfbc5a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/91cfbc5a

Branch: refs/heads/master
Commit: 91cfbc5aa4c6d625528f203ca6e0922d9131aafb
Parents: 8411195
Author: Stephan Ewen <se...@apache.org>
Authored: Sat Sep 20 22:00:03 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 22:00:03 2014 +0200

----------------------------------------------------------------------
 .../api/streamcomponent/StreamComponentTest.java       |  2 +-
 .../src/test/resources/logback-test.xml                |  2 +-
 .../apache/flink/runtime/jobmanager/JobManager.java    | 13 +++++++------
 3 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91cfbc5a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
index 9586253..bfc8c7a 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/streamcomponent/StreamComponentTest.java
@@ -81,7 +81,7 @@ public class StreamComponentTest {
 	private static final int SOURCE_PARALELISM = 1;
 	private static final long MEMORYSIZE = 32;
 
-	@Test
+//	@Test
 	public void wrongJobGraph() {
 		LocalStreamEnvironment env = StreamExecutionEnvironment
 				.createLocalEnvironment(SOURCE_PARALELISM);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91cfbc5a/flink-addons/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-addons/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml b/flink-addons/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml
index 45b3b92..4f56748 100644
--- a/flink-addons/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml
+++ b/flink-addons/flink-streaming/flink-streaming-core/src/test/resources/logback-test.xml
@@ -26,5 +26,5 @@
     <root level="WARN">
         <appender-ref ref="STDOUT"/>
     </root>
-    <logger name="org.apache.flink.streaming" level="WARN"/>
+    <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
 </configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/91cfbc5a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index 90bfdc6..b9416cd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -275,17 +275,18 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 
 	@Override
 	public JobSubmissionResult submitJob(JobGraph job) throws IOException {
-		
+		// First check the basics
+		if (job == null) {
+			return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, "Submitted job is null!");
+		}
+		if (job.getNumberOfVertices() == 0) {
+			return new JobSubmissionResult(ReturnCode.ERROR, "Job is empty.");
+		}
 		
 		ExecutionGraph executionGraph = null;
 		boolean success = false;
 		
 		try {
-			// First check if job is null
-			if (job == null) {
-				return new JobSubmissionResult(AbstractJobResult.ReturnCode.ERROR, "Submitted job is null!");
-			}
-	
 			if (LOG.isInfoEnabled()) {
 				LOG.info(String.format("Received job %s (%s)", job.getJobID(), job.getName()));
 			}


[48/63] [abbrv] git commit: Fix logging in EventCollector Fix comparisons (null pointer safe) in JobManagerITCase

Posted by se...@apache.org.
Fix logging in EventCollector
Fix comparisons (null pointer safe) in JobManagerITCase


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/d5d3a080
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/d5d3a080
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/d5d3a080

Branch: refs/heads/master
Commit: d5d3a08031b1eae806c371f5b3fee95991c87f57
Parents: 1fdd7e6
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 03:45:01 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../runtime/jobmanager/EventCollector.java      |  6 +-
 .../runtime/jobmanager/JobManagerITCase.java    | 80 ++++++++++++++------
 2 files changed, 59 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d5d3a080/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
index 551dce2..99b5374 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/EventCollector.java
@@ -26,8 +26,6 @@ import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.flink.runtime.event.job.AbstractEvent;
 import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent;
 import org.apache.flink.runtime.event.job.JobEvent;
@@ -46,6 +44,8 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.archive.ArchiveListener;
 import org.apache.flink.runtime.profiling.ProfilingListener;
 import org.apache.flink.runtime.profiling.types.ProfilingEvent;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * The event collector collects events which occurred during the execution of a job and prepares them
@@ -54,7 +54,7 @@ import org.apache.flink.runtime.profiling.types.ProfilingEvent;
  */
 public final class EventCollector extends TimerTask implements ProfilingListener {
 
-	private static final Log LOG = LogFactory.getLog(EventCollector.class);
+	private static final Logger LOG = LoggerFactory.getLogger(EventCollector.class);
 
 	/**
 	 * The execution listener wrapper is an auxiliary class. It is required

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/d5d3a080/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index a0224de..f4d74a3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -177,13 +177,13 @@ public class JobManagerITCase {
 					}
 					
 					assertTrue("The job did not finish successfully.", success);
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -223,6 +223,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 				
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -231,13 +234,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -282,6 +285,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -290,6 +296,8 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -297,7 +305,6 @@ public class JobManagerITCase {
 				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
-				assertEquals(0, eg.getRegisteredExecutions().size());
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
@@ -340,6 +347,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -348,13 +358,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -372,7 +382,7 @@ public class JobManagerITCase {
 	@Test
 	public void testTwoInputJobFailingEdgeMismatch() {
 		
-		final int NUM_TASKS = 2;
+		final int NUM_TASKS = 11;
 		
 		try {
 			final AbstractJobVertex sender1 = new AbstractJobVertex("Sender1");
@@ -384,15 +394,15 @@ public class JobManagerITCase {
 			receiver.setInvokableClass(AgnosticReceiver.class);
 			
 			sender1.setParallelism(NUM_TASKS);
-			sender2.setParallelism(NUM_TASKS);
-			receiver.setParallelism(NUM_TASKS);
+			sender2.setParallelism(2*NUM_TASKS);
+			receiver.setParallelism(3*NUM_TASKS);
 			
 			receiver.connectNewDataSetAsInput(sender1, DistributionPattern.POINTWISE);
 			receiver.connectNewDataSetAsInput(sender2, DistributionPattern.BIPARTITE);
 			
 			final JobGraph jobGraph = new JobGraph("Bipartite Job", sender1, receiver, sender2);
 			
-			final JobManager jm = startJobManager(3 * NUM_TASKS);
+			final JobManager jm = startJobManager(6*NUM_TASKS);
 			
 			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
 					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
@@ -403,6 +413,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -411,6 +424,8 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -418,7 +433,6 @@ public class JobManagerITCase {
 				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
-				assertEquals(0, eg.getRegisteredExecutions().size());
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
 			}
 			finally {
@@ -465,6 +479,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -473,13 +490,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -526,6 +543,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -534,13 +554,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -587,6 +607,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -595,13 +618,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -647,6 +670,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 				
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -655,13 +681,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -711,6 +737,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -719,13 +748,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());
@@ -775,6 +804,9 @@ public class JobManagerITCase {
 				
 				JobSubmissionResult result = jm.submitJob(jobGraph);
 
+				if (result.getReturnCode() != AbstractJobResult.ReturnCode.SUCCESS) {
+					System.out.println(result.getDescription());
+				}
 				assertEquals(AbstractJobResult.ReturnCode.SUCCESS, result.getReturnCode());
 				
 				// monitor the execution
@@ -783,13 +815,13 @@ public class JobManagerITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FAILED, eg.getState());
+					
+					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
 				}
 				
-				assertEquals(0, eg.getRegisteredExecutions().size());
-				
 				// make sure that in any case, the network buffers are all returned
 				waitForTaskThreadsToBeTerminated();
 				assertEquals(bp.numBuffers(), bp.numAvailableBuffers());


[58/63] [abbrv] git commit: Update web frontend javascript to new states Small fix in integration test

Posted by se...@apache.org.
Update web frontend javascript to new states
Small fix in integration test


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/3e644000
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/3e644000
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/3e644000

Branch: refs/heads/master
Commit: 3e6440008c5e310f0026a92182eb0b3557a13504
Parents: 9803657
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Sep 18 15:20:36 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:02 2014 +0200

----------------------------------------------------------------------
 .../web-docs-infoserver/js/jobmanagerFrontend.js      | 14 +++++---------
 .../flink/runtime/jobmanager/web/JsonFactory.java     |  2 +-
 .../jobmanager/CoLocationConstraintITCase.java        |  2 --
 3 files changed, 6 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3e644000/flink-runtime/resources/web-docs-infoserver/js/jobmanagerFrontend.js
----------------------------------------------------------------------
diff --git a/flink-runtime/resources/web-docs-infoserver/js/jobmanagerFrontend.js b/flink-runtime/resources/web-docs-infoserver/js/jobmanagerFrontend.js
index 5daabe8..c0f54f7 100644
--- a/flink-runtime/resources/web-docs-infoserver/js/jobmanagerFrontend.js
+++ b/flink-runtime/resources/web-docs-infoserver/js/jobmanagerFrontend.js
@@ -194,10 +194,10 @@ function fillTable(table, json) {
 		$.each(job.groupvertices, function(j, groupvertex) {
 			countGroups++;
 			countTasks += groupvertex.numberofgroupmembers;
-			starting = (groupvertex.CREATED + groupvertex.SCHEDULED + groupvertex.ASSIGNED + groupvertex.READY + groupvertex.STARTING);
+			starting = (groupvertex.CREATED + groupvertex.SCHEDULED + groupvertex.DEPLOYING);
 			countStarting += starting;
 			countRunning += groupvertex.RUNNING;
-			countFinished += groupvertex.FINISHING + groupvertex.FINISHED;
+			countFinished += groupvertex.FINISHED;
 			countCanceled += groupvertex.CANCELING + groupvertex.CANCELED;
 			countFailed += groupvertex.FAILED;
 			jobtable += "<tr>\
@@ -209,7 +209,7 @@ function fillTable(table, json) {
 							<td class=\"nummembers\">"+ groupvertex.numberofgroupmembers+ "</td>";
 			jobtable += progressBar(groupvertex.numberofgroupmembers, starting, 'starting');
 			jobtable += progressBar(groupvertex.numberofgroupmembers, groupvertex.RUNNING, 'running');
-			jobtable += progressBar(groupvertex.numberofgroupmembers, (groupvertex.FINISHING + groupvertex.FINISHED), 'success finished');
+			jobtable += progressBar(groupvertex.numberofgroupmembers, (groupvertex.FINISHED), 'success finished');
 			jobtable += progressBar(groupvertex.numberofgroupmembers, (groupvertex.CANCELING + groupvertex.CANCELED), 'warning canceled');
 			jobtable += progressBar(groupvertex.numberofgroupmembers, groupvertex.FAILED, 'danger failed');
 			jobtable +=	"</tr><tr>\
@@ -304,18 +304,14 @@ function updateTable(json) {
 		{
 			// not very nice
 			var oldstatus = ""+$("#"+event.vertexid).children(".status").html();
-			if(oldstatus == "CREATED" ||  oldstatus == "SCHEDULED" ||oldstatus == "ASSIGNED" ||oldstatus == "READY" ||oldstatus == "STARTING")
+			if (oldstatus == "CREATED" ||  oldstatus == "SCHEDULED" ||  oldstatus == "DEPLOYING")
 				oldstatus = "starting";
-			else if(oldstatus == "FINISHING")
-				oldstatus = "finished";
 			else if(oldstatus == "CANCELING")
 				oldstatus = "canceled";
 			
 			var newstate = event.newstate;
-			if(newstate == "CREATED" ||  newstate == "SCHEDULED" ||newstate == "ASSIGNED" ||newstate == "READY" ||newstate == "STARTING")
+			if(newstate == "CREATED" ||  newstate == "SCHEDULED" || newstate == "DEPLOYING")
 				newstate = "starting";
-			else if(newstate == "FINISHING")
-				newstate = "finished";
 			else if(newstate == "CANCELING")
 				newstate = "canceled";
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3e644000/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
index 6ac8613..dc1d60a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
@@ -77,7 +77,7 @@ public class JsonFactory {
 			}
 			
 			// Increment state status count
-			Integer count =  stateCounts.get(vertex.getExecutionState()) + new Integer(1);
+			int count =  stateCounts.get(vertex.getExecutionState()) + 1;
 			stateCounts.put(vertex.getExecutionState(), count);
 		}
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/3e644000/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
index 605c49f..8ef76d6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
@@ -89,8 +89,6 @@ public class CoLocationConstraintITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
-					
-					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;


[41/63] [abbrv] git commit: Adjusted job graph generator to new job graph classes

Posted by se...@apache.org.
Adjusted job graph generator to new job graph classes


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/09d1c33c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/09d1c33c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/09d1c33c

Branch: refs/heads/master
Commit: 09d1c33c9016748ee971a1a906b1e2549eb3f3ee
Parents: 25acb6b
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Sep 12 14:57:54 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../plantranslate/NepheleJobGraphGenerator.java | 188 ++++++++-----------
 .../runtime/jobgraph/AbstractJobVertex.java     |  15 +-
 2 files changed, 88 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/09d1c33c/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index ad6bb70..1783e1d 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -56,18 +56,18 @@ import org.apache.flink.compiler.plan.WorksetPlanNode;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.iterative.convergence.WorksetEmptyConvergenceCriterion;
 import org.apache.flink.runtime.iterative.io.FakeOutputTask;
 import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationSynchronizationSinkTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
-import org.apache.flink.runtime.jobgraph.AbstractJobOutputVertex;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
+import org.apache.flink.runtime.jobgraph.OutputFormatVertex;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetFirstDriver;
 import org.apache.flink.runtime.operators.CoGroupWithSolutionSetSecondDriver;
@@ -106,8 +106,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	
 	// ------------------------------------------------------------------------
 
-	private JobGraph jobGraph; // the job that is currently built
-
 	private Map<PlanNode, AbstractJobVertex> vertices; // a map from optimizer nodes to nephele vertices
 	
 	private Map<PlanNode, TaskInChain> chainedTasks; // a map from optimizer nodes to nephele vertices
@@ -117,8 +115,6 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	private List<TaskInChain> chainedTasksInSequence;
 	
 	private List<AbstractJobVertex> auxVertices; // auxiliary vertices which are added during job graph generation
-
-	private AbstractJobVertex maxDegreeVertex; // the vertex with the highest degree of parallelism
 	
 	private final int defaultMaxFan;
 	
@@ -155,13 +151,11 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	 * @return JobGraph generated from PACT plan.
 	 */
 	public JobGraph compileJobGraph(OptimizedPlan program) {
-		this.jobGraph = new JobGraph(program.getJobName());
 		this.vertices = new HashMap<PlanNode, AbstractJobVertex>();
 		this.chainedTasks = new HashMap<PlanNode, TaskInChain>();
 		this.chainedTasksInSequence = new ArrayList<TaskInChain>();
 		this.auxVertices = new ArrayList<AbstractJobVertex>();
 		this.iterations = new HashMap<IterationPlanNode, IterationDescriptor>();
-		this.maxDegreeVertex = null;
 		
 		// generate Nephele job graph
 		program.accept(this);
@@ -184,33 +178,36 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			TaskConfig t = new TaskConfig(tic.getContainingVertex().getConfiguration());
 			t.addChainedTask(tic.getChainedTask(), tic.getTaskConfig(), tic.getTaskName());
 		}
-
+		
+		// create the jobgraph object
+		JobGraph graph = new JobGraph(program.getJobName());
+		graph.setAllowQueuedScheduling(false);
+		
+		// all vertices share the same slot sharing group, for now
+		SlotSharingGroup sharingGroup = new SlotSharingGroup();
+		
+		// add vertices to the graph
 		for (AbstractJobVertex vertex : this.vertices.values()) {
-			if (vertex != this.maxDegreeVertex) {
-				vertex.setVertexToShareInstancesWith(this.maxDegreeVertex);
-			}
+			graph.addVertex(vertex);
+			vertex.setSlotSharingGroup(sharingGroup);
 		}
 		
 		for (AbstractJobVertex vertex : this.auxVertices) {
-			if (vertex != this.maxDegreeVertex) {
-				vertex.setVertexToShareInstancesWith(this.maxDegreeVertex);
-			}
+			graph.addVertex(vertex);
+			vertex.setSlotSharingGroup(sharingGroup);
 		}
-
+		
 		// add registered cache file into job configuration
 		for (Entry<String, DistributedCacheEntry> e : program.getOriginalPactPlan().getCachedFiles()) {
-			DistributedCache.writeFileInfoToConfig(e.getKey(), e.getValue(), this.jobGraph.getJobConfiguration());
+			DistributedCache.writeFileInfoToConfig(e.getKey(), e.getValue(), graph.getJobConfiguration());
 		}
-		JobGraph graph = this.jobGraph;
-
+		
 		// release all references again
-		this.maxDegreeVertex = null;
 		this.vertices = null;
 		this.chainedTasks = null;
 		this.chainedTasksInSequence = null;
 		this.auxVertices = null;
 		this.iterations = null;
-		this.jobGraph = null;
 
 		// return job graph
 		return graph;
@@ -347,12 +344,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		if (vertex != null) {
 			// set degree of parallelism
 			int pd = node.getDegreeOfParallelism();
-			vertex.setNumberOfSubtasks(pd);
-	
-			// check whether this is the vertex with the highest degree of parallelism
-			if (this.maxDegreeVertex == null || this.maxDegreeVertex.getNumberOfSubtasks() < pd) {
-				this.maxDegreeVertex = vertex;
-			}
+			vertex.setParallelism(pd);
 			
 			// check whether this vertex is part of an iteration step function
 			if (this.currentIteration != null) {
@@ -651,7 +643,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			if (inConn.isOnDynamicPath()) {
 				numChannelsDynamicPath++;
 				numDynamicSenderTasksTotal += getNumberOfSendersPerReceiver(pattern,
-					sourceVertex.getNumberOfSubtasks(), targetVertex.getNumberOfSubtasks());
+					sourceVertex.getParallelism(), targetVertex.getParallelism());
 			}
 		}
 		
@@ -702,7 +694,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	// Methods for creating individual vertices
 	// ------------------------------------------------------------------------
 	
-	private JobTaskVertex createSingleInputVertex(SingleInputPlanNode node) throws CompilerException {
+	private AbstractJobVertex createSingleInputVertex(SingleInputPlanNode node) throws CompilerException {
 		final String taskName = node.getNodeName();
 		final DriverStrategy ds = node.getDriverStrategy();
 		
@@ -745,7 +737,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			}
 		}
 		
-		final JobTaskVertex vertex;
+		final AbstractJobVertex vertex;
 		final TaskConfig config;
 		
 		if (chaining) {
@@ -754,7 +746,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			this.chainedTasks.put(node, new TaskInChain(ds.getPushChainDriverClass(), config, taskName));
 		} else {
 			// create task vertex
-			vertex = new JobTaskVertex(taskName, this.jobGraph);
+			vertex = new AbstractJobVertex(taskName);
 			vertex.setInvokableClass((this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
 			
 			config = new TaskConfig(vertex.getConfiguration());
@@ -776,10 +768,10 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		return vertex;
 	}
 
-	private JobTaskVertex createDualInputVertex(DualInputPlanNode node) throws CompilerException {
+	private AbstractJobVertex createDualInputVertex(DualInputPlanNode node) throws CompilerException {
 		final String taskName = node.getNodeName();
 		final DriverStrategy ds = node.getDriverStrategy();
-		final JobTaskVertex vertex = new JobTaskVertex(taskName, this.jobGraph);
+		final AbstractJobVertex vertex = new AbstractJobVertex(taskName);
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 		vertex.setInvokableClass( (this.currentIteration != null && node.isOnDynamicPath()) ? IterationIntermediatePactTask.class : RegularPactTask.class);
 		
@@ -806,7 +798,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	}
 
 	private InputFormatVertex createDataSourceVertex(SourcePlanNode node) throws CompilerException {
-		final InputFormatVertex vertex = new InputFormatVertex(node.getNodeName(), this.jobGraph);
+		final InputFormatVertex vertex = new InputFormatVertex(node.getNodeName());
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
 		vertex.setInvokableClass(DataSourceTask.class);
@@ -819,8 +811,8 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		return vertex;
 	}
 
-	private AbstractJobOutputVertex createDataSinkVertex(SinkPlanNode node) throws CompilerException {
-		final OutputFormatVertex vertex = new OutputFormatVertex(node.getNodeName(), this.jobGraph);
+	private AbstractJobVertex createDataSinkVertex(SinkPlanNode node) throws CompilerException {
+		final OutputFormatVertex vertex = new OutputFormatVertex(node.getNodeName());
 		final TaskConfig config = new TaskConfig(vertex.getConfiguration());
 
 		vertex.setInvokableClass(DataSinkTask.class);
@@ -833,7 +825,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		return vertex;
 	}
 	
-	private JobTaskVertex createBulkIterationHead(BulkPartialSolutionPlanNode pspn) {
+	private AbstractJobVertex createBulkIterationHead(BulkPartialSolutionPlanNode pspn) {
 		// get the bulk iteration that corresponds to this partial solution node
 		final BulkIterationPlanNode iteration = pspn.getContainingIterationNode();
 		
@@ -864,12 +856,12 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		}
 		
 		// create or adopt the head vertex
-		final JobTaskVertex toReturn;
-		final JobTaskVertex headVertex;
+		final AbstractJobVertex toReturn;
+		final AbstractJobVertex headVertex;
 		final TaskConfig headConfig;
 		if (merge) {
 			final PlanNode successor = pspn.getOutgoingChannels().get(0).getTarget();
-			headVertex = (JobTaskVertex) this.vertices.get(successor);
+			headVertex = (AbstractJobVertex) this.vertices.get(successor);
 			
 			if (headVertex == null) {
 				throw new CompilerException(
@@ -884,7 +876,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			// instantiate the head vertex and give it a no-op driver as the driver strategy.
 			// everything else happens in the post visit, after the input (the initial partial solution)
 			// is connected.
-			headVertex = new JobTaskVertex("PartialSolution ("+iteration.getNodeName()+")", this.jobGraph);
+			headVertex = new AbstractJobVertex("PartialSolution ("+iteration.getNodeName()+")");
 			headVertex.setInvokableClass(IterationHeadPactTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			headConfig.setDriver(NoOpDriver.class);
@@ -901,7 +893,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		return toReturn;
 	}
 	
-	private JobTaskVertex createWorksetIterationHead(WorksetPlanNode wspn) {
+	private AbstractJobVertex createWorksetIterationHead(WorksetPlanNode wspn) {
 		// get the bulk iteration that corresponds to this partial solution node
 		final WorksetIterationPlanNode iteration = wspn.getContainingIterationNode();
 		
@@ -932,12 +924,12 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		}
 		
 		// create or adopt the head vertex
-		final JobTaskVertex toReturn;
-		final JobTaskVertex headVertex;
+		final AbstractJobVertex toReturn;
+		final AbstractJobVertex headVertex;
 		final TaskConfig headConfig;
 		if (merge) {
 			final PlanNode successor = wspn.getOutgoingChannels().get(0).getTarget();
-			headVertex = (JobTaskVertex) this.vertices.get(successor);
+			headVertex = (AbstractJobVertex) this.vertices.get(successor);
 			
 			if (headVertex == null) {
 				throw new CompilerException(
@@ -952,7 +944,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			// instantiate the head vertex and give it a no-op driver as the driver strategy.
 			// everything else happens in the post visit, after the input (the initial partial solution)
 			// is connected.
-			headVertex = new JobTaskVertex("IterationHead("+iteration.getNodeName()+")", this.jobGraph);
+			headVertex = new AbstractJobVertex("IterationHead("+iteration.getNodeName()+")");
 			headVertex.setInvokableClass(IterationHeadPactTask.class);
 			headConfig = new TaskConfig(headVertex.getConfiguration());
 			headConfig.setDriver(NoOpDriver.class);
@@ -1002,35 +994,33 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	 * @param targetVertex
 	 * @param targetConfig
 	 * @param isBroadcast
-	 * @throws JobGraphDefinitionException
 	 * @throws CompilerException
 	 */
 	private DistributionPattern connectJobVertices(Channel channel, int inputNumber,
 			final AbstractJobVertex sourceVertex, final TaskConfig sourceConfig,
 			final AbstractJobVertex targetVertex, final TaskConfig targetConfig, boolean isBroadcast)
-	throws JobGraphDefinitionException, CompilerException
+	throws CompilerException
 	{
 		// ------------ connect the vertices to the job graph --------------
-		final ChannelType channelType;
 		final DistributionPattern distributionPattern;
 
 		switch (channel.getShipStrategy()) {
 			case FORWARD:
 				distributionPattern = DistributionPattern.POINTWISE;
-				channelType = ChannelType.NETWORK;
 				break;
 			case PARTITION_RANDOM:
 			case BROADCAST:
 			case PARTITION_HASH:
 			case PARTITION_RANGE:
 				distributionPattern = DistributionPattern.BIPARTITE;
-				channelType = ChannelType.NETWORK;
 				break;
 			default:
 				throw new RuntimeException("Unknown runtime ship strategy: " + channel.getShipStrategy());
 		}
 		
-		sourceVertex.connectTo(targetVertex, channelType, distributionPattern);
+		targetVertex.connectNewDataSetAsInput(sourceVertex, distributionPattern);
+		
+//		sourceVertex.conn/ectTo(targetVertex, channelType, distributionPattern);
 
 		// -------------- configure the source task's ship strategy strategies in task config --------------
 		final int outputIndex = sourceConfig.getNumOutputs();
@@ -1119,7 +1109,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	private void finalizeBulkIteration(IterationDescriptor descr) {
 		
 		final BulkIterationPlanNode bulkNode = (BulkIterationPlanNode) descr.getIterationNode();
-		final JobTaskVertex headVertex = descr.getHeadTask();
+		final AbstractJobVertex headVertex = descr.getHeadTask();
 		final TaskConfig headConfig = new TaskConfig(headVertex.getConfiguration());
 		final TaskConfig headFinalOutputConfig = descr.getHeadFinalResultConfig();
 		
@@ -1135,13 +1125,13 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		headConfig.setRelativeBackChannelMemory(relativeMemForBackChannel);
 		
 		// --------------------------- create the sync task ---------------------------
-		final SimpleOutputVertex sync = new SimpleOutputVertex("Sync(" + bulkNode.getNodeName() + ")", this.jobGraph);
+		final AbstractJobVertex sync = new AbstractJobVertex("Sync(" + bulkNode.getNodeName() + ")");
 		sync.setInvokableClass(IterationSynchronizationSinkTask.class);
-		sync.setNumberOfSubtasks(1);
+		sync.setParallelism(1);
 		this.auxVertices.add(sync);
 		
 		final TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
-		syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, headVertex.getNumberOfSubtasks());
+		syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, headVertex.getParallelism());
 
 		// set the number of iteration / convergence criterion for the sync
 		final int maxNumIterations = bulkNode.getIterationNode().getIterationContract().getMaximumNumberOfIterations();
@@ -1151,12 +1141,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		syncConfig.setNumberOfIterations(maxNumIterations);
 		
 		// connect the sync task
-		try {
-			headVertex.connectTo(sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
-		} catch (JobGraphDefinitionException e) {
-			throw new CompilerException("Bug: Cannot connect head vertex to sync task.");
-		}
-		
+		sync.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE);
 		
 		// ----------------------------- create the iteration tail ------------------------------
 		
@@ -1164,14 +1149,14 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		final PlanNode rootOfStepFunction = bulkNode.getRootOfStepFunction();
 		final TaskConfig tailConfig;
 		
-		JobTaskVertex rootOfStepFunctionVertex = (JobTaskVertex) this.vertices.get(rootOfStepFunction);
+		AbstractJobVertex rootOfStepFunctionVertex = (AbstractJobVertex) this.vertices.get(rootOfStepFunction);
 		if (rootOfStepFunctionVertex == null) {
 			// last op is chained
 			final TaskInChain taskInChain = this.chainedTasks.get(rootOfStepFunction);
 			if (taskInChain == null) {
 				throw new CompilerException("Bug: Tail of step function not found as vertex or chained task.");
 			}
-			rootOfStepFunctionVertex = (JobTaskVertex) taskInChain.getContainingVertex();
+			rootOfStepFunctionVertex = (AbstractJobVertex) taskInChain.getContainingVertex();
 
 			// the fake channel is statically typed to pact record. no data is sent over this channel anyways.
 			tailConfig = taskInChain.getTaskConfig();
@@ -1190,18 +1175,13 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			tailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 			
 			// create the fake output task
-			SimpleOutputVertex fakeTail = new SimpleOutputVertex("Fake Tail", this.jobGraph);
+			AbstractJobVertex fakeTail = new AbstractJobVertex("Fake Tail");
 			fakeTail.setInvokableClass(FakeOutputTask.class);
-			fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
+			fakeTail.setParallelism(headVertex.getParallelism());
 			this.auxVertices.add(fakeTail);
 			
 			// connect the fake tail
-			try {
-				rootOfStepFunctionVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-			} catch (JobGraphDefinitionException e) {
-				throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
-			}
-			
+			fakeTail.connectNewDataSetAsInput(rootOfStepFunctionVertex, DistributionPattern.POINTWISE);
 		}
 		
 		
@@ -1209,7 +1189,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		final TaskConfig tailConfigOfTerminationCriterion;
 		// If we have a termination criterion and it is not an intermediate node
 		if(rootOfTerminationCriterion != null && rootOfTerminationCriterion.getOutgoingChannels().isEmpty()) {
-			JobTaskVertex rootOfTerminationCriterionVertex = (JobTaskVertex) this.vertices.get(rootOfTerminationCriterion);
+			AbstractJobVertex rootOfTerminationCriterionVertex = (AbstractJobVertex) this.vertices.get(rootOfTerminationCriterion);
 			
 			
 			if (rootOfTerminationCriterionVertex == null) {
@@ -1218,7 +1198,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				if (taskInChain == null) {
 					throw new CompilerException("Bug: Tail of termination criterion not found as vertex or chained task.");
 				}
-				rootOfTerminationCriterionVertex = (JobTaskVertex) taskInChain.getContainingVertex();
+				rootOfTerminationCriterionVertex = (AbstractJobVertex) taskInChain.getContainingVertex();
 
 				// the fake channel is statically typed to pact record. no data is sent over this channel anyways.
 				tailConfigOfTerminationCriterion = taskInChain.getTaskConfig();
@@ -1232,17 +1212,13 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			tailConfigOfTerminationCriterion.setOutputSerializer(bulkNode.getSerializerForIterationChannel());
 			tailConfigOfTerminationCriterion.addOutputShipStrategy(ShipStrategyType.FORWARD);
 			
-			SimpleOutputVertex fakeTailTerminationCriterion = new SimpleOutputVertex("Fake Tail for Termination Criterion", this.jobGraph);
+			AbstractJobVertex fakeTailTerminationCriterion = new AbstractJobVertex("Fake Tail for Termination Criterion");
 			fakeTailTerminationCriterion.setInvokableClass(FakeOutputTask.class);
-			fakeTailTerminationCriterion.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
+			fakeTailTerminationCriterion.setParallelism(headVertex.getParallelism());
 			this.auxVertices.add(fakeTailTerminationCriterion);
 		
 			// connect the fake tail
-			try {
-				rootOfTerminationCriterionVertex.connectTo(fakeTailTerminationCriterion, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-			} catch (JobGraphDefinitionException e) {
-				throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task for termination criterion");
-			}
+			fakeTailTerminationCriterion.connectNewDataSetAsInput(rootOfTerminationCriterionVertex, DistributionPattern.POINTWISE);
 			
 			// tell the head that it needs to wait for the solution set updates
 			headConfig.setWaitForSolutionSetUpdate();
@@ -1272,7 +1248,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	
 	private void finalizeWorksetIteration(IterationDescriptor descr) {
 		final WorksetIterationPlanNode iterNode = (WorksetIterationPlanNode) descr.getIterationNode();
-		final JobTaskVertex headVertex = descr.getHeadTask();
+		final AbstractJobVertex headVertex = descr.getHeadTask();
 		final TaskConfig headConfig = new TaskConfig(headVertex.getConfiguration());
 		final TaskConfig headFinalOutputConfig = descr.getHeadFinalResultConfig();
 		
@@ -1299,13 +1275,13 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		// --------------------------- create the sync task ---------------------------
 		final TaskConfig syncConfig;
 		{
-			final SimpleOutputVertex sync = new SimpleOutputVertex("Sync (" + iterNode.getNodeName() + ")", this.jobGraph);
+			final AbstractJobVertex sync = new AbstractJobVertex("Sync (" + iterNode.getNodeName() + ")");
 			sync.setInvokableClass(IterationSynchronizationSinkTask.class);
-			sync.setNumberOfSubtasks(1);
+			sync.setParallelism(1);
 			this.auxVertices.add(sync);
 			
 			syncConfig = new TaskConfig(sync.getConfiguration());
-			syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, headVertex.getNumberOfSubtasks());
+			syncConfig.setGateIterativeWithNumberOfEventsUntilInterrupt(0, headVertex.getParallelism());
 	
 			// set the number of iteration / convergence criterion for the sync
 			final int maxNumIterations = iterNode.getIterationNode().getIterationContract().getMaximumNumberOfIterations();
@@ -1315,11 +1291,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			syncConfig.setNumberOfIterations(maxNumIterations);
 			
 			// connect the sync task
-			try {
-				headVertex.connectTo(sync, ChannelType.NETWORK, DistributionPattern.POINTWISE);
-			} catch (JobGraphDefinitionException e) {
-				throw new CompilerException("Bug: Cannot connect head vertex to sync task.");
-			}
+			sync.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE);
 		}
 		
 		// ----------------------------- create the iteration tails -----------------------------
@@ -1340,14 +1312,14 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			{
 				// get the vertex for the workset update
 				final TaskConfig worksetTailConfig;
-				JobTaskVertex nextWorksetVertex = (JobTaskVertex) this.vertices.get(nextWorksetNode);
+				AbstractJobVertex nextWorksetVertex = (AbstractJobVertex) this.vertices.get(nextWorksetNode);
 				if (nextWorksetVertex == null) {
 					// nextWorksetVertex is chained
 					TaskInChain taskInChain = this.chainedTasks.get(nextWorksetNode);
 					if (taskInChain == null) {
 						throw new CompilerException("Bug: Next workset node not found as vertex or chained task.");
 					}
-					nextWorksetVertex = (JobTaskVertex) taskInChain.getContainingVertex();
+					nextWorksetVertex = (AbstractJobVertex) taskInChain.getContainingVertex();
 					worksetTailConfig = taskInChain.getTaskConfig();
 				} else {
 					worksetTailConfig = new TaskConfig(nextWorksetVertex.getConfiguration());
@@ -1364,29 +1336,25 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					worksetTailConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 					
 					// create the fake output task
-					SimpleOutputVertex fakeTail = new SimpleOutputVertex("Fake Tail", this.jobGraph);
+					AbstractJobVertex fakeTail = new AbstractJobVertex("Fake Tail");
 					fakeTail.setInvokableClass(FakeOutputTask.class);
-					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
+					fakeTail.setParallelism(headVertex.getParallelism());
 					this.auxVertices.add(fakeTail);
 					
 					// connect the fake tail
-					try {
-						nextWorksetVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-					} catch (JobGraphDefinitionException e) {
-						throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
-					}
+					fakeTail.connectNewDataSetAsInput(nextWorksetVertex, DistributionPattern.POINTWISE);
 				}
 			}
 			{
 				final TaskConfig solutionDeltaConfig;
-				JobTaskVertex solutionDeltaVertex = (JobTaskVertex) this.vertices.get(solutionDeltaNode);
+				AbstractJobVertex solutionDeltaVertex = (AbstractJobVertex) this.vertices.get(solutionDeltaNode);
 				if (solutionDeltaVertex == null) {
 					// last op is chained
 					TaskInChain taskInChain = this.chainedTasks.get(solutionDeltaNode);
 					if (taskInChain == null) {
 						throw new CompilerException("Bug: Solution Set Delta not found as vertex or chained task.");
 					}
-					solutionDeltaVertex = (JobTaskVertex) taskInChain.getContainingVertex();
+					solutionDeltaVertex = (AbstractJobVertex) taskInChain.getContainingVertex();
 					solutionDeltaConfig = taskInChain.getTaskConfig();
 				} else {
 					solutionDeltaConfig = new TaskConfig(solutionDeltaVertex.getConfiguration());
@@ -1402,17 +1370,13 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 					solutionDeltaConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
 	
 					// create the fake output task
-					SimpleOutputVertex fakeTail = new SimpleOutputVertex("Fake Tail", this.jobGraph);
+					AbstractJobVertex fakeTail = new AbstractJobVertex("Fake Tail");
 					fakeTail.setInvokableClass(FakeOutputTask.class);
-					fakeTail.setNumberOfSubtasks(headVertex.getNumberOfSubtasks());
+					fakeTail.setParallelism(headVertex.getParallelism());
 					this.auxVertices.add(fakeTail);
 					
 					// connect the fake tail
-					try {
-						solutionDeltaVertex.connectTo(fakeTail, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-					} catch (JobGraphDefinitionException e) {
-						throw new CompilerException("Bug: Cannot connect iteration tail vertex fake tail task");
-					}
+					fakeTail.connectNewDataSetAsInput(solutionDeltaVertex, DistributionPattern.POINTWISE);
 					
 					// tell the head that it needs to wait for the solution set updates
 					headConfig.setWaitForSolutionSetUpdate();
@@ -1503,7 +1467,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 		
 		private final IterationPlanNode iterationNode;
 		
-		private JobTaskVertex headTask;
+		private AbstractJobVertex headTask;
 		
 		private TaskConfig headConfig;
 		
@@ -1520,7 +1484,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			return iterationNode;
 		}
 		
-		public void setHeadTask(JobTaskVertex headTask, TaskConfig headConfig) {
+		public void setHeadTask(AbstractJobVertex headTask, TaskConfig headConfig) {
 			this.headTask = headTask;
 			this.headFinalResultConfig = new TaskConfig(new Configuration());
 			
@@ -1532,7 +1496,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			this.headConfig = headConfig;
 		}
 		
-		public JobTaskVertex getHeadTask() {
+		public AbstractJobVertex getHeadTask() {
 			return headTask;
 		}
 		

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/09d1c33c/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index cc2cbd8..82823b2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -44,9 +44,6 @@ public class AbstractJobVertex implements java.io.Serializable {
 	/** The ID of the vertex. */
 	private final JobVertexID id;
 
-	/** The name of the vertex */
-	private final String name;
-
 	/** List of produced data sets, one per writer */
 	private final ArrayList<IntermediateDataSet> results = new ArrayList<IntermediateDataSet>();
 
@@ -67,6 +64,9 @@ public class AbstractJobVertex implements java.io.Serializable {
 	
 	/** Optionally, a sharing group that allows subtasks from different job vertices to run concurrently in one slot */
 	private SlotSharingGroup slotSharingGroup;
+	
+	/** The name of the vertex */
+	private String name;
 
 	// --------------------------------------------------------------------------------------------
 
@@ -109,6 +109,15 @@ public class AbstractJobVertex implements java.io.Serializable {
 	public String getName() {
 		return this.name;
 	}
+	
+	/**
+	 * Sets the name of the vertex
+	 * 
+	 * @param name The new name.
+	 */
+	public void setName(String name) {
+		this.name = name == null ? DEFAULT_NAME : name;
+	}
 
 	/**
 	 * Returns the number of produced intermediate data sets.


[31/63] [abbrv] git commit: Redesign Scheduler from pre-assignment to more flexible schedule-on-demand model

Posted by se...@apache.org.
Redesign Scheduler from pre-assignment to more flexible schedule-on-demand model


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/2d6199ff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/2d6199ff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/2d6199ff

Branch: refs/heads/master
Commit: 2d6199fff877b0532903a4b2ff2d5279671b33cb
Parents: c32569a
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Jul 20 13:11:23 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../flink/configuration/ConfigConstants.java    |    4 +-
 .../api/common/io/FileInputFormatTest.java      |    1 -
 .../runtime/executiongraph/ExecutionVertex.java |    4 +
 .../executiongraph/ExecutionVertex2.java        |   25 +
 .../runtime/instance/AllocatedResource.java     |  163 --
 .../flink/runtime/instance/AllocatedSlot.java   |   73 +-
 .../instance/DefaultInstanceManager.java        |  134 +-
 .../flink/runtime/instance/DummyInstance.java   |   64 -
 .../runtime/instance/HardwareDescription.java   |   10 +
 .../apache/flink/runtime/instance/Instance.java |  378 ++---
 .../instance/InstanceConnectionInfo.java        |    2 +-
 .../runtime/instance/InstanceDiedException.java |   34 +
 .../runtime/instance/InstanceException.java     |   42 -
 .../runtime/instance/InstanceListener.java      |   33 +-
 .../runtime/instance/InstanceNotifier.java      |   77 -
 .../scheduler/DefaultExecutionListener.java     |  133 --
 .../jobmanager/scheduler/DefaultScheduler.java  | 1568 ++++++++++--------
 .../scheduler/InstanceFillDegreeComparator.java |   31 +
 .../jobmanager/scheduler/LifoSetQueue.java      |  110 ++
 .../scheduler/NoResourceAvailableException.java |   33 +
 .../jobmanager/scheduler/ResourceId.java        |   20 +
 .../jobmanager/scheduler/ScheduledUnit.java     |   67 +
 .../scheduler/SchedulingException.java          |   44 -
 .../scheduler/SchedulingStrategy.java           |   33 +
 .../runtime/protocols/JobManagerProtocol.java   |   22 +-
 .../flink/runtime/taskmanager/TaskManager.java  |  370 +++--
 .../RegisterTaskManagerResult.java              |   56 -
 .../instance/LocalInstanceManagerTest.java      |   11 -
 .../scheduler/DefaultSchedulerTest.java         |  298 ++--
 .../jobmanager/scheduler/LifoSetQueueTest.java  |  128 ++
 .../scheduler/TestDeploymentManager.java        |  108 --
 .../scheduler/TestInstanceManager.java          |  194 ---
 .../splitassigner/DefaultSplitAssignerTest.java |   25 +-
 .../LocatableSplitAssignerTest.java             |   25 +-
 34 files changed, 2081 insertions(+), 2239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 4fba186..b2af886 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -384,9 +384,9 @@ public final class ConfigConstants {
 	public static final int DEFAULT_TASK_MANAGER_NET_NETTY_HIGH_WATER_MARK = -1;
 
 	/**
-	 * The default interval for TaskManager heart beats (2000 msecs).
+	 * The default interval for TaskManager heart beats (5000 msecs).
 	 */
-	public static final int DEFAULT_TASK_MANAGER_HEARTBEAT_INTERVAL = 2000;
+	public static final int DEFAULT_TASK_MANAGER_HEARTBEAT_INTERVAL = 5000;
 
 	/**
 	 * Flag indicating whether to start a thread, which repeatedly logs the memory usage of the JVM.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java b/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
index 9e93e67..707ecca 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/FileInputFormatTest.java
@@ -31,7 +31,6 @@ import org.apache.flink.core.fs.FileInputSplit;
 import org.apache.flink.testutils.TestFileUtils;
 import org.apache.flink.types.IntValue;
 import org.junit.Assert;
-import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class FileInputFormatTest { 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index 72e0696..d1ee262 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -1000,4 +1000,8 @@ public final class ExecutionVertex {
 
 		return tdd;
 	}
+	
+	public void handleException(Throwable t) {
+		
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
new file mode 100644
index 0000000..ab33ca0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
@@ -0,0 +1,25 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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;
+
+public class ExecutionVertex2 {
+
+	
+	
+	public void handleException(Throwable t) {
+		t.printStackTrace();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedResource.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedResource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedResource.java
deleted file mode 100644
index c626309..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedResource.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/**
- * 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.instance;
-
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-
-/**
- * An allocated resource object unambiguously defines the
- * hardware resources which have been assigned to an {@link org.apache.flink.runtime.executiongraph.ExecutionVertex} for
- * executing a task. The allocated resource is comprised of an {@link Instance}
- * which identifies the node the task is scheduled to run on as well as an
- * {@link org.apache.flink.runtime.instance.AllocationID} which determines the resources the task is scheduled to
- * allocate within the node.
- * <p>
- * The class is thread-safe.
- * 
- */
-public final class AllocatedResource {
-
-	/**
-	 * The instance a task is scheduled to run on.
-	 */
-	private final Instance instance;
-
-	/**
-	 * The allocation ID identifying the resources within the instance
-	 * which the task is expected to run on.
-	 */
-	private final AllocationID allocationID;
-
-	/**
-	 * The set stores the execution vertices which are currently scheduled to run this resource.
-	 */
-	private final Set<ExecutionVertex> assignedVertices = Collections
-		.newSetFromMap(new ConcurrentHashMap<ExecutionVertex, Boolean>());
-
-	/**
-	 * Constructs a new allocated resource object.
-	 * 
-	 * @param instance
-	 *        the instance a task is scheduled to run on.
-	 * @param allocationID
-	 *        the allocation ID identifying the allocated resources within the instance
-	 */
-	public AllocatedResource(final Instance instance, final AllocationID allocationID) {
-		this.instance = instance;
-		this.allocationID = allocationID;
-	}
-
-	/**
-	 * Returns the instance a task is scheduled to run on.
-	 *
-	 * @return the instance a task is scheduled to run on
-	 */
-	public Instance getInstance() {
-		return this.instance;
-	}
-
-	/**
-	 * Returns the allocation ID which identifies the resource allocated within the assigned instance.
-	 * 
-	 * @return the allocation ID or <code>null</code> if the assigned instance is of type {@link DummyInstance}
-	 */
-	public AllocationID getAllocationID() {
-		return this.allocationID;
-	}
-
-
-	@Override
-	public boolean equals(final Object obj) {
-
-		if (obj instanceof AllocatedResource) {
-
-			final AllocatedResource allocatedResource = (AllocatedResource) obj;
-			if (!this.instance.equals(allocatedResource.getInstance())) {
-				return false;
-			}
-
-			if (this.allocationID == null) {
-				if (allocatedResource.getAllocationID() != null) {
-					return false;
-				}
-			} else {
-				if (!this.allocationID.equals(allocatedResource.getAllocationID())) {
-					return false;
-				}
-			}
-
-			return true;
-		}
-
-		return false;
-	}
-
-
-	@Override
-	public int hashCode() {
-
-		if (this.allocationID == null) {
-			return 0;
-		}
-
-		return this.allocationID.hashCode();
-	}
-
-	/**
-	 * Assigns the given execution vertex to this allocated resource.
-	 * 
-	 * @param vertex
-	 *        the vertex to assign to this resource
-	 */
-	public void assignVertexToResource(final ExecutionVertex vertex) {
-
-		if (!this.assignedVertices.add(vertex)) {
-			throw new IllegalStateException("The vertex " + vertex + " has already been assigned to resource " + this);
-		}
-	}
-
-	/**
-	 * Returns an iterator over all execution vertices currently assigned to this allocated resource.
-	 * 
-	 * @return an iterator over all execution vertices currently assigned to this allocated resource
-	 */
-	public Iterator<ExecutionVertex> assignedVertices() {
-
-		return this.assignedVertices.iterator();
-	}
-
-	/**
-	 * Removes the given execution vertex from this allocated resource.
-	 * 
-	 * @param vertex
-	 *        the execution to be removed
-	 */
-	public void removeVertexFromResource(final ExecutionVertex vertex) {
-
-		if (!this.assignedVertices.remove(vertex)) {
-			throw new IllegalStateException("The vertex " + vertex + " has not been assigned to resource " + this);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
index d85bf39..71af9db 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
@@ -16,56 +16,67 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.instance;
 
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobmanager.scheduler.ResourceId;
 
 /**
- * An allocated slot is a part of an instance which is assigned to a job.
- * <p>
- * This class is thread-safe.
- * 
+ * An allocated slot is the unit in which resources are allocated on instances.
  */
 public class AllocatedSlot {
 
-	/**
-	 * The allocation ID which identifies the resources occupied by this slot.
-	 */
-	private final AllocationID allocationID;
+	/** The ID which identifies the resources occupied by this slot. */
+	private final ResourceId resourceId;
 
-	/**
-	 * The ID of the job this slice belongs to.
-	 */
+	/** The ID of the job this slice belongs to. */
 	private final JobID jobID;
+	
+	/** The instance on which the slot is allocated */
+	private final Instance instance;
+	
+	/** The number of the slot on which the task is deployed */
+	private final int slotNumber;
 
-	/**
-	 * Creates a new allocated slice on the given hosting instance.
-	 * 
-	 * @param jobID
-	 *        the ID of the job this slice belongs to
-	 */
-	public AllocatedSlot(final JobID jobID) {
 
-		this.allocationID = new AllocationID();
+	public AllocatedSlot(JobID jobID, ResourceId resourceId, Instance instance, int slotNumber) {
+		this.resourceId = resourceId;
 		this.jobID = jobID;
+		this.instance = instance;
+		this.slotNumber = slotNumber;
 	}
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * Returns the allocation ID of this slice.
-	 * 
-	 * @return the allocation ID of this slice
-	 */
-	public AllocationID getAllocationID() {
-		return this.allocationID;
-	}
-
-	/**
-	 * Returns the ID of the job this allocated slice belongs to.
+	 * Returns the ID of the job this allocated slot belongs to.
 	 * 
-	 * @return the ID of the job this allocated slice belongs to
+	 * @return the ID of the job this allocated slot belongs to
 	 */
 	public JobID getJobID() {
 		return this.jobID;
 	}
+	
+	public ResourceId getResourceId() {
+		return resourceId;
+	}
+	
+	public Instance getInstance() {
+		return instance;
+	}
+	
+	public int getSlotNumber() {
+		return slotNumber;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void runTask(ExecutionVertex2 vertex) {
+		
+	}
+	
+	public void cancelResource() {
+		
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java
index b19adbb..eca23c4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DefaultInstanceManager.java
@@ -21,14 +21,19 @@ package org.apache.flink.runtime.instance;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.Timer;
 import java.util.TimerTask;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.GlobalConfiguration;
 
@@ -54,6 +59,9 @@ public class DefaultInstanceManager implements InstanceManager {
 	
 	/** Set of hosts that were present once and have died */
 	private final Set<InstanceConnectionInfo> deadHosts;
+	
+	/** Listeners that want to be notified about availability and disappearance of instances */
+	private final List<InstanceListener> instanceListeners = new ArrayList<InstanceListener>();
 
 	/** Duration after which a task manager is considered dead if it did not send a heart-beat message. */
 	private final long heartbeatTimeout;
@@ -106,7 +114,7 @@ public class DefaultInstanceManager implements InstanceManager {
 			this.cleanupStaleMachines.cancel();
 
 			for (Instance i : this.registeredHostsById.values()) {
-				i.destroy();
+				i.markDead();
 			}
 			
 			this.registeredHostsById.clear();
@@ -183,6 +191,9 @@ public class DefaultInstanceManager implements InstanceManager {
 
 			host.reportHeartBeat();
 			
+			// notify all listeners (for example the scheduler)
+			notifyNewInstance(host);
+			
 			return id;
 		}
 	}
@@ -204,48 +215,99 @@ public class DefaultInstanceManager implements InstanceManager {
 	
 	// --------------------------------------------------------------------------------------------
 	
+	public void addInstanceListener(InstanceListener listener) {
+		synchronized (this.instanceListeners) {
+			this.instanceListeners.add(listener);
+		}
+	}
+	
+	public void removeInstanceListener(InstanceListener listener) {
+		synchronized (this.instanceListeners) {
+			this.instanceListeners.remove(listener);
+		}
+	}
+	
+	private void notifyNewInstance(Instance instance) {
+		synchronized (this.instanceListeners) {
+			for (InstanceListener listener : this.instanceListeners) {
+				try {
+					listener.newInstanceAvailable(instance);
+				}
+				catch (Throwable t) {
+					LOG.error("Notification of new instance availability failed.", t);
+				}
+			}
+		}
+	}
+	
+	private void notifyDeadInstance(Instance instance) {
+		synchronized (this.instanceListeners) {
+			for (InstanceListener listener : this.instanceListeners) {
+				try {
+					listener.instanceDied(instance);
+				}
+				catch (Throwable t) {
+					LOG.error("Notification of dead instance failed.", t);
+				}
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private void checkForDeadInstances() {
+		final long now = System.currentTimeMillis();
+		final long timeout = DefaultInstanceManager.this.heartbeatTimeout;
+		
+		synchronized (DefaultInstanceManager.this.lock) {
+			if (DefaultInstanceManager.this.shutdown) {
+				return;
+			}
+
+			final Iterator<Map.Entry<InstanceID, Instance>> entries = registeredHostsById.entrySet().iterator();
+			
+			// check all hosts whether they did not send heart-beat messages.
+			while (entries.hasNext()) {
+				
+				final Map.Entry<InstanceID, Instance> entry = entries.next();
+				final Instance host = entry.getValue();
+				
+				if (!host.isStillAlive(now, timeout)) {
+					
+					// remove from the living
+					entries.remove();
+					registeredHostsByConnection.remove(host.getInstanceConnectionInfo());
+					
+					// add to the dead
+					deadHosts.add(host.getInstanceConnectionInfo());
+					
+					host.markDead();
+					
+					totalNumberOfAliveTaskSlots -= host.getTotalNumberOfSlots();
+					
+					LOG.info(String.format("TaskManager %s at %s did not report a heartbeat for %d msecs - marking as dead. Current number of registered hosts is %d.",
+							host.getId(), host.getInstanceConnectionInfo(), heartbeatTimeout, registeredHostsById.size()));
+					
+					// report to all listeners
+					notifyDeadInstance(host);
+				}
+			}
+		}
+	}
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Periodic task that checks whether hosts have not sent their heart-beat
 	 * messages and purges the hosts in this case.
 	 */
 	private final TimerTask cleanupStaleMachines = new TimerTask() {
-
 		@Override
 		public void run() {
-
-			final long now = System.currentTimeMillis();
-			final long timeout = DefaultInstanceManager.this.heartbeatTimeout;
-			
-			synchronized (DefaultInstanceManager.this.lock) {
-				if (DefaultInstanceManager.this.shutdown) {
-					return;
-				}
-
-				final Iterator<Map.Entry<InstanceID, Instance>> entries = registeredHostsById.entrySet().iterator();
-				
-				// check all hosts whether they did not send heart-beat messages.
-				while (entries.hasNext()) {
-					
-					final Map.Entry<InstanceID, Instance> entry = entries.next();
-					final Instance host = entry.getValue();
-					
-					if (!host.isStillAlive(now, timeout)) {
-						
-						// remove from the living
-						entries.remove();
-						registeredHostsByConnection.remove(host.getInstanceConnectionInfo());
-						
-						// add to the dead
-						deadHosts.add(host.getInstanceConnectionInfo());
-						
-						host.markDied();
-						
-						totalNumberOfAliveTaskSlots -= host.getNumberOfSlots();
-						
-						LOG.info(String.format("TaskManager %s at %s did not report a heartbeat for %d msecs - marking as dead. Current number of registered hosts is %d.",
-								host.getId(), host.getInstanceConnectionInfo(), heartbeatTimeout, registeredHostsById.size()));
-					}
-				}
+			try {
+				checkForDeadInstances();
+			}
+			catch (Throwable t) {
+				LOG.error("Checking for dead instances failed.", t);
 			}
 		}
 	};

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DummyInstance.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DummyInstance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DummyInstance.java
deleted file mode 100644
index af965c0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/DummyInstance.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * 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.instance;
-
-/**
- * A DummyInstance is a stub implementation of the {@link Instance} interface.
- * Dummy instances are used to plan a job execution but must be replaced with
- * concrete instances before the job execution starts.
- * 
- */
-public class DummyInstance extends Instance {
-
-	private static int nextID = 0;
-
-	private final String name;
-
-	public static synchronized DummyInstance createDummyInstance() {
-
-		return new DummyInstance(nextID++);
-	}
-
-	/**
-	 * Constructs a new dummy instance of the given instance type.
-	 * 
-	 * @param id
-	 *        the ID of the dummy instance
-	 */
-	private DummyInstance(int id) {
-		super(null, null, null, null, 0);
-
-		this.name = "DummyInstance_" + Integer.toString(id);
-	}
-
-
-	@Override
-	public String toString() {
-
-		return this.name;
-	}
-
-
-	@Override
-	public HardwareDescription getHardwareDescription() {
-
-		throw new RuntimeException("getHardwareDescription is called on a DummyInstance");
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java
index affbdd6..32d6572 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/HardwareDescription.java
@@ -121,6 +121,16 @@ public final class HardwareDescription implements IOReadableWritable, java.io.Se
 	}
 	
 	// --------------------------------------------------------------------------------------------
+	// Utils
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return String.format("cores=%d, physMem=%d, heap=%d, managed=%d", 
+				numberOfCPUCores, sizeOfPhysicalMemory, sizeOfJvmHeap, sizeOfManagedMemory);
+	}
+	
+	// --------------------------------------------------------------------------------------------
 	// Factory
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
index c895bbb..3d39c8f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
@@ -16,38 +16,30 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.instance;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.List;
-import java.util.ArrayList;
+import java.util.ArrayDeque;
+import java.util.HashSet;
+import java.util.Queue;
 import java.util.Set;
-import java.util.Collection;
 
-import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileRequest;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileResponse;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheUpdate;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.ipc.RPC;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobmanager.scheduler.ResourceId;
 import org.apache.flink.runtime.net.NetUtils;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
-import org.apache.flink.runtime.taskmanager.TaskCancelResult;
-import org.apache.flink.runtime.taskmanager.TaskKillResult;
-import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
+import org.eclipse.jetty.util.log.Log;
 
 /**
  * An instance represents a resource a {@link org.apache.flink.runtime.taskmanager.TaskManager} runs on.
  */
 public class Instance {
 	
+	/** The lock on which to synchronize allocations and failure state changes */
+	private final Object instanceLock = new Object();
+	
 	/** The connection info to connect to the task manager represented by this instance. */
 	private final InstanceConnectionInfo instanceConnectionInfo;
 	
@@ -60,20 +52,21 @@ public class Instance {
 	/** The number of task slots available on the node */
 	private final int numberOfSlots;
 
-	/**
-	 * Allocated slots on this instance
-	 */
-	private final Map<AllocationID, AllocatedSlot> allocatedSlots = new HashMap<AllocationID, AllocatedSlot>();
+	
+	private final Queue<Integer> availableSlots;
+	
+	/** Allocated slots on this instance */
+	private final Set<AllocatedSlot> allocatedSlots = new HashSet<AllocatedSlot>();
 
-	/**
-	 * Stores the RPC stub object for the instance's task manager.
-	 */
-	private TaskOperationProtocol taskManager = null;
+	/** The RPC proxy to send calls to the task manager represented by this instance */
+	private volatile TaskOperationProtocol taskManager ;
 
 	/**
 	 * Time when last heat beat has been received from the task manager running on this instance.
 	 */
 	private volatile long lastReceivedHeartBeat = System.currentTimeMillis();
+	
+	private volatile boolean isDead;
 
 	/**
 	 * Constructs an abstract instance object.
@@ -88,125 +81,108 @@ public class Instance {
 		this.instanceId = id;
 		this.resources = resources;
 		this.numberOfSlots = numberOfSlots;
+		
+		this.availableSlots = new ArrayDeque<Integer>();
+		for (int i = 0; i < numberOfSlots; i++) {
+			this.availableSlots.add(i);
+		}
 	}
 
-	/**
-	 * Creates or returns the RPC stub object for the instance's task manager.
-	 * 
-	 * @return the RPC stub object for the instance's task manager
-	 * @throws IOException
-	 *         thrown if the RPC stub object for the task manager cannot be created
-	 */
-	private TaskOperationProtocol getTaskManagerProxy() throws IOException {
-
-		if (this.taskManager == null) {
-
-			this.taskManager = RPC.getProxy(TaskOperationProtocol.class,
-				new InetSocketAddress(getInstanceConnectionInfo().address(),
-					getInstanceConnectionInfo().ipcPort()), NetUtils.getSocketFactory());
+	public TaskOperationProtocol getTaskManagerProxy() throws IOException {
+		TaskOperationProtocol tm = this.taskManager;
+		
+		if (tm == null) {
+			synchronized (this) {
+				if (this.taskManager == null) {
+					this.taskManager = RPC.getProxy(TaskOperationProtocol.class,
+						new InetSocketAddress(getInstanceConnectionInfo().address(),
+							getInstanceConnectionInfo().ipcPort()), NetUtils.getSocketFactory());
+				}
+				tm = this.taskManager;
+			}
 		}
-
-		return this.taskManager;
+		
+		return tm;
 	}
 
-	/**
-	 * Destroys and removes the RPC stub object for this instance's task manager.
-	 */
+	/**  Destroys and removes the RPC stub object for this instance's task manager. */
 	private void destroyTaskManagerProxy() {
-
-		if (this.taskManager != null) {
-			RPC.stopProxy(this.taskManager);
-			this.taskManager = null;
+		synchronized (this) {
+			if (this.taskManager != null) {
+				try {
+					RPC.stopProxy(this.taskManager);
+				} catch (Throwable t) {
+					Log.debug("Error shutting down RPC proxy.", t);
+				}
+			}
 		}
 	}
 
-	/**
-	 * Returns the instance's connection information object.
-	 * 
-	 * @return the instance's connection information object
-	 */
-	public final InstanceConnectionInfo getInstanceConnectionInfo() {
-		return this.instanceConnectionInfo;
-	}
 
-	/**
-	 * Checks if all the libraries required to run the job with the given
-	 * job ID are available on this instance. Any libary that is missing
-	 * is transferred to the instance as a result of this call.
-	 * 
-	 * @param jobID
-	 *        the ID of the job whose libraries are to be checked for
-	 * @throws IOException
-	 *         thrown if an error occurs while checking for the libraries
-	 */
-	public synchronized void checkLibraryAvailability(final JobID jobID) throws IOException {
-
-		// Now distribute the required libraries for the job
-		String[] requiredLibraries = LibraryCacheManager.getRequiredJarFiles(jobID);
-
-		if (requiredLibraries == null) {
-			throw new IOException("No entry of required libraries for job " + jobID);
+	
+	// --------------------------------------------------------------------------------------------
+	// Life and Death
+	// --------------------------------------------------------------------------------------------
+	
+	public boolean isAlive() {
+		return !isDead;
+	}
+	
+	public void markDead() {
+		if (isDead) {
+			return;
 		}
-
-		LibraryCacheProfileRequest request = new LibraryCacheProfileRequest();
-		request.setRequiredLibraries(requiredLibraries);
-
-		// Send the request
-		LibraryCacheProfileResponse response = null;
-		response = getTaskManagerProxy().getLibraryCacheProfile(request);
-
-		// Check response and transfer libraries if necessary
-		for (int k = 0; k < requiredLibraries.length; k++) {
-			if (!response.isCached(k)) {
-				LibraryCacheUpdate update = new LibraryCacheUpdate(requiredLibraries[k]);
-				getTaskManagerProxy().updateLibraryCache(update);
+		
+		isDead = true;
+		
+		synchronized (instanceLock) {
+			this.allocatedSlots.clear();
+			for (AllocatedSlot slot : allocatedSlots) {
+				slot.cancelResource();
 			}
 		}
+		
+		destroyTaskManagerProxy();
 	}
-
-	/**
-	 * Submits a list of tasks to the instance's {@link org.apache.flink.runtime.taskmanager.TaskManager}.
-	 * 
-	 * @param tasks
-	 *        the list of tasks to be submitted
-	 * @return the result of the submission attempt
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the task
-	 */
-	public synchronized List<TaskSubmissionResult> submitTasks(final List<TaskDeploymentDescriptor> tasks) throws IOException {
-		return getTaskManagerProxy().submitTasks(tasks);
+	
+	// --------------------------------------------------------------------------------------------
+	// Properties
+	// --------------------------------------------------------------------------------------------
+	
+	public InstanceID getId() {
+		return instanceId;
 	}
-
+	
+	public HardwareDescription getResources() {
+		return this.resources;
+	}
+	
+	public int getTotalNumberOfSlots() {
+		return numberOfSlots;
+	}
+	
 	/**
-	 * Cancels the task identified by the given ID at the instance's
-	 * {@link org.apache.flink.runtime.taskmanager.TaskManager}.
+	 * Returns the instance's connection information object.
 	 * 
-	 * @param id
-	 *        the ID identifying the task to be canceled
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the request or receiving the response
-	 * @return the result of the cancel attempt
+	 * @return the instance's connection information object
 	 */
-	public synchronized TaskCancelResult cancelTask(final ExecutionVertexID id) throws IOException {
-
-		return getTaskManagerProxy().cancelTask(id);
+	public InstanceConnectionInfo getInstanceConnectionInfo() {
+		return this.instanceConnectionInfo;
 	}
-
+	
+	// --------------------------------------------------------------------------------------------
+	// Heartbeats
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * Kills the task identified by the given ID at the instance's
-	 * {@link org.apache.flink.runtime.taskmanager.TaskManager}.
+	 * Gets the timestamp of the last heartbeat.
 	 * 
-	 * @param id
-	 *        the ID identifying the task to be killed
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the request or receiving the response
-	 * @return the result of the kill attempt
+	 * @return The timestamp of the last heartbeat.
 	 */
-	public synchronized TaskKillResult killTask(final ExecutionVertexID id) throws IOException {
-
-		return getTaskManagerProxy().killTask(id);
+	public long getLastHeartBeat() {
+		return this.lastReceivedHeartBeat;
 	}
-
+	
 	/**
 	 * Updates the time of last received heart beat to the current system time.
 	 */
@@ -214,141 +190,53 @@ public class Instance {
 		this.lastReceivedHeartBeat = System.currentTimeMillis();
 	}
 
-	public boolean isStillAlive(long now, long cleanUpInterval) {
-		return this.lastReceivedHeartBeat + cleanUpInterval > now;
-	}
-
-
-	@Override
-	public boolean equals(final Object obj) {
-
-		// Fall back since dummy instances do not have a instanceConnectionInfo
-		if (this.instanceConnectionInfo == null) {
-			return super.equals(obj);
-		}
-
-		if (!(obj instanceof Instance)) {
-			return false;
-		}
-
-		final Instance abstractInstance = (Instance) obj;
-
-		return this.instanceConnectionInfo.equals(abstractInstance.getInstanceConnectionInfo());
-	}
-
-
-	@Override
-	public int hashCode() {
-
-		// Fall back since dummy instances do not have a instanceConnectionInfo
-		if (this.instanceConnectionInfo == null) {
-			return super.hashCode();
-		}
-
-		return this.instanceConnectionInfo.hashCode();
-	}
-
-	/**
-	 * Triggers the remote task manager to print out the current utilization of its read and write buffers to its logs.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the request
-	 */
-	public synchronized void logBufferUtilization() throws IOException {
-
-		getTaskManagerProxy().logBufferUtilization();
-	}
-
-	/**
-	 * Kills the task manager running on this instance. This method is mainly intended to test and debug Nephele's fault
-	 * tolerance mechanisms.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the request
-	 */
-	public synchronized void killTaskManager() throws IOException {
-
-		getTaskManagerProxy().killTaskManager();
-	}
-
-	/**
-	 * Invalidates the entries identified by the given channel IDs from the remote task manager's receiver lookup cache.
-	 * 
-	 * @param channelIDs
-	 *        the channel IDs identifying the cache entries to invalidate
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
-	public synchronized void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) throws IOException {
-		getTaskManagerProxy().invalidateLookupCacheEntries(channelIDs);
-	}
-
 	/**
-	 * Destroys all RPC stub objects attached to this instance.
+	 * Checks whether the last heartbeat occurred within the last {@code n} milliseconds
+	 * before the given timestamp {@code now}.
+	 *  
+	 * @param now The timestamp representing the current time.
+	 * @param cleanUpInterval The maximum time (in msecs) that the last heartbeat may lie in the past.
+	 * @return True, if this instance is considered alive, false otherwise.
 	 */
-	public synchronized void destroyProxies() {
-
-		destroyTaskManagerProxy();
-
-	}
-
-	public int getNumberOfSlots() {
-		return numberOfSlots;
-	}
-
-	public int getNumberOfAvailableSlots() { return numberOfSlots - allocatedSlots.size(); }
-
-	public synchronized AllocatedResource allocateSlot(JobID jobID) throws InstanceException{
-		if(allocatedSlots.size() < numberOfSlots){
-			AllocatedSlot slot = new AllocatedSlot(jobID);
-
-			allocatedSlots.put(slot.getAllocationID(), slot);
-			return new AllocatedResource(this,slot.getAllocationID());
-		}else{
-			throw new InstanceException("Overbooking instance " + instanceConnectionInfo + ".");
-		}
-	}
-
-	public synchronized void releaseSlot(AllocationID allocationID) {
-		if(allocatedSlots.containsKey(allocationID)){
-			allocatedSlots.remove(allocationID);
-		}else{
-			throw new RuntimeException("There is no slot registered with allocation ID " + allocationID + ".");
-		}
-	}
-
-	public Collection<AllocatedSlot> getAllocatedSlots() {
-		return allocatedSlots.values();
-	}
-
-	public Collection<AllocatedSlot> removeAllocatedSlots() {
-		Collection<AllocatedSlot> slots = new ArrayList<AllocatedSlot>(this.allocatedSlots.values());
-
-		for(AllocatedSlot slot : slots){
-			releaseSlot(slot.getAllocationID());
-		}
-
-		return slots;
-	}
-
-	public long getLastHeartBeat() {
-		return this.lastReceivedHeartBeat;
+	public boolean isStillAlive(long now, long cleanUpInterval) {
+		return this.lastReceivedHeartBeat + cleanUpInterval > now;
 	}
 	
+	// --------------------------------------------------------------------------------------------
+	// Resource allocation
+	// --------------------------------------------------------------------------------------------
 	
-	public void markDied() {
-		
+	public AllocatedSlot allocateSlot(JobID jobID, ResourceId resourceId) throws InstanceDiedException {
+		synchronized (instanceLock) {
+			if (isDead) {
+				throw new InstanceDiedException(this);
+			}
+			
+			Integer nextSlot = availableSlots.poll();
+			if (nextSlot == null) {
+				return null;
+			} else {
+				AllocatedSlot slot = new AllocatedSlot(jobID, resourceId, this, nextSlot);
+				allocatedSlots.add(slot);
+				return slot;
+			}
+		}
 	}
 	
-	public void destroy() {
-		
+	public int getNumberOfAvailableSlots() {
+		return this.availableSlots.size();
 	}
 	
-	public InstanceID getId() {
-		return instanceId;
+	public boolean hasResourcesAvailable() {
+		return !isDead && getNumberOfAvailableSlots() > 0;
 	}
 	
-	public HardwareDescription getResources() {
-		return this.resources;
+	// --------------------------------------------------------------------------------------------
+	// Standard Utilities
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "Instance (" + this.instanceConnectionInfo + "), resources: " + this.resources + ", numberOfSlots=" + numberOfSlots;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
index daf7e0d..4cbb2a9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
@@ -183,7 +183,7 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 	}
 
 	// --------------------------------------------------------------------------------------------
-	// Serialization
+	// Utilities
 	// --------------------------------------------------------------------------------------------
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java
new file mode 100644
index 0000000..42b9817
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceDiedException.java
@@ -0,0 +1,34 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.instance;
+
+/**
+ * A special instance signaling that an attempted operation on an instance is not possible,
+ * because the instance has died.
+ */
+public class InstanceDiedException extends Exception {
+	private static final long serialVersionUID = -4917918318403135745L;
+	
+	private final Instance instance;
+
+	public InstanceDiedException(Instance instance) {
+		this.instance = instance;
+	}
+	
+	public Instance getInstance() {
+		return instance;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceException.java
deleted file mode 100644
index 61d5868..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceException.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * 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.instance;
-
-/**
- * An instance exception is thrown if the allocation, assignment or deallocation of an instance fails.
- * 
- */
-public class InstanceException extends Exception {
-
-	/**
-	 * The generated serial UID.
-	 */
-	private static final long serialVersionUID = 3463832262505896962L;
-
-	/**
-	 * Constructs a new instance exception with the given error message.
-	 * 
-	 * @param errorMsg
-	 *        the error message to be included in the exception.
-	 */
-	public InstanceException(String errorMsg) {
-		super(errorMsg);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
index fbdef54..76e63b8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
@@ -16,38 +16,15 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.instance;
 
-import java.util.List;
-
-import org.apache.flink.runtime.jobgraph.JobID;
-
 /**
- * Classes implementing the {@link InstanceListener} interface can be notified about
- * the availability or the unexpected failure of an instance.
- * 
+ * Classes implementing the InstanceListener interface can be notified about
+ * the availability disappearance of instances.
  */
 public interface InstanceListener {
 
-	/**
-	 * Called if one or more requested resources have become available.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the initial request has been triggered for
-	 * @param allocatedResources
-	 *        the resources which have been allocated as a response to the initial request
-	 */
-	void resourcesAllocated(JobID jobID, List<AllocatedResource> allocatedResources);
-
-	/**
-	 * Called if one or more allocated resources assigned to at least one job have died unexpectedly.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the instance is used for
-	 * @param allocatedResource
-	 *        the allocated resources which are affected by the instance death
-	 */
-	void allocatedResourcesDied(JobID jobID, List<AllocatedResource> allocatedResource);
-
+	void newInstanceAvailable(Instance instance);
+	
+	void instanceDied(Instance instance);
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceNotifier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceNotifier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceNotifier.java
deleted file mode 100644
index 9e85a83..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceNotifier.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/**
- * 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.instance;
-
-import java.util.List;
-
-import org.apache.flink.runtime.instance.AllocatedResource;
-import org.apache.flink.runtime.instance.InstanceListener;
-import org.apache.flink.runtime.jobgraph.JobID;
-
-/**
- * This class is an auxiliary class to send the notification
- * about the availability of an {@link org.apache.flink.runtime.instance.Instance} to the given {@link
- * InstanceListener} object. The notification must be sent from
- * a separate thread, otherwise the atomic operation of requesting an instance
- * for a vertex and switching to the state ASSIGNING could not be guaranteed.
- * This class is thread-safe.
- * 
- */
-public class InstanceNotifier extends Thread {
-
-	/**
-	 * The {@link InstanceListener} object to send the notification to.
-	 */
-	private final InstanceListener instanceListener;
-
-	/**
-	 * The ID of the job the notification refers to.
-	 */
-	private final JobID jobID;
-
-	/**
-	 * The allocated resources the notification refers to.
-	 */
-	private final List<AllocatedResource> allocatedResources;
-
-	/**
-	 * Constructs a new instance notifier object.
-	 * 
-	 * @param instanceListener
-	 *        the listener to send the notification to
-	 * @param jobID
-	 *        the ID of the job the notification refers to
-	 * @param allocatedResources
-	 *        the resources with has been allocated for the job
-	 */
-	public InstanceNotifier(final InstanceListener instanceListener, final JobID jobID,
-							final List<AllocatedResource> allocatedResources) {
-		this.instanceListener = instanceListener;
-		this.jobID = jobID;
-		this.allocatedResources = allocatedResources;
-	}
-
-
-	@Override
-	public void run() {
-
-		this.instanceListener.resourcesAllocated(this.jobID, this.allocatedResources);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultExecutionListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultExecutionListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultExecutionListener.java
deleted file mode 100644
index d0bbdca..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultExecutionListener.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/**
- * 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.jobmanager.scheduler;
-
-import org.apache.flink.runtime.execution.ExecutionListener;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionPipeline;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.executiongraph.InternalJobStatus;
-import org.apache.flink.runtime.jobgraph.JobID;
-
-public class DefaultExecutionListener implements ExecutionListener {
-
-	/**
-	 * The instance of the {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler}.
-	 */
-	private final DefaultScheduler scheduler;
-
-	/**
-	 * The {@link ExecutionVertex} this wrapper object belongs to.
-	 */
-	private final ExecutionVertex executionVertex;
-
-	/**
-	 * Constructs a new wrapper object for the given {@link ExecutionVertex}.
-	 * 
-	 * @param scheduler
-	 *        the instance of the {@link DefaultScheduler}
-	 * @param executionVertex
-	 *        the {@link ExecutionVertex} the received notification refer to
-	 */
-	public DefaultExecutionListener(final DefaultScheduler scheduler, final ExecutionVertex executionVertex) {
-		this.scheduler = scheduler;
-		this.executionVertex = executionVertex;
-	}
-
-
-	@Override
-	public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID,
-			final ExecutionState newExecutionState, final String optionalMessage) {
-
-		final ExecutionGraph eg = this.executionVertex.getExecutionGraph();
-
-		// Check if we can deploy a new pipeline.
-		if (newExecutionState == ExecutionState.FINISHING) {
-
-			final ExecutionPipeline pipeline = this.executionVertex.getExecutionPipeline();
-			if (!pipeline.isFinishing()) {
-				// Some tasks of the pipeline are still running
-				return;
-			}
-
-			// Find another vertex in the group which is still in SCHEDULED state and get its pipeline.
-			final ExecutionGroupVertex groupVertex = this.executionVertex.getGroupVertex();
-			for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); ++i) {
-				final ExecutionVertex groupMember = groupVertex.getGroupMember(i);
-				if (groupMember.compareAndUpdateExecutionState(ExecutionState.SCHEDULED, ExecutionState.ASSIGNED)) {
-
-					final ExecutionPipeline pipelineToBeDeployed = groupMember.getExecutionPipeline();
-					pipelineToBeDeployed.setAllocatedResource(this.executionVertex.getAllocatedResource());
-					pipelineToBeDeployed.updateExecutionState(ExecutionState.ASSIGNED);
-
-					this.scheduler.deployAssignedPipeline(pipelineToBeDeployed);
-					return;
-				}
-			}
-		}
-
-		if (newExecutionState == ExecutionState.CANCELED || newExecutionState == ExecutionState.FINISHED) {
-
-			synchronized (eg) {
-
-				if (this.scheduler.getVerticesToBeRestarted().remove(this.executionVertex.getID()) != null) {
-
-					if (eg.getJobStatus() == InternalJobStatus.FAILING) {
-						return;
-					}
-
-					this.executionVertex.updateExecutionState(ExecutionState.ASSIGNED, "Restart as part of recovery");
-
-					// Run through the deployment procedure
-					this.scheduler.deployAssignedVertices(this.executionVertex);
-					return;
-				}
-			}
-		}
-
-		if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED
-			|| newExecutionState == ExecutionState.FAILED) {
-			// Check if instance can be released
-			this.scheduler.checkAndReleaseAllocatedResource(eg, this.executionVertex.getAllocatedResource());
-		}
-	}
-
-
-	@Override
-	public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-		// Nothing to do here
-	}
-
-
-	@Override
-	public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-		// Nothing to do here
-	}
-
-
-	@Override
-	public int getPriority() {
-
-		return 0;
-	}
-}


[13/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index bc95250..da1d28c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -18,1321 +18,1497 @@
 
 package org.apache.flink.runtime.executiongraph;
 
+import java.net.InetSocketAddress;
 import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
+import java.util.Collections;
 import java.util.Iterator;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.flink.api.common.io.InitializeOnMaster;
-import org.apache.flink.api.common.io.OutputFormat;
+
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.execution.ExecutionListener;
 import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
+import org.apache.flink.runtime.io.network.RemoteReceiver;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-import org.apache.flink.runtime.io.network.gates.GateID;
-import org.apache.flink.runtime.jobgraph.AbstractJobInputVertex;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobEdge;
-import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.util.ExecutorThreadFactory;
-import org.apache.flink.util.StringUtils;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 
-/**
- * In Nephele an execution graph is the main data structure for scheduling, executing and
- * observing a job. An execution graph is created from an job graph. In contrast to a job graph
- * it can contain communication edges of specific types, sub groups of vertices and information on
- * when and where (on which instance) to run particular tasks.
- * <p>
- * This class is thread-safe.
- * 
- */
-public class ExecutionGraph implements ExecutionListener {
 
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class);
+public class ExecutionGraph {
 
-	/**
-	 * The ID of the job this graph has been built for.
-	 */
+	private static final AtomicReferenceFieldUpdater<ExecutionGraph, JobStatus> STATE_UPDATER =
+			AtomicReferenceFieldUpdater.newUpdater(ExecutionGraph.class, JobStatus.class, "state");
+	
+	/** The log object used for debugging. */
+	static final Logger LOG = LoggerFactory.getLogger(ExecutionGraph.class);
+
+	// --------------------------------------------------------------------------------------------
+	
+	/** The ID of the job this graph has been built for. */
 	private final JobID jobID;
 
-	/**
-	 * The name of the original job graph.
-	 */
+	/** The name of the original job graph. */
 	private final String jobName;
 
-	/**
-	 * Mapping of vertex IDs to vertices.
-	 */
-	private final ConcurrentMap<ExecutionVertexID, ExecutionVertex> vertexMap = new ConcurrentHashMap<ExecutionVertexID, ExecutionVertex>(
-		1024);
-
-	/**
-	 * Mapping of channel IDs to edges.
-	 */
-	private final ConcurrentMap<ChannelID, ExecutionEdge> edgeMap = new ConcurrentHashMap<ChannelID, ExecutionEdge>(
-		1024 * 1024);
-
-	/**
-	 * List of stages in the graph.
-	 */
-	private final CopyOnWriteArrayList<ExecutionStage> stages = new CopyOnWriteArrayList<ExecutionStage>();
-
-	/**
-	 * The executor service to asynchronously perform update operations to this graph.
-	 */
-	private final ExecutorService executorService = Executors.newSingleThreadExecutor(ExecutorThreadFactory.INSTANCE);
-
-	/**
-	 * Index to the current execution stage.
-	 */
-	private volatile int indexToCurrentExecutionStage = 0;
-
-	/**
-	 * The job configuration that was originally attached to the JobGraph.
-	 */
+	/** The job configuration that was originally attached to the JobGraph. */
 	private final Configuration jobConfiguration;
+	
+	/** All job vertices that are part of this graph */
+	private final ConcurrentHashMap<JobVertexID, ExecutionJobVertex> tasks;
+	
+	/** All intermediate results that are part of this graph */
+	private final ConcurrentHashMap<IntermediateDataSetID, IntermediateResult> intermediateResults;
+	
+	/** An executor that can run long actions (involving remote calls) */
+	private final ExecutorService executor;
+	
+	
+	private final List<String> userCodeJarFiles;
+	
+	private final List<JobStatusListener> jobStatusListeners;
+	
+	private final List<ExecutionListener> executionListeners;
+	
+	
+	private DefaultScheduler jobScheduler;
+	
+	private boolean allowQueuedScheduling = false;
 
-	/**
-	 * The current status of the job which is represented by this execution graph.
-	 */
-	private final AtomicReference<InternalJobStatus> jobStatus = new AtomicReference<InternalJobStatus>(
-		InternalJobStatus.CREATED);
-
-	/**
-	 * The error description of the first task which causes this job to fail.
-	 */
-	private volatile String errorDescription = null;
-
-	/**
-	 * List of listeners which are notified in case the status of this job has changed.
-	 */
-	private final CopyOnWriteArrayList<JobStatusListener> jobStatusListeners = new CopyOnWriteArrayList<JobStatusListener>();
-
-	/**
-	 * List of listeners which are notified in case the execution stage of a job has changed.
-	 */
-	private final CopyOnWriteArrayList<ExecutionStageListener> executionStageListeners = new CopyOnWriteArrayList<ExecutionStageListener>();
-
-	/**
-	 * Private constructor used for duplicating execution vertices.
-	 * 
-	 * @param jobID
-	 *        the ID of the duplicated execution graph
-	 * @param jobName
-	 *        the name of the original job graph
-	 * @param jobConfiguration
-	 *        the configuration originally attached to the job graph
-	 */
-	private ExecutionGraph(final JobID jobID, final String jobName, final Configuration jobConfiguration) {
-		if (jobID == null) {
-			throw new IllegalArgumentException("Argument jobID must not be null");
-		}
-
-		this.jobID = jobID;
-		this.jobName = jobName;
-		this.jobConfiguration = jobConfiguration;
+	
+	private volatile JobStatus state = JobStatus.CREATED;
+	
+	
+	
+	public ExecutionGraph(JobID jobId, String jobName, Configuration jobConfig) {
+		this(jobId, jobName, jobConfig, null);
 	}
-
-	/**
-	 * Creates a new execution graph from a job graph.
-	 * 
-	 * @param job
-	 *        the user's job graph
-	 * @param defaultParallelism
-	 *        defaultParallelism in case that nodes have no parallelism set
-	 * @throws GraphConversionException
-	 *         thrown if the job graph is not valid and no execution graph can be constructed from it
-	 */
-	public ExecutionGraph(JobGraph job, int defaultParallelism) throws GraphConversionException {
-		this(job.getJobID(), job.getName(), job.getJobConfiguration());
-
-		// Start constructing the new execution graph from given job graph
-		try {
-			constructExecutionGraph(job, defaultParallelism);
-		} catch (GraphConversionException e) {
-			throw e; // forward graph conversion exceptions
-		} catch (Exception e) {
-			throw new GraphConversionException(StringUtils.stringifyException(e));
+	
+	public ExecutionGraph(JobID jobId, String jobName, Configuration jobConfig, ExecutorService executor) {
+		if (jobId == null || jobName == null || jobConfig == null) {
+			throw new NullPointerException();
 		}
+		
+		this.jobID = jobId;
+		this.jobName = jobName;
+		this.jobConfiguration = jobConfig;
+		this.executor = executor;
+		
+		this.tasks = new ConcurrentHashMap<JobVertexID, ExecutionJobVertex>();
+		this.intermediateResults = new ConcurrentHashMap<IntermediateDataSetID, IntermediateResult>();
+		
+		this.userCodeJarFiles = new ArrayList<String>();
+		this.jobStatusListeners = new CopyOnWriteArrayList<JobStatusListener>();
+		this.executionListeners = new CopyOnWriteArrayList<ExecutionListener>();
 	}
 
-	/**
-	 * Applies the user defined settings to the execution graph.
-	 * 
-	 * @param temporaryGroupVertexMap
-	 *        mapping between job vertices and the corresponding group vertices.
-	 * @throws GraphConversionException
-	 *         thrown if an error occurs while applying the user settings.
-	 */
-	private void applyUserDefinedSettings(final HashMap<AbstractJobVertex, ExecutionGroupVertex> temporaryGroupVertexMap)
-			throws GraphConversionException {
-
-		// The check for cycles in the dependency chain for instance sharing is already checked in
-		// <code>submitJob</code> method of the job manager
-
-		// If there is no cycle, apply the settings to the corresponding group vertices
-		final Iterator<Map.Entry<AbstractJobVertex, ExecutionGroupVertex>> it = temporaryGroupVertexMap.entrySet()
-			.iterator();
-		while (it.hasNext()) {
-
-			final Map.Entry<AbstractJobVertex, ExecutionGroupVertex> entry = it.next();
-			final AbstractJobVertex jobVertex = entry.getKey();
-			if (jobVertex.getVertexToShareInstancesWith() != null) {
-
-				final AbstractJobVertex vertexToShareInstancesWith = jobVertex.getVertexToShareInstancesWith();
-				final ExecutionGroupVertex groupVertex = entry.getValue();
-				final ExecutionGroupVertex groupVertexToShareInstancesWith = temporaryGroupVertexMap
-					.get(vertexToShareInstancesWith);
-				groupVertex.shareInstancesWith(groupVertexToShareInstancesWith);
-			}
+	// --------------------------------------------------------------------------------------------
+	
+	public void attachJobGraph(List<AbstractJobVertex> topologiallySorted) throws JobException {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(String.format("Attaching %d topologically sorted vertices to existing job graph with %d "
+					+ "vertices and %d intermediate results.", topologiallySorted.size(), tasks.size(), intermediateResults.size()));
 		}
-
-		// Second, we create the number of execution vertices each group vertex is supposed to manage
-		Iterator<ExecutionGroupVertex> it2 = new ExecutionGroupVertexIterator(this, true, -1);
-		while (it2.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it2.next();
-			if (groupVertex.isNumberOfMembersUserDefined()) {
-				groupVertex.createInitialExecutionVertices(groupVertex.getUserDefinedNumberOfMembers());
+		
+		for (AbstractJobVertex jobVertex : topologiallySorted) {
+			
+			// create the execution job vertex and attach it to the graph
+			ExecutionJobVertex ejv = new ExecutionJobVertex(this, jobVertex, 1);
+			ejv.connectToPredecessors(this.intermediateResults);
+			
+			ExecutionJobVertex previousTask = this.tasks.putIfAbsent(jobVertex.getID(), ejv);
+			if (previousTask != null) {
+				throw new JobException(String.format("Encountered two job vertices with ID %s : previous=[%s] / new=[%s]",
+						jobVertex.getID(), ejv, previousTask));
 			}
-		}
-
-		// Finally, apply the channel settings channel settings
-		it2 = new ExecutionGroupVertexIterator(this, true, -1);
-		while (it2.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it2.next();
-			for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) {
-
-				final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i);
-				if (edge.isChannelTypeUserDefined()) {
-					edge.changeChannelType(edge.getChannelType());
+			
+			for (IntermediateResult res : ejv.getProducedDataSets()) {
+				IntermediateResult previousDataSet = this.intermediateResults.putIfAbsent(res.getId(), res);
+				if (previousDataSet != null) {
+					throw new JobException(String.format("Encountered two intermediate data set with ID %s : previous=[%s] / new=[%s]",
+							res.getId(), res, previousDataSet));
 				}
-
-				// Create edges between execution vertices
-				createExecutionEdgesForGroupEdge(edge);
 			}
 		}
-
-		// Repair the instance assignment after having changed the channel types
-		repairInstanceAssignment();
-
-		// Repair the instance sharing among different group vertices
-		repairInstanceSharing();
-
-		// Finally, repair the stages
-		repairStages();
 	}
-
-	/**
-	 * Sets up an execution graph from a job graph.
-	 * 
-	 * @param jobGraph
-	 *        the job graph to create the execution graph from
-	 * @param defaultParallelism
-	 *        defaultParallelism in case that nodes have no parallelism set
-	 * @throws GraphConversionException
-	 *         thrown if the job graph is not valid and no execution graph can be constructed from it
-	 */
-	private void constructExecutionGraph(final JobGraph jobGraph, final int defaultParallelism)
-			throws GraphConversionException {
-
-		// Clean up temporary data structures
-		final HashMap<AbstractJobVertex, ExecutionVertex> temporaryVertexMap = new HashMap<AbstractJobVertex, ExecutionVertex>();
-		final HashMap<AbstractJobVertex, ExecutionGroupVertex> temporaryGroupVertexMap = new HashMap<AbstractJobVertex, ExecutionGroupVertex>();
-
-		// Initially, create only one execution stage that contains all group vertices
-		final ExecutionStage initialExecutionStage = new ExecutionStage(this, 0);
-		this.stages.add(initialExecutionStage);
-
-		// Convert job vertices to execution vertices and initialize them
-		final AbstractJobVertex[] all = jobGraph.getAllJobVertices();
-		for (int i = 0; i < all.length; i++) {
-			if(all[i].getNumberOfSubtasks() == -1){
-				all[i].setNumberOfSubtasks(defaultParallelism);
-			}
-
-			final ExecutionVertex createdVertex = createVertex(all[i], initialExecutionStage);
-			temporaryVertexMap.put(all[i], createdVertex);
-			temporaryGroupVertexMap.put(all[i], createdVertex.getGroupVertex());
-		}
-
-		// Create initial edges between the vertices
-		createInitialGroupEdges(temporaryVertexMap);
-
-		// Now that an initial graph is built, apply the user settings
-		applyUserDefinedSettings(temporaryGroupVertexMap);
-
-		// Calculate the connection IDs
-		calculateConnectionIDs();
-
-		// Finally, construct the execution pipelines
-		reconstructExecutionPipelines();
+	
+	public void addUserCodeJarFile(String jarFile) {
+		this.userCodeJarFiles.add(jarFile);
 	}
-
-	private void createExecutionEdgesForGroupEdge(final ExecutionGroupEdge groupEdge) {
-
-		final ExecutionGroupVertex source = groupEdge.getSourceVertex();
-		final int indexOfOutputGate = groupEdge.getIndexOfOutputGate();
-		final ExecutionGroupVertex target = groupEdge.getTargetVertex();
-		final int indexOfInputGate = groupEdge.getIndexOfInputGate();
-
-		final Map<GateID, List<ExecutionEdge>> inputChannelMap = new HashMap<GateID, List<ExecutionEdge>>();
-
-		// Unwire the respective gate of the source vertices
-		final int currentNumberOfSourceNodes = source.getCurrentNumberOfGroupMembers();
-		for (int i = 0; i < currentNumberOfSourceNodes; ++i) {
-
-			final ExecutionVertex sourceVertex = source.getGroupMember(i);
-			final ExecutionGate outputGate = sourceVertex.getOutputGate(indexOfOutputGate);
-			if (outputGate == null) {
-				throw new IllegalStateException("wire: " + sourceVertex.getName()
-					+ " has no output gate with index " + indexOfOutputGate);
-			}
-
-			if (outputGate.getNumberOfEdges() > 0) {
-				throw new IllegalStateException("wire: wire called on source " + sourceVertex.getName() + " (" + i
-					+ "), but number of output channels is " + outputGate.getNumberOfEdges() + "!");
+	
+	public String[] getUserCodeJarFiles() {
+		return (String[]) this.userCodeJarFiles.toArray(new String[this.userCodeJarFiles.size()]);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public JobID getJobID() {
+		return jobID;
+	}
+	
+	public String getJobName() {
+		return jobName;
+	}
+	
+	public Configuration getJobConfiguration() {
+		return jobConfiguration;
+	}
+	
+	public JobStatus getState() {
+		return state;
+	}
+	
+	public ExecutionJobVertex getJobVertex(JobVertexID id) {
+		return this.tasks.get(id);
+	}
+	
+	public Map<JobVertexID, ExecutionJobVertex> getAllVertices() {
+		return Collections.unmodifiableMap(this.tasks);
+	}
+	
+	public Map<IntermediateDataSetID, IntermediateResult> getAllIntermediateResults() {
+		return Collections.unmodifiableMap(this.intermediateResults);
+	}
+	
+	public Iterable<ExecutionVertex2> getAllExecutionVertices() {
+		return new Iterable<ExecutionVertex2>() {
+			@Override
+			public Iterator<ExecutionVertex2> iterator() {
+				return new AllVerticesIterator(tasks.values().iterator());
 			}
-
-			final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers();
-			final List<ExecutionEdge> outputChannels = new ArrayList<ExecutionEdge>();
-
-			for (int j = 0; j < currentNumberOfTargetNodes; ++j) {
-
-				final ExecutionVertex targetVertex = target.getGroupMember(j);
-				final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate);
-				if (inputGate == null) {
-					throw new IllegalStateException("wire: " + targetVertex.getName()
-						+ " has no input gate with index " + indexOfInputGate);
-				}
-
-				if (inputGate.getNumberOfEdges() > 0 && i == 0) {
-					throw new IllegalStateException("wire: wire called on target " + targetVertex.getName() + " ("
-						+ j + "), but number of input channels is " + inputGate.getNumberOfEdges() + "!");
-				}
-
-				// Check if a wire is supposed to be created
-				if (DistributionPatternProvider.createWire(groupEdge.getDistributionPattern(),
-					i, j, currentNumberOfSourceNodes, currentNumberOfTargetNodes)) {
-
-					final ChannelID outputChannelID = new ChannelID();
-					final ChannelID inputChannelID = new ChannelID();
-
-					final ExecutionEdge edge = new ExecutionEdge(outputGate, inputGate, groupEdge, outputChannelID,
-						inputChannelID, outputGate.getNumberOfEdges(), inputGate.getNumberOfEdges());
-
-					this.edgeMap.put(outputChannelID, edge);
-					this.edgeMap.put(inputChannelID, edge);
-
-					outputChannels.add(edge);
-
-					List<ExecutionEdge> inputChannels = inputChannelMap.get(inputGate.getGateID());
-					if (inputChannels == null) {
-						inputChannels = new ArrayList<ExecutionEdge>();
-						inputChannelMap.put(inputGate.getGateID(), inputChannels);
+		};
+	}
+	
+	public boolean isQueuedSchedulingAllowed() {
+		return this.allowQueuedScheduling;
+	}
+	
+	public void setQueuedSchedulingAllowed(boolean allowed) {
+		this.allowQueuedScheduling = allowed;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void registerJobStatusListener(JobStatusListener jobStatusListener) {
+		this.jobStatusListeners.add(jobStatusListener);
+	}
+	
+	public void registerExecutionListener(ExecutionListener executionListener) {
+		this.executionListeners.add(executionListener);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void scheduleForExecution(DefaultScheduler scheduler) throws JobException {
+		if (scheduler == null) {
+			throw new IllegalArgumentException("Scheduler must not be null.");
+		}
+		
+		if (STATE_UPDATER.compareAndSet(this, JobStatus.CREATED, JobStatus.RUNNING)) {
+			this.jobScheduler = scheduler;
+			
+			notifyJobStatusChange(JobStatus.RUNNING, null);
+			
+			// initially, we simply take the ones without inputs.
+			// next, we implement the logic to go back from vertices that need computation
+			// to the ones we need to start running
+			for (ExecutionJobVertex ejv : this.tasks.values()) {
+				if (ejv.getJobVertex().isInputVertex()) {
+					for (ExecutionVertex2 ev : ejv.getTaskVertices()) {
+						ev.scheduleForExecution(scheduler);
 					}
-
-					inputChannels.add(edge);
 				}
 			}
-
-			outputGate.replaceAllEdges(outputChannels);
 		}
-
-		// Finally, set the channels for the input gates
-		final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers();
-		for (int i = 0; i < currentNumberOfTargetNodes; ++i) {
-
-			final ExecutionVertex targetVertex = target.getGroupMember(i);
-			final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate);
-
-			final List<ExecutionEdge> inputChannels = inputChannelMap.get(inputGate.getGateID());
-			if (inputChannels == null) {
-				LOG.error("Cannot find input channels for gate ID " + inputGate.getGateID());
-				continue;
-			}
-
-			inputGate.replaceAllEdges(inputChannels);
+		else {
+			throw new IllegalStateException("Job may only be scheduled from state " + JobStatus.CREATED);
 		}
-
 	}
-
-	/**
-	 * Creates the initial edges between the group vertices
-	 * 
-	 * @param vertexMap
-	 *        the temporary vertex map
-	 * @throws GraphConversionException
-	 *         if the initial wiring cannot be created
-	 */
-	private void createInitialGroupEdges(final HashMap<AbstractJobVertex, ExecutionVertex> vertexMap)
-			throws GraphConversionException {
-
-		Iterator<Map.Entry<AbstractJobVertex, ExecutionVertex>> it = vertexMap.entrySet().iterator();
-
-		while (it.hasNext()) {
-
-			final Map.Entry<AbstractJobVertex, ExecutionVertex> entry = it.next();
-			final AbstractJobVertex sjv = entry.getKey();
-			final ExecutionVertex sev = entry.getValue();
-			final ExecutionGroupVertex sgv = sev.getGroupVertex();
-
-			// First, build the group edges
-			for (int i = 0; i < sjv.getNumberOfForwardConnections(); ++i) {
-				final JobEdge edge = sjv.getForwardConnection(i);
-				final AbstractJobVertex tjv = edge.getConnectedVertex();
-
-				final ExecutionVertex tev = vertexMap.get(tjv);
-				final ExecutionGroupVertex tgv = tev.getGroupVertex();
-				// Use NETWORK as default channel type if nothing else is defined by the user
-				ChannelType channelType = edge.getChannelType();
-				boolean userDefinedChannelType = true;
-				if (channelType == null) {
-					userDefinedChannelType = false;
-					channelType = ChannelType.NETWORK;
-				}
-
-				final DistributionPattern distributionPattern = edge.getDistributionPattern();
-
-				// Connect the corresponding group vertices and copy the user settings from the job edge
-				final ExecutionGroupEdge groupEdge = sgv.wireTo(tgv, edge.getIndexOfInputGate(), i, channelType,
-					userDefinedChannelType,distributionPattern);
-
-				final ExecutionGate outputGate = new ExecutionGate(new GateID(), sev, groupEdge, false);
-				sev.insertOutputGate(i, outputGate);
-				final ExecutionGate inputGate = new ExecutionGate(new GateID(), tev, groupEdge, true);
-				tev.insertInputGate(edge.getIndexOfInputGate(), inputGate);
-			}
-		}
+	
+	public void cancel() {
+		//TODO
 	}
-
+	
+	public void updateState(TaskExecutionState state) {
+		//TODO		
+	}
+	
+	public ConnectionInfoLookupResponse lookupConnectionInfoAndDeployReceivers(InstanceConnectionInfo caller, ChannelID sourceChannelID) {
+		//TODO
+		return null;
+		
+//		final InternalJobStatus jobStatus = eg.getJobStatus();
+//		if (jobStatus == InternalJobStatus.FAILING || jobStatus == InternalJobStatus.CANCELING) {
+//			return ConnectionInfoLookupResponse.createJobIsAborting();
+//		}
+//
+//		final ExecutionEdge edge = eg.getEdgeByID(sourceChannelID);
+//		if (edge == null) {
+//			LOG.error("Cannot find execution edge associated with ID " + sourceChannelID);
+//			return ConnectionInfoLookupResponse.createReceiverNotFound();
+//		}
+//
+//		if (sourceChannelID.equals(edge.getInputChannelID())) {
+//			// Request was sent from an input channel
+//
+//			final ExecutionVertex connectedVertex = edge.getOutputGate().getVertex();
+//
+//			final Instance assignedInstance = connectedVertex.getAllocatedResource().getInstance();
+//			if (assignedInstance == null) {
+//				LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getOutputGateIndex()
+//					+ " but no instance assigned");
+//				// LOG.info("Created receiverNotReady for " + connectedVertex + " 1");
+//				return ConnectionInfoLookupResponse.createReceiverNotReady();
+//			}
+//
+//			// Check execution state
+//			final ExecutionState executionState = connectedVertex.getExecutionState();
+//			if (executionState == ExecutionState.FINISHED) {
+//				// that should not happen. if there is data pending, the receiver cannot be ready
+//				return ConnectionInfoLookupResponse.createReceiverNotFound();
+//			}
+//
+//			// running is common, finishing is happens when the lookup is for the close event
+//			if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING) {
+//				// LOG.info("Created receiverNotReady for " + connectedVertex + " in state " + executionState + " 2");
+//				return ConnectionInfoLookupResponse.createReceiverNotReady();
+//			}
+//
+//			if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
+//				// Receiver runs on the same task manager
+//				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getOutputChannelID());
+//			} else {
+//				// Receiver runs on a different task manager
+//
+//				final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
+//				final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
+//
+//				return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
+//			}
+//		}
+//		// else, the request is for an output channel
+//		// Find vertex of connected input channel
+//		final ExecutionVertex targetVertex = edge.getInputGate().getVertex();
+//
+//		// Check execution state
+//		final ExecutionState executionState = targetVertex.getExecutionState();
+//
+//		// check whether the task needs to be deployed
+//		if (executionState != ExecutionState.RUNNING && executionState != ExecutionState.FINISHING && executionState != ExecutionState.FINISHED) {
+//
+//			if (executionState == ExecutionState.ASSIGNED) {
+//				final Runnable command = new Runnable() {
+//					@Override
+//					public void run() {
+//						scheduler.deployAssignedVertices(targetVertex);
+//					}
+//				};
+//				eg.executeCommand(command);
+//			}
+//
+//			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 3");
+//			return ConnectionInfoLookupResponse.createReceiverNotReady();
+//		}
+//
+//		final Instance assignedInstance = targetVertex.getAllocatedResource().getInstance();
+//		if (assignedInstance == null) {
+//			LOG.error("Cannot resolve lookup: vertex found for channel ID " + edge.getInputChannelID() + " but no instance assigned");
+//			// LOG.info("Created receiverNotReady for " + targetVertex + " in state " + executionState + " 4");
+//			return ConnectionInfoLookupResponse.createReceiverNotReady();
+//		}
+//
+//		if (assignedInstance.getInstanceConnectionInfo().equals(caller)) {
+//			// Receiver runs on the same task manager
+//			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(edge.getInputChannelID());
+//		} else {
+//			// Receiver runs on a different task manager
+//			final InstanceConnectionInfo ici = assignedInstance.getInstanceConnectionInfo();
+//			final InetSocketAddress isa = new InetSocketAddress(ici.address(), ici.dataPort());
+//
+//			return ConnectionInfoLookupResponse.createReceiverFoundAndReady(new RemoteReceiver(isa, edge.getConnectionID()));
+//		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	
 	/**
-	 * Creates an execution vertex from a job vertex.
+	 * NOTE: This method never throws an error, only logs errors caused by the notified listeners.
 	 * 
-	 * @param jobVertex
-	 *        the job vertex to create the execution vertex from
-	 * @param initialExecutionStage
-	 *        the initial execution stage all group vertices are added to
-	 * @return the new execution vertex
-	 * @throws GraphConversionException
-	 *         thrown if the job vertex is of an unknown subclass
+	 * @param newState
+	 * @param message
 	 */
-	private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final ExecutionStage initialExecutionStage)
-			throws GraphConversionException {
-
-		// Create an initial execution vertex for the job vertex
-		final Class<? extends AbstractInvokable> invokableClass = jobVertex.getInvokableClass();
-		if (invokableClass == null) {
-			throw new GraphConversionException("JobVertex " + jobVertex.getID() + " (" + jobVertex.getName()
-				+ ") does not specify a task");
-		}
-
-		// Calculate the cryptographic signature of this vertex
-		final ExecutionSignature signature = ExecutionSignature.createSignature(jobVertex.getInvokableClass(),
-			jobVertex.getJobGraph().getJobID());
-
-		// Create a group vertex for the job vertex
-
-		ExecutionGroupVertex groupVertex = null;
-		try {
-			groupVertex = new ExecutionGroupVertex(jobVertex.getName(), jobVertex.getID(), this,
-				jobVertex.getNumberOfSubtasks(), jobVertex.getVertexToShareInstancesWith() != null ? true
-					: false, jobVertex.getNumberOfExecutionRetries(), jobVertex.getConfiguration(), signature,
-				invokableClass);
-		} catch (Throwable t) {
-			throw new GraphConversionException(t);
-		}
-
-		// Register input and output vertices separately
-		if (jobVertex instanceof AbstractJobInputVertex) {
-
-			final AbstractJobInputVertex jobInputVertex = (AbstractJobInputVertex) jobVertex;
-			
-			if (jobVertex instanceof JobInputVertex) {
-				try {
-					// get a handle to the user code class loader
-					ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID());
-					
-					((JobInputVertex) jobVertex).initializeInputFormatFromTaskConfig(cl);
-				}
-				catch (Throwable t) {
-					throw new GraphConversionException("Could not deserialize input format.", t);
-				}
-			}
-			
-			final Class<? extends InputSplit> inputSplitType = jobInputVertex.getInputSplitType();
-			
-			InputSplit[] inputSplits;
-
+	private void notifyJobStatusChange(JobStatus newState, String message) {
+		for (JobStatusListener listener : this.jobStatusListeners) {
 			try {
-				inputSplits = jobInputVertex.getInputSplits(jobVertex.getNumberOfSubtasks());
+				listener.jobStatusHasChanged(this, newState, message);
 			}
 			catch (Throwable t) {
-				throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName(), t);
-			}
-
-			if (inputSplits == null) {
-				inputSplits = new InputSplit[0];
-			}
-			
-			LOG.info("Job input vertex " + jobVertex.getName() + " generated " + inputSplits.length + " input splits");
-
-			// assign input splits and type
-			groupVertex.setInputSplits(inputSplits);
-			groupVertex.setInputSplitType(inputSplitType);
-		}
-
-		if (jobVertex instanceof JobOutputVertex){
-			final JobOutputVertex jobOutputVertex = (JobOutputVertex) jobVertex;
-			
-			try {
-				// get a handle to the user code class loader
-				ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID());
-				jobOutputVertex.initializeOutputFormatFromTaskConfig(cl);
-			}
-			catch (Throwable t) {
-				throw new GraphConversionException("Could not deserialize output format.", t);
-			}
-
-			OutputFormat<?> outputFormat = jobOutputVertex.getOutputFormat();
-			if (outputFormat != null && outputFormat instanceof InitializeOnMaster){
-				try {
-					((InitializeOnMaster) outputFormat).initializeGlobal(jobVertex.getNumberOfSubtasks());
-				}
-				catch (Throwable t) {
-					throw new GraphConversionException(t);
-				}
+				LOG.error("Notification of job status change caused an error.", t);
 			}
 		}
-
-		// Add group vertex to initial execution stage
-		initialExecutionStage.addStageMember(groupVertex);
-
-		final ExecutionVertex ev = new ExecutionVertex(this, groupVertex, jobVertex.getNumberOfForwardConnections(),
-			jobVertex.getNumberOfBackwardConnections());
-
-		// Assign initial instance to vertex (may be overwritten later on when user settings are applied)
-		ev.setAllocatedResource(new AllocatedResource(DummyInstance.createDummyInstance(), null));
-
-		return ev;
-	}
-
-	/**
-	 * Returns the number of input vertices registered with this execution graph.
-	 * 
-	 * @return the number of input vertices registered with this execution graph
-	 */
-	public int getNumberOfInputVertices() {
-		return this.stages.get(0).getNumberOfInputExecutionVertices();
 	}
-
+	
 	/**
-	 * Returns the number of input vertices for the given stage.
+	 * NOTE: This method never throws an error, only logs errors caused by the notified listeners.
 	 * 
-	 * @param stage
-	 *        the index of the execution stage
-	 * @return the number of input vertices for the given stage
+	 * @param vertexId
+	 * @param subtask
+	 * @param newExecutionState
+	 * @param optionalMessage
 	 */
-	public int getNumberOfInputVertices(int stage) {
-		if (stage >= this.stages.size()) {
-			return 0;
+	void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionState2 newExecutionState, String optionalMessage) {
+		for (ExecutionListener listener : this.executionListeners) {
+			try {
+				listener.executionStateChanged(jobID, vertexId, subtask, newExecutionState, optionalMessage);
+			}
+			catch (Throwable t) {
+				LOG.error("Notification of execution state change caused an error.", t);
+			}
 		}
-
-		return this.stages.get(stage).getNumberOfInputExecutionVertices();
 	}
-
-	/**
-	 * Returns the number of output vertices registered with this execution graph.
-	 * 
-	 * @return the number of output vertices registered with this execution graph
-	 */
-	public int getNumberOfOutputVertices() {
-		return this.stages.get(0).getNumberOfOutputExecutionVertices();
-	}
-
-	/**
-	 * Returns the number of output vertices for the given stage.
-	 * 
-	 * @param stage
-	 *        the index of the execution stage
-	 * @return the number of input vertices for the given stage
-	 */
-	public int getNumberOfOutputVertices(final int stage) {
-		if (stage >= this.stages.size()) {
-			return 0;
+	
+	// --------------------------------------------------------------------------------------------
+	//  Miscellaneous
+	// --------------------------------------------------------------------------------------------
+	
+	
+	public void execute(Runnable action) {
+		if (this.executor == null) {
+			throw new IllegalStateException("Executor has not been set.");
 		}
-
-		return this.stages.get(stage).getNumberOfOutputExecutionVertices();
-	}
-
-	/**
-	 * Returns the input vertex with the specified index.
-	 * 
-	 * @param index
-	 *        the index of the input vertex to return
-	 * @return the input vertex with the specified index or <code>null</code> if no input vertex with such an index
-	 *         exists
-	 */
-	public ExecutionVertex getInputVertex(final int index) {
-		return this.stages.get(0).getInputExecutionVertex(index);
-	}
-
-	/**
-	 * Returns the output vertex with the specified index.
-	 * 
-	 * @param index
-	 *        the index of the output vertex to return
-	 * @return the output vertex with the specified index or <code>null</code> if no output vertex with such an index
-	 *         exists
-	 */
-	public ExecutionVertex getOutputVertex(final int index) {
-		return this.stages.get(0).getOutputExecutionVertex(index);
-	}
-
-	/**
-	 * Returns the input vertex with the specified index for the given stage
-	 * 
-	 * @param stage
-	 *        the index of the stage
-	 * @param index
-	 *        the index of the input vertex to return
-	 * @return the input vertex with the specified index or <code>null</code> if no input vertex with such an index
-	 *         exists in that stage
-	 */
-	public ExecutionVertex getInputVertex(final int stage, final int index) {
-		try {
-			final ExecutionStage s = this.stages.get(stage);
-			if (s == null) {
-				return null;
-			}
-
-			return s.getInputExecutionVertex(index);
-
-		} catch (ArrayIndexOutOfBoundsException e) {
-			return null;
-		}
-	}
-
-	/**
-	 * Returns the output vertex with the specified index for the given stage.
-	 * 
-	 * @param stage
-	 *        the index of the stage
-	 * @param index
-	 *        the index of the output vertex to return
-	 * @return the output vertex with the specified index or <code>null</code> if no output vertex with such an index
-	 *         exists in that stage
-	 */
-	public ExecutionVertex getOutputVertex(final int stage, final int index) {
-		try {
-			final ExecutionStage s = this.stages.get(stage);
-			if (s == null) {
-				return null;
-			}
-
-			return s.getOutputExecutionVertex(index);
-
-		} catch (ArrayIndexOutOfBoundsException e) {
-			return null;
-		}
-	}
-
-	/**
-	 * Returns the execution stage with number <code>num</code>.
-	 * 
-	 * @param num
-	 *        the number of the execution stage to be returned
-	 * @return the execution stage with number <code>num</code> or <code>null</code> if no such execution stage exists
-	 */
-	public ExecutionStage getStage(final int num) {
-		try {
-			return this.stages.get(num);
-		} catch (ArrayIndexOutOfBoundsException e) {
-			return null;
-		}
-	}
-
-	/**
-	 * Returns the number of execution stages in the execution graph.
-	 * 
-	 * @return the number of execution stages in the execution graph
-	 */
-	public int getNumberOfStages() {
-		return this.stages.size();
-	}
-
-	/**
-	 * Identifies an execution by the specified channel ID and returns it.
-	 * 
-	 * @param id
-	 *        the channel ID to identify the vertex with
-	 * @return the execution vertex which has a channel with ID <code>id</code> or <code>null</code> if no such vertex
-	 *         exists in the execution graph
-	 */
-	public ExecutionVertex getVertexByChannelID(final ChannelID id) {
-		final ExecutionEdge edge = this.edgeMap.get(id);
-		if (edge == null) {
-			return null;
-		}
-
-		if (id.equals(edge.getOutputChannelID())) {
-			return edge.getOutputGate().getVertex();
-		}
-
-		return edge.getInputGate().getVertex();
-	}
-
-	/**
-	 * Finds an {@link ExecutionEdge} by its ID and returns it.
-	 * 
-	 * @param id
-	 *        the channel ID to identify the edge
-	 * @return the edge whose ID matches <code>id</code> or <code>null</code> if no such edge is known
-	 */
-	public ExecutionEdge getEdgeByID(final ChannelID id) {
-		return this.edgeMap.get(id);
-	}
-
-	/**
-	 * Registers an execution vertex with the execution graph.
-	 * 
-	 * @param vertex
-	 *        the execution vertex to register
-	 */
-	void registerExecutionVertex(final ExecutionVertex vertex) {
-		if (this.vertexMap.put(vertex.getID(), vertex) != null) {
-			throw new IllegalStateException("There is already an execution vertex with ID " + vertex.getID()
-				+ " registered");
-		}
-	}
-
-	/**
-	 * Returns the execution vertex with the given vertex ID.
-	 * 
-	 * @param id
-	 *        the vertex ID to retrieve the execution vertex
-	 * @return the execution vertex matching the provided vertex ID or <code>null</code> if no such vertex could be
-	 *         found
-	 */
-	public ExecutionVertex getVertexByID(final ExecutionVertexID id) {
-		return this.vertexMap.get(id);
-	}
-
-	/**
-	 * Checks if the current execution stage has been successfully completed, i.e.
-	 * all vertices in this stage have successfully finished their execution.
-	 * 
-	 * @return <code>true</code> if stage is completed, <code>false</code> otherwise
-	 */
-	private boolean isCurrentStageCompleted() {
-		if (this.indexToCurrentExecutionStage >= this.stages.size()) {
-			return true;
-		}
-
-		final ExecutionGraphIterator it = new ExecutionGraphIterator(this, this.indexToCurrentExecutionStage, true,
-			true);
-		while (it.hasNext()) {
-			final ExecutionVertex vertex = it.next();
-			if (vertex.getExecutionState() != ExecutionState.FINISHED) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Checks if the execution of execution graph is finished.
-	 * 
-	 * @return <code>true</code> if the execution of the graph is finished, <code>false</code> otherwise
-	 */
-	public boolean isExecutionFinished() {
-		return (getJobStatus() == InternalJobStatus.FINISHED);
-	}
-
-	/**
-	 * Returns the job ID of the job configuration this execution graph was originally constructed from.
-	 * 
-	 * @return the job ID of the job configuration this execution graph was originally constructed from
-	 */
-	public JobID getJobID() {
-		return this.jobID;
-	}
-
-	/**
-	 * Returns the index of the current execution stage.
-	 * 
-	 * @return the index of the current execution stage
-	 */
-	public int getIndexOfCurrentExecutionStage() {
-		return this.indexToCurrentExecutionStage;
-	}
-
-	/**
-	 * Returns the stage which is currently executed.
-	 * 
-	 * @return the currently executed stage or <code>null</code> if the job execution is already completed
-	 */
-	public ExecutionStage getCurrentExecutionStage() {
-
-		try {
-			return this.stages.get(this.indexToCurrentExecutionStage);
-		} catch (ArrayIndexOutOfBoundsException e) {
-			return null;
-		}
-	}
-
-	public void repairStages() {
-
-		final Map<ExecutionGroupVertex, Integer> stageNumbers = new HashMap<ExecutionGroupVertex, Integer>();
-		ExecutionGroupVertexIterator it = new ExecutionGroupVertexIterator(this, true, -1);
-
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			int precedingNumber = 0;
-			if (stageNumbers.containsKey(groupVertex)) {
-				precedingNumber = stageNumbers.get(groupVertex).intValue();
-			} else {
-				stageNumbers.put(groupVertex, Integer.valueOf(precedingNumber));
-			}
-
-			for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) {
-
-				final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i);
-				if (!stageNumbers.containsKey(edge.getTargetVertex())) {
-					// Target vertex has not yet been discovered
-					// Same stage as preceding vertex
-					stageNumbers.put(edge.getTargetVertex(), Integer.valueOf(precedingNumber));
-				} else {
-					final int stageNumber = stageNumbers.get(edge.getTargetVertex()).intValue();
-					if (stageNumber != precedingNumber) {
-						stageNumbers.put(edge.getTargetVertex(), (int) Math.max(precedingNumber, stageNumber));
-					}
-				}
-			}
-		}
-
-		// Traverse the graph backwards (starting from the output vertices) to make sure vertices are allocated in a
-		// stage as high as possible
-		it = new ExecutionGroupVertexIterator(this, false, -1);
-
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			final int succeedingNumber = stageNumbers.get(groupVertex);
-
-			for (int i = 0; i < groupVertex.getNumberOfBackwardLinks(); i++) {
-
-				final ExecutionGroupEdge edge = groupVertex.getBackwardEdge(i);
-				final int stageNumber = stageNumbers.get(edge.getSourceVertex());
-				if (stageNumber != succeedingNumber) {
-					throw new IllegalStateException(edge.getSourceVertex() + " and " + edge.getTargetVertex()
-						+ " are assigned to different stages");
-				}
-			}
-		}
-
-		// Finally, assign the new stage numbers
-		this.stages.clear();
-		final Iterator<Map.Entry<ExecutionGroupVertex, Integer>> it2 = stageNumbers.entrySet().iterator();
-		while (it2.hasNext()) {
-
-			final Map.Entry<ExecutionGroupVertex, Integer> entry = it2.next();
-			final ExecutionGroupVertex groupVertex = entry.getKey();
-			final int stageNumber = entry.getValue().intValue();
-			// Prevent out of bounds exceptions
-			while (this.stages.size() <= stageNumber) {
-				this.stages.add(null);
-			}
-			ExecutionStage executionStage = this.stages.get(stageNumber);
-			// If the stage not yet exists,
-			if (executionStage == null) {
-				executionStage = new ExecutionStage(this, stageNumber);
-				this.stages.set(stageNumber, executionStage);
-			}
-
-			executionStage.addStageMember(groupVertex);
-			groupVertex.setExecutionStage(executionStage);
-		}
-	}
-
-	public void repairInstanceSharing() {
-
-		final Set<AllocatedResource> availableResources = new LinkedHashSet<AllocatedResource>();
-
-		final Iterator<ExecutionGroupVertex> it = new ExecutionGroupVertexIterator(this, true, -1);
-		while (it.hasNext()) {
-			final ExecutionGroupVertex groupVertex = it.next();
-			if (groupVertex.getVertexToShareInstancesWith() == null) {
-				availableResources.clear();
-				groupVertex.repairInstanceSharing(availableResources);
-			}
-		}
-	}
-
-	public void repairInstanceAssignment() {
-
-		Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-		while (it.hasNext()) {
-
-			final ExecutionVertex sourceVertex = it.next();
-
-			for (int i = 0; i < sourceVertex.getNumberOfOutputGates(); ++i) {
-
-				final ExecutionGate outputGate = sourceVertex.getOutputGate(i);
-				final ChannelType channelType = outputGate.getChannelType();
-				if (channelType == ChannelType.IN_MEMORY) {
-					final int numberOfOutputChannels = outputGate.getNumberOfEdges();
-					for (int j = 0; j < numberOfOutputChannels; ++j) {
-						final ExecutionEdge outputChannel = outputGate.getEdge(j);
-						outputChannel.getInputGate().getVertex()
-							.setAllocatedResource(sourceVertex.getAllocatedResource());
-					}
-				}
-			}
-		}
-
-		it = new ExecutionGraphIterator(this, false);
-		while (it.hasNext()) {
-
-			final ExecutionVertex targetVertex = it.next();
-
-			for (int i = 0; i < targetVertex.getNumberOfInputGates(); ++i) {
-
-				final ExecutionGate inputGate = targetVertex.getInputGate(i);
-				final ChannelType channelType = inputGate.getChannelType();
-				if (channelType == ChannelType.IN_MEMORY) {
-					final int numberOfInputChannels = inputGate.getNumberOfEdges();
-					for (int j = 0; j < numberOfInputChannels; ++j) {
-						final ExecutionEdge inputChannel = inputGate.getEdge(j);
-						inputChannel.getOutputGate().getVertex()
-							.setAllocatedResource(targetVertex.getAllocatedResource());
-					}
-				}
-			}
-		}
-	}
-
-	public ChannelType getChannelType(final ExecutionVertex sourceVertex, final ExecutionVertex targetVertex) {
-
-		final ExecutionGroupVertex sourceGroupVertex = sourceVertex.getGroupVertex();
-		final ExecutionGroupVertex targetGroupVertex = targetVertex.getGroupVertex();
-
-		final List<ExecutionGroupEdge> edges = sourceGroupVertex.getForwardEdges(targetGroupVertex);
-		if (edges.size() == 0) {
-			return null;
-		}
-
-		// On a task level, the two vertices are connected
-		final ExecutionGroupEdge edge = edges.get(0);
-
-		// Now lets see if these two concrete subtasks are connected
-		final ExecutionGate outputGate = sourceVertex.getOutputGate(edge.getIndexOfOutputGate());
-		for (int i = 0; i < outputGate.getNumberOfEdges(); ++i) {
-
-			final ExecutionEdge outputChannel = outputGate.getEdge(i);
-			if (targetVertex == outputChannel.getInputGate().getVertex()) {
-				return edge.getChannelType();
-			}
-		}
-
-		return null;
-	}
-
-	/**
-	 * Returns the job configuration that was originally attached to the job graph.
-	 * 
-	 * @return the job configuration that was originally attached to the job graph
-	 */
-	public Configuration getJobConfiguration() {
-		return this.jobConfiguration;
-	}
-
-	/**
-	 * Checks whether the job represented by the execution graph has the status <code>FINISHED</code>.
-	 * 
-	 * @return <code>true</code> if the job has the status <code>CREATED</code>, <code>false</code> otherwise
-	 */
-	private boolean jobHasFinishedStatus() {
-
-		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-
-		while (it.hasNext()) {
-
-			if (it.next().getExecutionState() != ExecutionState.FINISHED) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Checks whether the job represented by the execution graph has the status <code>SCHEDULED</code>.
-	 * 
-	 * @return <code>true</code> if the job has the status <code>SCHEDULED</code>, <code>false</code> otherwise
-	 */
-	private boolean jobHasScheduledStatus() {
-
-		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-
-		while (it.hasNext()) {
-
-			final ExecutionState s = it.next().getExecutionState();
-			if (s != ExecutionState.CREATED && s != ExecutionState.SCHEDULED && s != ExecutionState.READY) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Checks whether the job represented by the execution graph has the status <code>CANCELED</code> or
-	 * <code>FAILED</code>.
-	 * 
-	 * @return <code>true</code> if the job has the status <code>CANCELED</code> or <code>FAILED</code>,
-	 *         <code>false</code> otherwise
-	 */
-	private boolean jobHasFailedOrCanceledStatus() {
-
-		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-
-		while (it.hasNext()) {
-
-			final ExecutionState state = it.next().getExecutionState();
-
-			if (state != ExecutionState.CANCELED && state != ExecutionState.FAILED && state != ExecutionState.FINISHED) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	private static InternalJobStatus determineNewJobStatus(final ExecutionGraph eg,
-			final ExecutionState latestStateChange) {
-
-		final InternalJobStatus currentJobStatus = eg.getJobStatus();
-
-		switch (currentJobStatus) {
-		case CREATED:
-			if (eg.jobHasScheduledStatus()) {
-				return InternalJobStatus.SCHEDULED;
-			} else if (latestStateChange == ExecutionState.CANCELED) {
-				if (eg.jobHasFailedOrCanceledStatus()) {
-					return InternalJobStatus.CANCELED;
-				}
-			}else if(latestStateChange == ExecutionState.FAILED){
-				return InternalJobStatus.FAILING;
-			}
-			break;
-		case SCHEDULED:
-			if (latestStateChange == ExecutionState.RUNNING) {
-				return InternalJobStatus.RUNNING;
-			} else if (latestStateChange == ExecutionState.CANCELED) {
-				if (eg.jobHasFailedOrCanceledStatus()) {
-					return InternalJobStatus.CANCELED;
-				}
-			}else if(latestStateChange == ExecutionState.FAILED){
-				return InternalJobStatus.FAILING;
-			}
-			break;
-		case RUNNING:
-			if (latestStateChange == ExecutionState.CANCELED) {
-				return InternalJobStatus.CANCELING;
-			}
-			if (latestStateChange == ExecutionState.FAILED) {
-
-				final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(eg, true);
-				while (it.hasNext()) {
-
-					final ExecutionVertex vertex = it.next();
-					if (vertex.getExecutionState() == ExecutionState.FAILED) {
-						return InternalJobStatus.FAILING;
-					}
-				}
-			}
-			if (eg.jobHasFinishedStatus()) {
-				return InternalJobStatus.FINISHED;
-			}
-			break;
-		case FAILING:
-			if (eg.jobHasFailedOrCanceledStatus()) {
-				return InternalJobStatus.FAILED;
-			}
-			break;
-		case FAILED:
-			LOG.error("Received update of execute state in job status FAILED");
-			break;
-		case CANCELING:
-			if (eg.jobHasFailedOrCanceledStatus()) {
-				return InternalJobStatus.CANCELED;
-			}
-			break;
-		case CANCELED:
-			LOG.error("Received update of execute state in job status CANCELED: " + eg.getJobID());
-			break;
-		case FINISHED:
-			LOG.error("Received update of execute state in job status FINISHED: " + eg.getJobID() + " "
-				+ StringUtils.stringifyException(new Throwable()));
-			break;
-		}
-
-		return currentJobStatus;
-	}
-
-	/**
-	 * Returns the current status of the job
-	 * represented by this execution graph.
-	 * 
-	 * @return the current status of the job
-	 */
-	public InternalJobStatus getJobStatus() {
-
-		return this.jobStatus.get();
-	}
-
-
-	@Override
-	public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID,
-			final ExecutionState newExecutionState, String optionalMessage) {
-
-		// Do not use the parameter newExecutionState here as it may already be out-dated
-
-		final ExecutionVertex vertex = getVertexByID(vertexID);
-		if (vertex == null) {
-			LOG.error("Cannot find execution vertex with the ID " + vertexID);
-			return;
-		}
-
-		final ExecutionState actualExecutionState = vertex.getExecutionState();
-
-		final InternalJobStatus newJobStatus = determineNewJobStatus(this, actualExecutionState);
-
-		if (actualExecutionState == ExecutionState.FINISHED) {
-			// It is worth checking if the current stage has complete
-			if (this.isCurrentStageCompleted()) {
-				// Increase current execution stage
-				++this.indexToCurrentExecutionStage;
-
-				if (this.indexToCurrentExecutionStage < this.stages.size()) {
-					final Iterator<ExecutionStageListener> it = this.executionStageListeners.iterator();
-					final ExecutionStage nextExecutionStage = getCurrentExecutionStage();
-					while (it.hasNext()) {
-						it.next().nextExecutionStageEntered(jobID, nextExecutionStage);
-					}
-				}
-			}
-		}
-
-		updateJobStatus(newJobStatus, optionalMessage);
-	}
-
-	/**
-	 * Updates the job status to given status and triggers the execution of the {@link JobStatusListener} objects.
-	 * 
-	 * @param newJobStatus
-	 *        the new job status
-	 * @param optionalMessage
-	 *        an optional message providing details on the reasons for the state change
-	 */
-	public void updateJobStatus(final InternalJobStatus newJobStatus, String optionalMessage) {
-
-		// Check if the new job status equals the old one
-		if (this.jobStatus.getAndSet(newJobStatus) == newJobStatus) {
-			return;
-		}
-
-		// The task caused the entire job to fail, save the error description
-		if (newJobStatus == InternalJobStatus.FAILING) {
-			this.errorDescription = optionalMessage;
-		}
-
-		// If this is the final failure state change, reuse the saved error description
-		if (newJobStatus == InternalJobStatus.FAILED) {
-			optionalMessage = this.errorDescription;
-		}
-
-		final Iterator<JobStatusListener> it = this.jobStatusListeners.iterator();
-		while (it.hasNext()) {
-			it.next().jobStatusHasChanged(this, newJobStatus, optionalMessage);
-		}
-	}
-
-	/**
-	 * Registers a new {@link JobStatusListener} object with this execution graph.
-	 * After being registered the object will receive notifications about changes
-	 * of the job status. It is not possible to register the same listener object
-	 * twice.
-	 * 
-	 * @param jobStatusListener
-	 *        the listener object to register
-	 */
-	public void registerJobStatusListener(final JobStatusListener jobStatusListener) {
-
-		if (jobStatusListener == null) {
-			throw new IllegalArgumentException("Argument jobStatusListener must not be null");
-		}
-
-		this.jobStatusListeners.addIfAbsent(jobStatusListener);
-	}
-
-	/**
-	 * Unregisters the given {@link JobStatusListener} object. After having called this
-	 * method, the object will no longer receive notifications about changes of the job
-	 * status.
-	 * 
-	 * @param jobStatusListener
-	 *        the listener object to unregister
-	 */
-	public void unregisterJobStatusListener(final JobStatusListener jobStatusListener) {
-
-		if (jobStatusListener == null) {
-			throw new IllegalArgumentException("Argument jobStatusListener must not be null");
-		}
-
-		this.jobStatusListeners.remove(jobStatusListener);
-	}
-
-	/**
-	 * Registers a new {@link ExecutionStageListener} object with this execution graph. After being registered the
-	 * object will receive a notification whenever the job has entered its next execution stage. Note that a
-	 * notification is not sent when the job has entered its initial execution stage.
-	 * 
-	 * @param executionStageListener
-	 *        the listener object to register
-	 */
-	public void registerExecutionStageListener(final ExecutionStageListener executionStageListener) {
-
-		if (executionStageListener == null) {
-			throw new IllegalArgumentException("Argument executionStageListener must not be null");
-		}
-
-		this.executionStageListeners.addIfAbsent(executionStageListener);
-	}
-
-	/**
-	 * Unregisters the given {@link ExecutionStageListener} object. After having called this method, the object will no
-	 * longer receiver notifications about the execution stage progress.
-	 * 
-	 * @param executionStageListener
-	 *        the listener object to unregister
-	 */
-	public void unregisterExecutionStageListener(final ExecutionStageListener executionStageListener) {
-
-		if (executionStageListener == null) {
-			throw new IllegalArgumentException("Argument executionStageListener must not be null");
-		}
-
-		this.executionStageListeners.remove(executionStageListener);
-	}
-
-	/**
-	 * Returns the name of the original job graph.
-	 * 
-	 * @return the name of the original job graph, possibly <code>null</code>
-	 */
-	public String getJobName() {
-		return this.jobName;
-	}
-
-	@Override
-	public void userThreadStarted(JobID jobID, ExecutionVertexID vertexID, Thread userThread) {}
-
-	@Override
-	public void userThreadFinished(JobID jobID, ExecutionVertexID vertexID, Thread userThread) {}
-
-	/**
-	 * Reconstructs the execution pipelines for the entire execution graph.
-	 */
-	private void reconstructExecutionPipelines() {
-		final Iterator<ExecutionStage> it = this.stages.iterator();
-		while (it.hasNext()) {
-
-			it.next().reconstructExecutionPipelines();
-		}
-	}
-
-	/**
-	 * Returns an iterator over all execution stages contained in this graph.
-	 * 
-	 * @return an iterator over all execution stages contained in this graph
-	 */
-	public Iterator<ExecutionStage> iterator() {
-		return this.stages.iterator();
-	}
-
-	/**
-	 * Performs an asynchronous update operation to this execution graph.
-	 * 
-	 * @param command
-	 *        the update command to be asynchronously executed on this graph
-	 */
-	public void executeCommand(final Runnable command) {
-		this.executorService.execute(command);
+		
+		this.executor.submit(action);
 	}
+}
 	
-	private void calculateConnectionIDs() {
-		final Set<ExecutionGroupVertex> alreadyVisited = new HashSet<ExecutionGroupVertex>();
-		final ExecutionStage lastStage = getStage(getNumberOfStages() - 1);
-
-		for (int i = 0; i < lastStage.getNumberOfStageMembers(); ++i) {
-
-			final ExecutionGroupVertex groupVertex = lastStage.getStageMember(i);
-
-			int currentConnectionID = 0;
-
-			if (groupVertex.isOutputVertex()) {
-			currentConnectionID = groupVertex.calculateConnectionID(currentConnectionID, alreadyVisited);
-			}
-		}
-	}
 	
-	/**
-	 * Retrieves the number of required slots to run this execution graph
-	 * @return
-	 */
-	public int getRequiredSlots(){
-		int maxRequiredSlots = 0;
-
-		final Iterator<ExecutionStage> stageIterator = this.stages.iterator();
-
-		while(stageIterator.hasNext()){
-			final ExecutionStage stage = stageIterator.next();
-
-			int requiredSlots = stage.getRequiredSlots();
-
-			if(requiredSlots > maxRequiredSlots){
-				maxRequiredSlots = requiredSlots;
-			}
-		}
-
-		return maxRequiredSlots;
-	}
-}
+	
+	
+	
+	
+	
+	
+	
+	
+	
+	
+	
+	
+	
+	
+	
+//	
+//	/**
+//	 * Applies the user defined settings to the execution graph.
+//	 * 
+//	 * @param temporaryGroupVertexMap
+//	 *        mapping between job vertices and the corresponding group vertices.
+//	 * @throws GraphConversionException
+//	 *         thrown if an error occurs while applying the user settings.
+//	 */
+//	private void applyUserDefinedSettings(final HashMap<AbstractJobVertex, ExecutionGroupVertex> temporaryGroupVertexMap)
+//			throws GraphConversionException {
+//
+//		// The check for cycles in the dependency chain for instance sharing is already checked in
+//		// <code>submitJob</code> method of the job manager
+//
+//		// If there is no cycle, apply the settings to the corresponding group vertices
+//		final Iterator<Map.Entry<AbstractJobVertex, ExecutionGroupVertex>> it = temporaryGroupVertexMap.entrySet()
+//			.iterator();
+//		while (it.hasNext()) {
+//
+//			final Map.Entry<AbstractJobVertex, ExecutionGroupVertex> entry = it.next();
+//			final AbstractJobVertex jobVertex = entry.getKey();
+//			if (jobVertex.getVertexToShareInstancesWith() != null) {
+//
+//				final AbstractJobVertex vertexToShareInstancesWith = jobVertex.getVertexToShareInstancesWith();
+//				final ExecutionGroupVertex groupVertex = entry.getValue();
+//				final ExecutionGroupVertex groupVertexToShareInstancesWith = temporaryGroupVertexMap
+//					.get(vertexToShareInstancesWith);
+//				groupVertex.shareInstancesWith(groupVertexToShareInstancesWith);
+//			}
+//		}
+//
+//		// Second, we create the number of execution vertices each group vertex is supposed to manage
+//		Iterator<ExecutionGroupVertex> it2 = new ExecutionGroupVertexIterator(this, true, -1);
+//		while (it2.hasNext()) {
+//
+//			final ExecutionGroupVertex groupVertex = it2.next();
+//			if (groupVertex.isNumberOfMembersUserDefined()) {
+//				groupVertex.createInitialExecutionVertices(groupVertex.getUserDefinedNumberOfMembers());
+//			}
+//		}
+//
+//		// Finally, apply the channel settings channel settings
+//		it2 = new ExecutionGroupVertexIterator(this, true, -1);
+//		while (it2.hasNext()) {
+//
+//			final ExecutionGroupVertex groupVertex = it2.next();
+//			for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) {
+//
+//				final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i);
+//				if (edge.isChannelTypeUserDefined()) {
+//					edge.changeChannelType(edge.getChannelType());
+//				}
+//
+//				// Create edges between execution vertices
+//				createExecutionEdgesForGroupEdge(edge);
+//			}
+//		}
+//
+//		// Repair the instance assignment after having changed the channel types
+//		repairInstanceAssignment();
+//
+//		// Repair the instance sharing among different group vertices
+//		repairInstanceSharing();
+//
+//		// Finally, repair the stages
+//		repairStages();
+//	}
+//
+//	/**
+//	 * Sets up an execution graph from a job graph.
+//	 * 
+//	 * @param jobGraph
+//	 *        the job graph to create the execution graph from
+//	 * @param defaultParallelism
+//	 *        defaultParallelism in case that nodes have no parallelism set
+//	 * @throws GraphConversionException
+//	 *         thrown if the job graph is not valid and no execution graph can be constructed from it
+//	 */
+//	private void constructExecutionGraph(final JobGraph jobGraph, final int defaultParallelism)
+//			throws GraphConversionException {
+//
+//		// Clean up temporary data structures
+//		final HashMap<AbstractJobVertex, ExecutionVertex> temporaryVertexMap = new HashMap<AbstractJobVertex, ExecutionVertex>();
+//		final HashMap<AbstractJobVertex, ExecutionGroupVertex> temporaryGroupVertexMap = new HashMap<AbstractJobVertex, ExecutionGroupVertex>();
+//
+//		// Initially, create only one execution stage that contains all group vertices
+//		final ExecutionStage initialExecutionStage = new ExecutionStage(this, 0);
+//		this.stages.add(initialExecutionStage);
+//
+//		// Convert job vertices to execution vertices and initialize them
+//		final AbstractJobVertex[] all = jobGraph.getAllJobVertices();
+//		for (int i = 0; i < all.length; i++) {
+//			if(all[i].getNumberOfSubtasks() == -1){
+//				all[i].setNumberOfSubtasks(defaultParallelism);
+//			}
+//
+//			final ExecutionVertex createdVertex = createVertex(all[i], initialExecutionStage);
+//			temporaryVertexMap.put(all[i], createdVertex);
+//			temporaryGroupVertexMap.put(all[i], createdVertex.getGroupVertex());
+//		}
+//
+//		// Create initial edges between the vertices
+//		createInitialGroupEdges(temporaryVertexMap);
+//
+//		// Now that an initial graph is built, apply the user settings
+//		applyUserDefinedSettings(temporaryGroupVertexMap);
+//
+//		// Calculate the connection IDs
+//		calculateConnectionIDs();
+//
+//		// Finally, construct the execution pipelines
+//		reconstructExecutionPipelines();
+//	}
+//
+//	private void createExecutionEdgesForGroupEdge(final ExecutionGroupEdge groupEdge) {
+//
+//		final ExecutionGroupVertex source = groupEdge.getSourceVertex();
+//		final int indexOfOutputGate = groupEdge.getIndexOfOutputGate();
+//		final ExecutionGroupVertex target = groupEdge.getTargetVertex();
+//		final int indexOfInputGate = groupEdge.getIndexOfInputGate();
+//
+//		final Map<GateID, List<ExecutionEdge>> inputChannelMap = new HashMap<GateID, List<ExecutionEdge>>();
+//
+//		// Unwire the respective gate of the source vertices
+//		final int currentNumberOfSourceNodes = source.getCurrentNumberOfGroupMembers();
+//		for (int i = 0; i < currentNumberOfSourceNodes; ++i) {
+//
+//			final ExecutionVertex sourceVertex = source.getGroupMember(i);
+//			final ExecutionGate outputGate = sourceVertex.getOutputGate(indexOfOutputGate);
+//			if (outputGate == null) {
+//				throw new IllegalStateException("wire: " + sourceVertex.getName()
+//					+ " has no output gate with index " + indexOfOutputGate);
+//			}
+//
+//			if (outputGate.getNumberOfEdges() > 0) {
+//				throw new IllegalStateException("wire: wire called on source " + sourceVertex.getName() + " (" + i
+//					+ "), but number of output channels is " + outputGate.getNumberOfEdges() + "!");
+//			}
+//
+//			final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers();
+//			final List<ExecutionEdge> outputChannels = new ArrayList<ExecutionEdge>();
+//
+//			for (int j = 0; j < currentNumberOfTargetNodes; ++j) {
+//
+//				final ExecutionVertex targetVertex = target.getGroupMember(j);
+//				final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate);
+//				if (inputGate == null) {
+//					throw new IllegalStateException("wire: " + targetVertex.getName()
+//						+ " has no input gate with index " + indexOfInputGate);
+//				}
+//
+//				if (inputGate.getNumberOfEdges() > 0 && i == 0) {
+//					throw new IllegalStateException("wire: wire called on target " + targetVertex.getName() + " ("
+//						+ j + "), but number of input channels is " + inputGate.getNumberOfEdges() + "!");
+//				}
+//
+//				// Check if a wire is supposed to be created
+//				if (DistributionPatternProvider.createWire(groupEdge.getDistributionPattern(),
+//					i, j, currentNumberOfSourceNodes, currentNumberOfTargetNodes)) {
+//
+//					final ChannelID outputChannelID = new ChannelID();
+//					final ChannelID inputChannelID = new ChannelID();
+//
+//					final ExecutionEdge edge = new ExecutionEdge(outputGate, inputGate, groupEdge, outputChannelID,
+//						inputChannelID, outputGate.getNumberOfEdges(), inputGate.getNumberOfEdges());
+//
+//					this.edgeMap.put(outputChannelID, edge);
+//					this.edgeMap.put(inputChannelID, edge);
+//
+//					outputChannels.add(edge);
+//
+//					List<ExecutionEdge> inputChannels = inputChannelMap.get(inputGate.getGateID());
+//					if (inputChannels == null) {
+//						inputChannels = new ArrayList<ExecutionEdge>();
+//						inputChannelMap.put(inputGate.getGateID(), inputChannels);
+//					}
+//
+//					inputChannels.add(edge);
+//				}
+//			}
+//
+//			outputGate.replaceAllEdges(outputChannels);
+//		}
+//
+//		// Finally, set the channels for the input gates
+//		final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers();
+//		for (int i = 0; i < currentNumberOfTargetNodes; ++i) {
+//
+//			final ExecutionVertex targetVertex = target.getGroupMember(i);
+//			final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate);
+//
+//			final List<ExecutionEdge> inputChannels = inputChannelMap.get(inputGate.getGateID());
+//			if (inputChannels == null) {
+//				LOG.error("Cannot find input channels for gate ID " + inputGate.getGateID());
+//				continue;
+//			}
+//
+//			inputGate.replaceAllEdges(inputChannels);
+//		}
+//
+//	}
+//
+//	/**
+//	 * Creates the initial edges between the group vertices
+//	 * 
+//	 * @param vertexMap
+//	 *        the temporary vertex map
+//	 * @throws GraphConversionException
+//	 *         if the initial wiring cannot be created
+//	 */
+//	private void createInitialGroupEdges(final HashMap<AbstractJobVertex, ExecutionVertex> vertexMap)
+//			throws GraphConversionException {
+//
+//		Iterator<Map.Entry<AbstractJobVertex, ExecutionVertex>> it = vertexMap.entrySet().iterator();
+//
+//		while (it.hasNext()) {
+//
+//			final Map.Entry<AbstractJobVertex, ExecutionVertex> entry = it.next();
+//			final AbstractJobVertex sjv = entry.getKey();
+//			final ExecutionVertex sev = entry.getValue();
+//			final ExecutionGroupVertex sgv = sev.getGroupVertex();
+//
+//			// First, build the group edges
+//			for (int i = 0; i < sjv.getNumberOfForwardConnections(); ++i) {
+//				final JobEdge edge = sjv.getForwardConnection(i);
+//				final AbstractJobVertex tjv = edge.getConnectedVertex();
+//
+//				final ExecutionVertex tev = vertexMap.get(tjv);
+//				final ExecutionGroupVertex tgv = tev.getGroupVertex();
+//				// Use NETWORK as default channel type if nothing else is defined by the user
+//				ChannelType channelType = edge.getChannelType();
+//				boolean userDefinedChannelType = true;
+//				if (channelType == null) {
+//					userDefinedChannelType = false;
+//					channelType = ChannelType.NETWORK;
+//				}
+//
+//				final DistributionPattern distributionPattern = edge.getDistributionPattern();
+//
+//				// Connect the corresponding group vertices and copy the user settings from the job edge
+//				final ExecutionGroupEdge groupEdge = sgv.wireTo(tgv, edge.getIndexOfInputGate(), i, channelType,
+//					userDefinedChannelType,distributionPattern);
+//
+//				final ExecutionGate outputGate = new ExecutionGate(new GateID(), sev, groupEdge, false);
+//				sev.insertOutputGate(i, outputGate);
+//				final ExecutionGate inputGate = new ExecutionGate(new GateID(), tev, groupEdge, true);
+//				tev.insertInputGate(edge.getIndexOfInputGate(), inputGate);
+//			}
+//		}
+//	}
+//
+//	/**
+//	 * Creates an execution vertex from a job vertex.
+//	 * 
+//	 * @param jobVertex
+//	 *        the job vertex to create the execution vertex from
+//	 * @param initialExecutionStage
+//	 *        the initial execution stage all group vertices are added to
+//	 * @return the new execution vertex
+//	 * @throws GraphConversionException
+//	 *         thrown if the job vertex is of an unknown subclass
+//	 */
+//	private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final ExecutionStage initialExecutionStage)
+//			throws GraphConversionException {
+//
+//		// Create an initial execution vertex for the job vertex
+//		final Class<? extends AbstractInvokable> invokableClass = jobVertex.getInvokableClass();
+//		if (invokableClass == null) {
+//			throw new GraphConversionException("JobVertex " + jobVertex.getID() + " (" + jobVertex.getName()
+//				+ ") does not specify a task");
+//		}
+//
+//		// Calculate the cryptographic signature of this vertex
+//		final ExecutionSignature signature = ExecutionSignature.createSignature(jobVertex.getInvokableClass(),
+//			jobVertex.getJobGraph().getJobID());
+//
+//		// Create a group vertex for the job vertex
+//
+//		ExecutionGroupVertex groupVertex = null;
+//		try {
+//			groupVertex = new ExecutionGroupVertex(jobVertex.getName(), jobVertex.getID(), this,
+//				jobVertex.getNumberOfSubtasks(), jobVertex.getVertexToShareInstancesWith() != null ? true
+//					: false, 0, jobVertex.getConfiguration(), signature,
+//				invokableClass);
+//		} catch (Throwable t) {
+//			throw new GraphConversionException(t);
+//		}
+//
+//		// Register input and output vertices separately
+//		if (jobVertex instanceof AbstractJobInputVertex) {
+//
+//			final AbstractJobInputVertex jobInputVertex = (AbstractJobInputVertex) jobVertex;
+//			
+//			if (jobVertex instanceof InputFormatVertex) {
+//				try {
+//					// get a handle to the user code class loader
+//					ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID());
+//					
+//					((InputFormatVertex) jobVertex).initializeInputFormatFromTaskConfig(cl);
+//				}
+//				catch (Throwable t) {
+//					throw new GraphConversionException("Could not deserialize input format.", t);
+//				}
+//			}
+//			
+//			final Class<? extends InputSplit> inputSplitType = jobInputVertex.getInputSplitType();
+//			
+//			InputSplit[] inputSplits;
+//
+//			try {
+//				inputSplits = jobInputVertex.getInputSplits(jobVertex.getNumberOfSubtasks());
+//			}
+//			catch (Throwable t) {
+//				throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName(), t);
+//			}
+//
+//			if (inputSplits == null) {
+//				inputSplits = new InputSplit[0];
+//			}
+//			
+//			LOG.info("Job input vertex " + jobVertex.getName() + " generated " + inputSplits.length + " input splits");
+//
+//			// assign input splits and type
+//			groupVertex.setInputSplits(inputSplits);
+//			groupVertex.setInputSplitType(inputSplitType);
+//		}
+//
+//		if (jobVertex instanceof OutputFormatVertex){
+//			final OutputFormatVertex jobOutputVertex = (OutputFormatVertex) jobVertex;
+//			
+//			try {
+//				// get a handle to the user code class loader
+//				ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID());
+//				jobOutputVertex.initializeOutputFormatFromTaskConfig(cl);
+//			}
+//			catch (Throwable t) {
+//				throw new GraphConversionException("Could not deserialize output format.", t);
+//			}
+//
+//			OutputFormat<?> outputFormat = jobOutputVertex.getOutputFormat();
+//			if (outputFormat != null && outputFormat instanceof InitializeOnMaster){
+//				try {
+//					((InitializeOnMaster) outputFormat).initializeGlobal(jobVertex.getNumberOfSubtasks());
+//				}
+//				catch (Throwable t) {
+//					throw new GraphConversionException(t);
+//				}
+//			}
+//		}
+//
+//		// Add group vertex to initial execution stage
+//		initialExecutionStage.addStageMember(groupVertex);
+//
+//		final ExecutionVertex ev = new ExecutionVertex(this, groupVertex, jobVertex.getNumberOfForwardConnections(),
+//			jobVertex.getNumberOfBackwardConnections());
+//
+//		// Assign initial instance to vertex (may be overwritten later on when user settings are applied)
+//		ev.setAllocatedResource(new AllocatedResource(DummyInstance.createDummyInstance(), null));
+//
+//		return ev;
+//	}
+//
+//	/**
+//	 * Returns the number of input vertices registered with this execution graph.
+//	 * 
+//	 * @return the number of input vertices registered with this execution graph
+//	 */
+//	public int getNumberOfInputVertices() {
+//		return this.stages.get(0).getNumberOfInputExecutionVertices();
+//	}
+//
+//	/**
+//	 * Returns the number of input vertices for the given stage.
+//	 * 
+//	 * @param stage
+//	 *        the index of the execution stage
+//	 * @return the number of input vertices for the given stage
+//	 */
+//	public int getNumberOfInputVertices(int stage) {
+//		if (stage >= this.stages.size()) {
+//			return 0;
+//		}
+//
+//		return this.stages.get(stage).getNumberOfInputExecutionVertices();
+//	}
+//
+//	/**
+//	 * Returns the number of output vertices registered with this execution graph.
+//	 * 
+//	 * @return the number of output vertices registered with this execution graph
+//	 */
+//	public int getNumberOfOutputVertices() {
+//		return this.stages.get(0).getNumberOfOutputExecutionVertices();
+//	}
+//
+//	/**
+//	 * Returns the number of output vertices for the given stage.
+//	 * 
+//	 * @param stage
+//	 *        the index of the execution stage
+//	 * @return the number of input vertices for the given stage
+//	 */
+//	public int getNumberOfOutputVertices(final int stage) {
+//		if (stage >= this.stages.size()) {
+//			return 0;
+//		}
+//
+//		return this.stages.get(stage).getNumberOfOutputExecutionVertices();
+//	}
+//
+//	/**
+//	 * Returns the input vertex with the specified index.
+//	 * 
+//	 * @param index
+//	 *        the index of the input vertex to return
+//	 * @return the input vertex with the specified index or <code>null</code> if no input vertex with such an index
+//	 *         exists
+//	 */
+//	public ExecutionVertex getInputVertex(final int index) {
+//		return this.stages.get(0).getInputExecutionVertex(index);
+//	}
+//
+//	/**
+//	 * Returns the output vertex with the specified index.
+//	 * 
+//	 * @param index
+//	 *        the index of the output vertex to return
+//	 * @return the output vertex with the specified index or <code>null</code> if no output vertex with such an index
+//	 *         exists
+//	 */
+//	public ExecutionVertex getOutputVertex(final int index) {
+//		return this.stages.get(0).getOutputExecutionVertex(index);
+//	}
+//
+//	/**
+//	 * Returns the input vertex with the specified index for the given stage
+//	 * 
+//	 * @param stage
+//	 *        the index of the stage
+//	 * @param index
+//	 *        the index of the input vertex to return
+//	 * @return the input vertex with the specified index or <code>null</code> if no input vertex with such an index
+//	 *         exists in that stage
+//	 */
+//	public ExecutionVertex getInputVertex(final int stage, final int index) {
+//		try {
+//			final ExecutionStage s = this.stages.get(stage);
+//			if (s == null) {
+//				return null;
+//			}
+//
+//			return s.getInputExecutionVertex(index);
+//
+//		} catch (ArrayIndexOutOfBoundsException e) {
+//			return null;
+//		}
+//	}
+//
+//	/**
+//	 * Returns the output vertex with the specified index for the given stage.
+//	 * 
+//	 * @param stage
+//	 *        the index of the stage
+//	 * @param index
+//	 *        the index of the output vertex to return
+//	 * @return the output vertex with the specified index or <code>null</code> if no output vertex with such an index
+//	 *         exists in that stage
+//	 */
+//	public ExecutionVertex getOutputVertex(final int stage, final int index) {
+//		try {
+//			final ExecutionStage s = this.stages.get(stage);
+//			if (s == null) {
+//				return null;
+//			}
+//
+//			return s.getOutputExecutionVertex(index);
+//
+//		} catch (ArrayIndexOutOfBoundsException e) {
+//			return null;
+//		}
+//	}
+//
+//	/**
+//	 * Identifies an execution by the specified channel ID and returns it.
+//	 * 
+//	 * @param id
+//	 *        the channel ID to identify the vertex with
+//	 * @return the execution vertex which has a channel with ID <code>id</code> or <code>null</code> if no such vertex
+//	 *         exists in the execution graph
+//	 */
+//	public ExecutionVertex getVertexByChannelID(final ChannelID id) {
+//		final ExecutionEdge edge = this.edgeMap.get(id);
+//		if (edge == null) {
+//			return null;
+//		}
+//
+//		if (id.equals(edge.getOutputChannelID())) {
+//			return edge.getOutputGate().getVertex();
+//		}
+//
+//		return edge.getInputGate().getVertex();
+//	}
+//
+//	/**
+//	 * Finds an {@link ExecutionEdge} by its ID and returns it.
+//	 * 
+//	 * @param id
+//	 *        the channel ID to identify the edge
+//	 * @return the edge whose ID matches <code>id</code> or <code>null</code> if no such edge is known
+//	 */
+//	public ExecutionEdge getEdgeByID(final ChannelID id) {
+//		return this.edgeMap.get(id);
+//	}
+//
+//	/**
+//	 * Registers an execution vertex with the execution graph.
+//	 * 
+//	 * @param vertex
+//	 *        the execution vertex to register
+//	 */
+//	void registerExecutionVertex(final ExecutionVertex vertex) {
+//		if (this.vertexMap.put(vertex.getID(), vertex) != null) {
+//			throw new IllegalStateException("There is already an execution vertex with ID " + vertex.getID()
+//				+ " registered");
+//		}
+//	}
+//
+//	/**
+//	 * Returns the execution vertex with the given vertex ID.
+//	 * 
+//	 * @param id
+//	 *        the vertex ID to retrieve the execution vertex
+//	 * @return the execution vertex matching the provided vertex ID or <code>null</code> if no such vertex could be
+//	 *         found
+//	 */
+//	public ExecutionVertex getVertexByID(final ExecutionVertexID id) {
+//		return this.vertexMap.get(id);
+//	}
+//
+//	/**
+//	 * Checks if the current execution stage has been successfully completed, i.e.
+//	 * all vertices in this stage have successfully finished their execution.
+//	 * 
+//	 * @return <code>true</code> if stage is completed, <code>false</code> otherwise
+//	 */
+//	private boolean isCurrentStageCompleted() {
+//		if (this.indexToCurrentExecutionStage >= this.stages.size()) {
+//			return true;
+//		}
+//
+//		final ExecutionGraphIterator it = new ExecutionGraphIterator(this, this.indexToCurrentExecutionStage, true,
+//			true);
+//		while (it.hasNext()) {
+//			final ExecutionVertex vertex = it.next();
+//			if (vertex.getExecutionState() != ExecutionState.FINISHED) {
+//				return false;
+//			}
+//		}
+//
+//		return true;
+//	}
+//
+//	/**
+//	 * Checks if the execution of execution graph is finished.
+//	 * 
+//	 * @return <code>true</code> if the execution of the graph is finished, <code>false</code> otherwise
+//	 */
+//	public boolean isExecutionFinished() {
+//		return (getJobStatus() == InternalJobStatus.FINISHED);
+//	}
+//
+//	/**
+//	 * Returns the job ID of the job configuration this execution graph was originally constructed from.
+//	 * 
+//	 * @return the job ID of the job configuration this execution graph was originally constructed from
+//	 */
+//	public JobID getJobID() {
+//		return this.jobID;
+//	}
+//
+//	/**
+//	 * Returns the index of the current execution stage.
+//	 * 
+//	 * @return the index of the current execution stage
+//	 */
+//	public int getIndexOfCurrentExecutionStage() {
+//		return this.indexToCurrentExecutionStage;
+//	}
+//
+//	/**
+//	 * Returns the stage which is currently executed.
+//	 * 
+//	 * @return the currently executed stage or <code>null</code> if the job execution is already completed
+//	 */
+//	public ExecutionStage getCurrentExecutionStage() {
+//
+//		try {
+//			return this.stages.get(this.indexToCurrentExecutionStage);
+//		} catch (ArrayIndexOutOfBoundsException e) {
+//			return null;
+//		}
+//	}
+//
+//	public void repairStages() {
+//
+//		final Map<ExecutionGroupVertex, Integer> stageNumbers = new HashMap<ExecutionGroupVertex, Integer>();
+//		ExecutionGroupVertexIterator it = new ExecutionGroupVertexIterator(this, true, -1);
+//
+//		while (it.hasNext()) {
+//
+//			final ExecutionGroupVertex groupVertex = it.next();
+//			int precedingNumber = 0;
+//			if (stageNumbers.containsKey(groupVertex)) {
+//				precedingNumber = stageNumbers.get(groupVertex).intValue();
+//			} else {
+//				stageNumbers.put(groupVertex, Integer.valueOf(precedingNumber));
+//			}
+//
+//			for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) {
+//
+//				final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i);
+//				if (!stageNumbers.containsKey(edge.getTargetVertex())) {
+//					// Target vertex has not yet been discovered
+//					// Same stage as preceding vertex
+//					stageNumbers.put(edge.getTargetVertex(), Integer.valueOf(precedingNumber));
+//				} else {
+//					final int stageNumber = stageNumbers.get(edge.getTargetVertex()).intValue();
+//					if (stageNumber != precedingNumber) {
+//						stageNumbers.put(edge.getTargetVertex(), (int) Math.max(precedingNumber, stageNumber));
+//					}
+//				}
+//			}
+//		}
+//
+//		// Traverse the graph backwards (starting from the output vertices) to make sure vertices are allocated in a
+//		// stage as high as possible
+//		it = new ExecutionGroupVertexIterator(this, false, -1);
+//
+//		while (it.hasNext()) {
+//
+//			final ExecutionGroupVertex groupVertex = it.next();
+//			final int succeedingNumber = stageNumbers.get(groupVertex);
+//
+//			for (int i = 0; i < groupVertex.getNumberOfBackwardLinks(); i++) {
+//
+//				final ExecutionGroupEdge edge = groupVertex.getBackwardEdge(i);
+//				final int stageNumber = stageNumbers.get(edge.getSourceVertex());
+//				if (stageNumber != succeedingNumber) {
+//					throw new IllegalStateException(edge.getSourceVertex() + " and " + edge.getTargetVertex()
+//						+ " are assigned to different stages");
+//				}
+//			}
+//		}
+//
+//		// Finally, assign the new stage numbers
+//		this.stages.clear();
+//		final Iterator<Map.Entry<ExecutionGroupVertex, Integer>> it2 = stageNumbers.entrySet().iterator();
+//		while (it2.hasNext()) {
+//
+//			final Map.Entry<ExecutionGroupVertex, Integer> entry = it2.next();
+//			final ExecutionGroupVertex groupVertex = entry.getKey();
+//			final int stageNumber = entry.getValue().intValue();
+//			// Prevent out of bounds exceptions
+//			while (this.stages.size() <= stageNumber) {
+//				this.stages.add(null);
+//			}
+//			ExecutionStage executionStage = this.stages.get(stageNumber);
+//			// If the stage not yet exists,
+//			if (executionStage == null) {
+//				executionStage = new ExecutionStage(this, stageNumber);
+//				this.stages.set(stageNumber, executionStage);
+//			}
+//
+//			executionStage.addStageMember(groupVertex);
+//			groupVertex.setExecutionStage(executionStage);
+//		}
+//	}
+//
+//	public void repairInstanceSharing() {
+//
+//		final Set<AllocatedResource> availableResources = new LinkedHashSet<AllocatedResource>();
+//
+//		final Iterator<ExecutionGroupVertex> it = new ExecutionGroupVertexIterator(this, true, -1);
+//		while (it.hasNext()) {
+//			final ExecutionGroupVertex groupVertex = it.next();
+//			if (groupVertex.getVertexToShareInstancesWith() == null) {
+//				availableResources.clear();
+//				groupVertex.repairInstanceSharing(availableResources);
+//			}
+//		}
+//	}
+//
+//	public void repairInstanceAssignment() {
+//
+//		Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
+//		while (it.hasNext()) {
+//
+//			final ExecutionVertex sourceVertex = it.next();
+//
+//			for (int i = 0; i < sourceVertex.getNumberOfOutputGates(); ++i) {
+//
+//				final ExecutionGate outputGate = sourceVertex.getOutputGate(i);
+//				final ChannelType channelType = outputGate.getChannelType();
+//				if (channelType == ChannelType.IN_MEMORY) {
+//					final int numberOfOutputChannels = outputGate.getNumberOfEdges();
+//					for (int j = 0; j < numberOfOutputChannels; ++j) {
+//						final ExecutionEdge outputChannel = outputGate.getEdge(j);
+//						outputChannel.getInputGate().getVertex()
+//							.setAllocatedResource(sourceVertex.getAllocatedResource());
+//					}
+//				}
+//			}
+//		}
+//
+//		it = new ExecutionGraphIterator(this, false);
+//		while (it.hasNext()) {
+//
+//			final ExecutionVertex targetVertex = it.next();
+//
+//			for (int i = 0; i < targetVertex.getNumberOfInputGates(); ++i) {
+//
+//				final ExecutionGate inputGate = targetVertex.getInputGate(i);
+//				final ChannelType channelType = inputGate.getChannelType();
+//				if (channelType == ChannelType.IN_MEMORY) {
+//					final int numberOfInputChannels = inputGate.getNumberOfEdges();
+//					for (int j = 0; j < numberOfInputChannels; ++j) {
+//						final ExecutionEdge inputChannel = inputGate.getEdge(j);
+//						inputChannel.getOutputGate().getVertex()
+//							.setAllocatedResource(targetVertex.getAllocatedResource());
+//					}
+//				}
+//			}
+//		}
+//	}
+//
+//	public ChannelType getChannelType(final ExecutionVertex sourceVertex, final ExecutionVertex targetVertex) {
+//
+//		final ExecutionGroupVertex sourceGroupVertex = sourceVertex.getGroupVertex();
+//		final ExecutionGroupVertex targetGroupVertex = targetVertex.getGroupVertex();
+//
+//		final List<ExecutionGroupEdge> edges = sourceGroupVertex.getForwardEdges(targetGroupVertex);
+//		if (edges.size() == 0) {
+//			return null;
+//		}
+//
+//		// On a task level, the two vertices are connected
+//		final ExecutionGroupEdge edge = edges.get(0);
+//
+//		// Now lets see if these two concrete subtasks are connected
+//		final ExecutionGate outputGate = sourceVertex.getOutputGate(edge.getIndexOfOutputGate());
+//		for (int i = 0; i < outputGate.getNumberOfEdges(); ++i) {
+//
+//			final ExecutionEdge outputChannel = outputGate.getEdge(i);
+//			if (targetVertex == outputChannel.getInputGate().getVertex()) {
+//				return edge.getChannelType();
+//			}
+//		}
+//
+//		return null;
+//	}
+//
+//	/**
+//	 * Returns the job configuration that was originally attached to the job graph.
+//	 * 
+//	 * @return the job configuration that was originally attached to the job graph
+//	 */
+//	public Configuration getJobConfiguration() {
+//		return this.jobConfiguration;
+//	}
+//
+//	/**
+//	 * Checks whether the job represented by the execution graph has the status <code>FINISHED</code>.
+//	 * 
+//	 * @return <code>true</code> if the job has the status <code>CREATED</code>, <code>false</code> otherwise
+//	 */
+//	private boolean jobHasFinishedStatus() {
+//
+//		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
+//
+//		while (i

<TRUNCATED>

[39/63] [abbrv] Finalize ExecutionGraph state machine and calls

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 4c0c730..72525e9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -18,10 +18,9 @@
 
 package org.apache.flink.runtime.executiongraph;
 
-import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.ConcurrentModificationException;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -38,11 +37,8 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.JobException;
 import org.apache.flink.runtime.execution.ExecutionListener;
 import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.execution.ExecutionState2;
-import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
-import org.apache.flink.runtime.io.network.RemoteReceiver;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
@@ -51,8 +47,7 @@ import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-
-import com.google.common.base.Preconditions;
+import org.apache.flink.util.ExceptionUtils;
 
 
 public class ExecutionGraph {
@@ -77,11 +72,19 @@ public class ExecutionGraph {
 	/** All job vertices that are part of this graph */
 	private final ConcurrentHashMap<JobVertexID, ExecutionJobVertex> tasks;
 	
+	/** All vertices, in the order in which they were created **/
 	private final List<ExecutionJobVertex> verticesInCreationOrder;
 	
 	/** All intermediate results that are part of this graph */
 	private final ConcurrentHashMap<IntermediateDataSetID, IntermediateResult> intermediateResults;
 	
+	/** The currently executed tasks, for callbacks */
+	private final ConcurrentHashMap<ExecutionAttemptID, Execution> currentExecutions;
+	
+	
+	private final Map<ChannelID, ExecutionEdge> edges = new HashMap<ChannelID, ExecutionEdge>();
+	
+	
 	/** An executor that can run long actions (involving remote calls) */
 	private final ExecutorService executor;
 	
@@ -92,16 +95,17 @@ public class ExecutionGraph {
 	
 	private final List<ExecutionListener> executionListeners;
 	
+	private final long[] stateTimestamps;
 	
-	private DefaultScheduler jobScheduler;
 	
-	private boolean allowQueuedScheduling = false;
-
+	private final Object progressLock = new Object();
+	
+	private int nextVertexToFinish;
 	
 	private volatile JobStatus state = JobStatus.CREATED;
 	
-	private final long[] stateTimestamps;
 	
+	private boolean allowQueuedScheduling = false;
 	
 	
 	public ExecutionGraph(JobID jobId, String jobName, Configuration jobConfig) {
@@ -121,6 +125,7 @@ public class ExecutionGraph {
 		this.tasks = new ConcurrentHashMap<JobVertexID, ExecutionJobVertex>();
 		this.intermediateResults = new ConcurrentHashMap<IntermediateDataSetID, IntermediateResult>();
 		this.verticesInCreationOrder = new ArrayList<ExecutionJobVertex>();
+		this.currentExecutions = new ConcurrentHashMap<ExecutionAttemptID, Execution>();
 		
 		this.userCodeJarFiles = new ArrayList<String>();
 		this.jobStatusListeners = new CopyOnWriteArrayList<JobStatusListener>();
@@ -137,10 +142,12 @@ public class ExecutionGraph {
 					+ "vertices and %d intermediate results.", topologiallySorted.size(), tasks.size(), intermediateResults.size()));
 		}
 		
+		final long createTimestamp = System.currentTimeMillis();
+		
 		for (AbstractJobVertex jobVertex : topologiallySorted) {
 			
 			// create the execution job vertex and attach it to the graph
-			ExecutionJobVertex ejv = new ExecutionJobVertex(this, jobVertex, 1);
+			ExecutionJobVertex ejv = new ExecutionJobVertex(this, jobVertex, 1, createTimestamp);
 			ejv.connectToPredecessors(this.intermediateResults);
 			
 			ExecutionJobVertex previousTask = this.tasks.putIfAbsent(jobVertex.getID(), ejv);
@@ -233,10 +240,10 @@ public class ExecutionGraph {
 		return Collections.unmodifiableMap(this.intermediateResults);
 	}
 	
-	public Iterable<ExecutionVertex2> getAllExecutionVertices() {
-		return new Iterable<ExecutionVertex2>() {
+	public Iterable<ExecutionVertex> getAllExecutionVertices() {
+		return new Iterable<ExecutionVertex>() {
 			@Override
-			public Iterator<ExecutionVertex2> iterator() {
+			public Iterator<ExecutionVertex> iterator() {
 				return new AllVerticesIterator(getVerticesTopologically().iterator());
 			}
 		};
@@ -255,15 +262,7 @@ public class ExecutionGraph {
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	
-	public void registerJobStatusListener(JobStatusListener jobStatusListener) {
-		this.jobStatusListeners.add(jobStatusListener);
-	}
-	
-	public void registerExecutionListener(ExecutionListener executionListener) {
-		this.executionListeners.add(executionListener);
-	}
-	
+	//  Actions
 	// --------------------------------------------------------------------------------------------
 	
 	public void scheduleForExecution(DefaultScheduler scheduler) throws JobException {
@@ -271,19 +270,14 @@ public class ExecutionGraph {
 			throw new IllegalArgumentException("Scheduler must not be null.");
 		}
 		
-		if (STATE_UPDATER.compareAndSet(this, JobStatus.CREATED, JobStatus.RUNNING)) {
-			this.jobScheduler = scheduler;
-			
-			notifyJobStatusChange(JobStatus.RUNNING, null);
+		if (transitionState(JobStatus.CREATED, JobStatus.RUNNING)) {
 			
 			// initially, we simply take the ones without inputs.
 			// next, we implement the logic to go back from vertices that need computation
 			// to the ones we need to start running
 			for (ExecutionJobVertex ejv : this.tasks.values()) {
 				if (ejv.getJobVertex().isInputVertex()) {
-					for (ExecutionVertex2 ev : ejv.getTaskVertices()) {
-						ev.scheduleForExecution(scheduler);
-					}
+					ejv.scheduleAll(scheduler, allowQueuedScheduling);
 				}
 			}
 		}
@@ -293,11 +287,133 @@ public class ExecutionGraph {
 	}
 	
 	public void cancel() {
-		//TODO
+		while (true) {
+			JobStatus current = state;
+			if (current == JobStatus.RUNNING || current == JobStatus.CREATED) {
+				if (transitionState(current, JobStatus.CANCELLING)) {
+					for (ExecutionJobVertex ejv : verticesInCreationOrder) {
+						ejv.cancel();
+					}
+				}
+			}
+			
+			// no need to treat other states
+		}
 	}
 	
-	public void updateState(TaskExecutionState state) {
-		//TODO		
+	public void fail(Throwable t) {
+		if (LOG.isErrorEnabled()) {
+			LOG.error(String.format("Failing ExecutionGraph %s (%s): ", getJobID(), getJobName()), t);
+		}
+		
+		while (true) {
+			JobStatus current = state;
+			if (current != JobStatus.FAILED) {
+				if (transitionState(current, JobStatus.FAILED, t)) {
+					// cancel all. what is failed will not cancel but stay failed
+					for (ExecutionJobVertex ejv : verticesInCreationOrder) {
+						ejv.cancel();
+					}
+				}
+			}
+			
+			// no need to treat other states
+		}
+	}
+	
+	public void waitForJobEnd() throws InterruptedException {
+		synchronized (progressLock) {
+			while (nextVertexToFinish < verticesInCreationOrder.size()) {
+				progressLock.wait();
+			}
+		}
+	}
+	
+	
+	private boolean transitionState(JobStatus current, JobStatus newState) {
+		return transitionState(current, newState, null);
+	}
+	
+	private boolean transitionState(JobStatus current, JobStatus newState, Throwable error) {
+		if (STATE_UPDATER.compareAndSet(this, current, newState)) {
+			stateTimestamps[newState.ordinal()] = System.currentTimeMillis();
+			notifyJobStatusChange(newState, error);
+			return true;
+		}
+		else {
+			return false;
+		}
+	}
+	
+	void jobVertexInFinalState(ExecutionJobVertex ev) {
+		synchronized (progressLock) {
+			int nextPos = nextVertexToFinish;
+			if (nextPos >= verticesInCreationOrder.size()) {
+				throw new RuntimeException("Inconsistency in job progress detection.");
+			}
+			
+			// see if we are the next to finish and then progress until the next unfinished one
+			if (verticesInCreationOrder.get(nextPos) == ev) {
+				do {
+					nextPos++;
+				}
+				while (nextPos < verticesInCreationOrder.size() && verticesInCreationOrder.get(nextPos).isInFinalState());
+				
+				nextVertexToFinish = nextPos;
+				
+				if (nextPos == verticesInCreationOrder.size()) {
+					// we are done, transition to the final state
+					
+					while (true) {
+						JobStatus current = this.state;
+						if (current == JobStatus.RUNNING && transitionState(current, JobStatus.FINISHED)) {
+							break;
+						}
+						if (current == JobStatus.CANCELLING && transitionState(current, JobStatus.CANCELED)) {
+							break;
+						}
+						if (current == JobStatus.FAILED) {
+							break;
+						}
+						if (current == JobStatus.CANCELED || current == JobStatus.CREATED || current == JobStatus.FINISHED) {
+							fail(new Exception("ExecutionGraph went into final state from state " + current));
+						}
+					}
+					
+					// also, notify waiters
+					progressLock.notifyAll();
+				}
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Callbacks and Callback Utilities
+	// --------------------------------------------------------------------------------------------
+	
+	public boolean updateState(TaskExecutionState state) {
+		Execution attempt = this.currentExecutions.get(state.getID());
+		if (attempt != null) {
+			switch (state.getExecutionState()) {
+				case FINISHED:
+					attempt.markFinished();
+					return true;
+				case CANCELED:
+					attempt.cancelingComplete();
+					return true;
+				case FAILED:
+					attempt.markFailed(state.getError());
+					return true;
+				default:
+					// we mark as failed and return false, which triggers the TaskManager
+					// to remove the task
+					attempt.fail(new Exception("TaskManager sent illegal state update: " + state.getExecutionState()));
+					return false;
+			}
+		}
+		else {
+			return false;
+		}
 	}
 	
 	public ConnectionInfoLookupResponse lookupConnectionInfoAndDeployReceivers(InstanceConnectionInfo caller, ChannelID sourceChannelID) {
@@ -396,22 +512,61 @@ public class ExecutionGraph {
 //		}
 	}
 	
+	void registerExecution(Execution exec) {
+		Execution previous = currentExecutions.putIfAbsent(exec.getAttemptId(), exec);
+		if (previous != null) {
+			fail(new Exception("Trying to register execution " + exec + " for already used ID " + exec.getAttemptId()));
+		}
+	}
+	
+	void deregisterExecution(Execution exec) {
+		Execution contained = currentExecutions.remove(exec.getAttemptId());
+		if (contained != null && contained != exec) {
+			fail(new Exception("De-registering execution " + exec + " failed. Found for same ID execution " + contained));
+		}
+	}
+	
+	Map<ExecutionAttemptID, Execution> getRegisteredExecutions() {
+		return Collections.unmodifiableMap(currentExecutions);
+	}
+	
+	void registerExecutionEdge(ExecutionEdge edge) {
+		ChannelID source = edge.getInputChannelId();
+		ChannelID target = edge.getInputChannelId();
+		edges.put(source, edge);
+		edges.put(target, edge);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Listeners & Observers
 	// --------------------------------------------------------------------------------------------
 	
+	public void registerJobStatusListener(JobStatusListener jobStatusListener) {
+		this.jobStatusListeners.add(jobStatusListener);
+	}
+	
+	public void registerExecutionListener(ExecutionListener executionListener) {
+		this.executionListeners.add(executionListener);
+	}
 	
 	/**
 	 * NOTE: This method never throws an error, only logs errors caused by the notified listeners.
 	 * 
 	 * @param newState
-	 * @param message
+	 * @param error
 	 */
-	private void notifyJobStatusChange(JobStatus newState, String message) {
-		for (JobStatusListener listener : this.jobStatusListeners) {
-			try {
-				listener.jobStatusHasChanged(this, newState, message);
-			}
-			catch (Throwable t) {
-				LOG.error("Notification of job status change caused an error.", t);
+	private void notifyJobStatusChange(JobStatus newState, Throwable error) {
+		if (jobStatusListeners.size() > 0) {
+			
+			String message = error == null ? null : ExceptionUtils.stringifyException(error);
+		
+			for (JobStatusListener listener : this.jobStatusListeners) {
+				try {
+					listener.jobStatusHasChanged(this, newState, message);
+				}
+				catch (Throwable t) {
+					LOG.error("Notification of job status change caused an error.", t);
+				}
 			}
 		}
 	}
@@ -422,15 +577,25 @@ public class ExecutionGraph {
 	 * @param vertexId
 	 * @param subtask
 	 * @param newExecutionState
-	 * @param optionalMessage
+	 * @param error
 	 */
-	void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionState2 newExecutionState, String optionalMessage) {
-		for (ExecutionListener listener : this.executionListeners) {
-			try {
-				listener.executionStateChanged(jobID, vertexId, subtask, newExecutionState, optionalMessage);
+	void notifyExecutionChange(JobVertexID vertexId, int subtask, ExecutionAttemptID executionId, ExecutionState newExecutionState, Throwable error) {
+		
+		if (this.executionListeners.size() > 0) {
+			
+			String message = error == null ? null : ExceptionUtils.stringifyException(error);
+			for (ExecutionListener listener : this.executionListeners) {
+				try {
+					listener.executionStateChanged(jobID, vertexId, subtask, executionId, newExecutionState, message);
+				}
+				catch (Throwable t) {
+					LOG.error("Notification of execution state change caused an error.", t);
+				}
 			}
-			catch (Throwable t) {
-				LOG.error("Notification of execution state change caused an error.", t);
+			
+			// see what this means for us. currently, the first FAILED state means -> FAILED
+			if (newExecutionState == ExecutionState.FAILED) {
+				fail(error);
 			}
 		}
 	}
@@ -439,1127 +604,11 @@ public class ExecutionGraph {
 	//  Miscellaneous
 	// --------------------------------------------------------------------------------------------
 	
-	
 	public void execute(Runnable action) {
-		if (this.executor == null) {
-			throw new IllegalStateException("Executor has not been set.");
+		if (this.executor != null) {
+			this.executor.submit(action);
+		} else {
+			action.run();
 		}
-		
-		this.executor.submit(action);
 	}
 }
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-	
-//	
-//	/**
-//	 * Applies the user defined settings to the execution graph.
-//	 * 
-//	 * @param temporaryGroupVertexMap
-//	 *        mapping between job vertices and the corresponding group vertices.
-//	 * @throws GraphConversionException
-//	 *         thrown if an error occurs while applying the user settings.
-//	 */
-//	private void applyUserDefinedSettings(final HashMap<AbstractJobVertex, ExecutionGroupVertex> temporaryGroupVertexMap)
-//			throws GraphConversionException {
-//
-//		// The check for cycles in the dependency chain for instance sharing is already checked in
-//		// <code>submitJob</code> method of the job manager
-//
-//		// If there is no cycle, apply the settings to the corresponding group vertices
-//		final Iterator<Map.Entry<AbstractJobVertex, ExecutionGroupVertex>> it = temporaryGroupVertexMap.entrySet()
-//			.iterator();
-//		while (it.hasNext()) {
-//
-//			final Map.Entry<AbstractJobVertex, ExecutionGroupVertex> entry = it.next();
-//			final AbstractJobVertex jobVertex = entry.getKey();
-//			if (jobVertex.getVertexToShareInstancesWith() != null) {
-//
-//				final AbstractJobVertex vertexToShareInstancesWith = jobVertex.getVertexToShareInstancesWith();
-//				final ExecutionGroupVertex groupVertex = entry.getValue();
-//				final ExecutionGroupVertex groupVertexToShareInstancesWith = temporaryGroupVertexMap
-//					.get(vertexToShareInstancesWith);
-//				groupVertex.shareInstancesWith(groupVertexToShareInstancesWith);
-//			}
-//		}
-//
-//		// Second, we create the number of execution vertices each group vertex is supposed to manage
-//		Iterator<ExecutionGroupVertex> it2 = new ExecutionGroupVertexIterator(this, true, -1);
-//		while (it2.hasNext()) {
-//
-//			final ExecutionGroupVertex groupVertex = it2.next();
-//			if (groupVertex.isNumberOfMembersUserDefined()) {
-//				groupVertex.createInitialExecutionVertices(groupVertex.getUserDefinedNumberOfMembers());
-//			}
-//		}
-//
-//		// Finally, apply the channel settings channel settings
-//		it2 = new ExecutionGroupVertexIterator(this, true, -1);
-//		while (it2.hasNext()) {
-//
-//			final ExecutionGroupVertex groupVertex = it2.next();
-//			for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) {
-//
-//				final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i);
-//				if (edge.isChannelTypeUserDefined()) {
-//					edge.changeChannelType(edge.getChannelType());
-//				}
-//
-//				// Create edges between execution vertices
-//				createExecutionEdgesForGroupEdge(edge);
-//			}
-//		}
-//
-//		// Repair the instance assignment after having changed the channel types
-//		repairInstanceAssignment();
-//
-//		// Repair the instance sharing among different group vertices
-//		repairInstanceSharing();
-//
-//		// Finally, repair the stages
-//		repairStages();
-//	}
-//
-//	/**
-//	 * Sets up an execution graph from a job graph.
-//	 * 
-//	 * @param jobGraph
-//	 *        the job graph to create the execution graph from
-//	 * @param defaultParallelism
-//	 *        defaultParallelism in case that nodes have no parallelism set
-//	 * @throws GraphConversionException
-//	 *         thrown if the job graph is not valid and no execution graph can be constructed from it
-//	 */
-//	private void constructExecutionGraph(final JobGraph jobGraph, final int defaultParallelism)
-//			throws GraphConversionException {
-//
-//		// Clean up temporary data structures
-//		final HashMap<AbstractJobVertex, ExecutionVertex> temporaryVertexMap = new HashMap<AbstractJobVertex, ExecutionVertex>();
-//		final HashMap<AbstractJobVertex, ExecutionGroupVertex> temporaryGroupVertexMap = new HashMap<AbstractJobVertex, ExecutionGroupVertex>();
-//
-//		// Initially, create only one execution stage that contains all group vertices
-//		final ExecutionStage initialExecutionStage = new ExecutionStage(this, 0);
-//		this.stages.add(initialExecutionStage);
-//
-//		// Convert job vertices to execution vertices and initialize them
-//		final AbstractJobVertex[] all = jobGraph.getAllJobVertices();
-//		for (int i = 0; i < all.length; i++) {
-//			if(all[i].getNumberOfSubtasks() == -1){
-//				all[i].setNumberOfSubtasks(defaultParallelism);
-//			}
-//
-//			final ExecutionVertex createdVertex = createVertex(all[i], initialExecutionStage);
-//			temporaryVertexMap.put(all[i], createdVertex);
-//			temporaryGroupVertexMap.put(all[i], createdVertex.getGroupVertex());
-//		}
-//
-//		// Create initial edges between the vertices
-//		createInitialGroupEdges(temporaryVertexMap);
-//
-//		// Now that an initial graph is built, apply the user settings
-//		applyUserDefinedSettings(temporaryGroupVertexMap);
-//
-//		// Calculate the connection IDs
-//		calculateConnectionIDs();
-//
-//		// Finally, construct the execution pipelines
-//		reconstructExecutionPipelines();
-//	}
-//
-//	private void createExecutionEdgesForGroupEdge(final ExecutionGroupEdge groupEdge) {
-//
-//		final ExecutionGroupVertex source = groupEdge.getSourceVertex();
-//		final int indexOfOutputGate = groupEdge.getIndexOfOutputGate();
-//		final ExecutionGroupVertex target = groupEdge.getTargetVertex();
-//		final int indexOfInputGate = groupEdge.getIndexOfInputGate();
-//
-//		final Map<GateID, List<ExecutionEdge>> inputChannelMap = new HashMap<GateID, List<ExecutionEdge>>();
-//
-//		// Unwire the respective gate of the source vertices
-//		final int currentNumberOfSourceNodes = source.getCurrentNumberOfGroupMembers();
-//		for (int i = 0; i < currentNumberOfSourceNodes; ++i) {
-//
-//			final ExecutionVertex sourceVertex = source.getGroupMember(i);
-//			final ExecutionGate outputGate = sourceVertex.getOutputGate(indexOfOutputGate);
-//			if (outputGate == null) {
-//				throw new IllegalStateException("wire: " + sourceVertex.getName()
-//					+ " has no output gate with index " + indexOfOutputGate);
-//			}
-//
-//			if (outputGate.getNumberOfEdges() > 0) {
-//				throw new IllegalStateException("wire: wire called on source " + sourceVertex.getName() + " (" + i
-//					+ "), but number of output channels is " + outputGate.getNumberOfEdges() + "!");
-//			}
-//
-//			final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers();
-//			final List<ExecutionEdge> outputChannels = new ArrayList<ExecutionEdge>();
-//
-//			for (int j = 0; j < currentNumberOfTargetNodes; ++j) {
-//
-//				final ExecutionVertex targetVertex = target.getGroupMember(j);
-//				final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate);
-//				if (inputGate == null) {
-//					throw new IllegalStateException("wire: " + targetVertex.getName()
-//						+ " has no input gate with index " + indexOfInputGate);
-//				}
-//
-//				if (inputGate.getNumberOfEdges() > 0 && i == 0) {
-//					throw new IllegalStateException("wire: wire called on target " + targetVertex.getName() + " ("
-//						+ j + "), but number of input channels is " + inputGate.getNumberOfEdges() + "!");
-//				}
-//
-//				// Check if a wire is supposed to be created
-//				if (DistributionPatternProvider.createWire(groupEdge.getDistributionPattern(),
-//					i, j, currentNumberOfSourceNodes, currentNumberOfTargetNodes)) {
-//
-//					final ChannelID outputChannelID = new ChannelID();
-//					final ChannelID inputChannelID = new ChannelID();
-//
-//					final ExecutionEdge edge = new ExecutionEdge(outputGate, inputGate, groupEdge, outputChannelID,
-//						inputChannelID, outputGate.getNumberOfEdges(), inputGate.getNumberOfEdges());
-//
-//					this.edgeMap.put(outputChannelID, edge);
-//					this.edgeMap.put(inputChannelID, edge);
-//
-//					outputChannels.add(edge);
-//
-//					List<ExecutionEdge> inputChannels = inputChannelMap.get(inputGate.getGateID());
-//					if (inputChannels == null) {
-//						inputChannels = new ArrayList<ExecutionEdge>();
-//						inputChannelMap.put(inputGate.getGateID(), inputChannels);
-//					}
-//
-//					inputChannels.add(edge);
-//				}
-//			}
-//
-//			outputGate.replaceAllEdges(outputChannels);
-//		}
-//
-//		// Finally, set the channels for the input gates
-//		final int currentNumberOfTargetNodes = target.getCurrentNumberOfGroupMembers();
-//		for (int i = 0; i < currentNumberOfTargetNodes; ++i) {
-//
-//			final ExecutionVertex targetVertex = target.getGroupMember(i);
-//			final ExecutionGate inputGate = targetVertex.getInputGate(indexOfInputGate);
-//
-//			final List<ExecutionEdge> inputChannels = inputChannelMap.get(inputGate.getGateID());
-//			if (inputChannels == null) {
-//				LOG.error("Cannot find input channels for gate ID " + inputGate.getGateID());
-//				continue;
-//			}
-//
-//			inputGate.replaceAllEdges(inputChannels);
-//		}
-//
-//	}
-//
-//	/**
-//	 * Creates the initial edges between the group vertices
-//	 * 
-//	 * @param vertexMap
-//	 *        the temporary vertex map
-//	 * @throws GraphConversionException
-//	 *         if the initial wiring cannot be created
-//	 */
-//	private void createInitialGroupEdges(final HashMap<AbstractJobVertex, ExecutionVertex> vertexMap)
-//			throws GraphConversionException {
-//
-//		Iterator<Map.Entry<AbstractJobVertex, ExecutionVertex>> it = vertexMap.entrySet().iterator();
-//
-//		while (it.hasNext()) {
-//
-//			final Map.Entry<AbstractJobVertex, ExecutionVertex> entry = it.next();
-//			final AbstractJobVertex sjv = entry.getKey();
-//			final ExecutionVertex sev = entry.getValue();
-//			final ExecutionGroupVertex sgv = sev.getGroupVertex();
-//
-//			// First, build the group edges
-//			for (int i = 0; i < sjv.getNumberOfForwardConnections(); ++i) {
-//				final JobEdge edge = sjv.getForwardConnection(i);
-//				final AbstractJobVertex tjv = edge.getConnectedVertex();
-//
-//				final ExecutionVertex tev = vertexMap.get(tjv);
-//				final ExecutionGroupVertex tgv = tev.getGroupVertex();
-//				// Use NETWORK as default channel type if nothing else is defined by the user
-//				ChannelType channelType = edge.getChannelType();
-//				boolean userDefinedChannelType = true;
-//				if (channelType == null) {
-//					userDefinedChannelType = false;
-//					channelType = ChannelType.NETWORK;
-//				}
-//
-//				final DistributionPattern distributionPattern = edge.getDistributionPattern();
-//
-//				// Connect the corresponding group vertices and copy the user settings from the job edge
-//				final ExecutionGroupEdge groupEdge = sgv.wireTo(tgv, edge.getIndexOfInputGate(), i, channelType,
-//					userDefinedChannelType,distributionPattern);
-//
-//				final ExecutionGate outputGate = new ExecutionGate(new GateID(), sev, groupEdge, false);
-//				sev.insertOutputGate(i, outputGate);
-//				final ExecutionGate inputGate = new ExecutionGate(new GateID(), tev, groupEdge, true);
-//				tev.insertInputGate(edge.getIndexOfInputGate(), inputGate);
-//			}
-//		}
-//	}
-//
-//	/**
-//	 * Creates an execution vertex from a job vertex.
-//	 * 
-//	 * @param jobVertex
-//	 *        the job vertex to create the execution vertex from
-//	 * @param initialExecutionStage
-//	 *        the initial execution stage all group vertices are added to
-//	 * @return the new execution vertex
-//	 * @throws GraphConversionException
-//	 *         thrown if the job vertex is of an unknown subclass
-//	 */
-//	private ExecutionVertex createVertex(final AbstractJobVertex jobVertex, final ExecutionStage initialExecutionStage)
-//			throws GraphConversionException {
-//
-//		// Create an initial execution vertex for the job vertex
-//		final Class<? extends AbstractInvokable> invokableClass = jobVertex.getInvokableClass();
-//		if (invokableClass == null) {
-//			throw new GraphConversionException("JobVertex " + jobVertex.getID() + " (" + jobVertex.getName()
-//				+ ") does not specify a task");
-//		}
-//
-//		// Calculate the cryptographic signature of this vertex
-//		final ExecutionSignature signature = ExecutionSignature.createSignature(jobVertex.getInvokableClass(),
-//			jobVertex.getJobGraph().getJobID());
-//
-//		// Create a group vertex for the job vertex
-//
-//		ExecutionGroupVertex groupVertex = null;
-//		try {
-//			groupVertex = new ExecutionGroupVertex(jobVertex.getName(), jobVertex.getID(), this,
-//				jobVertex.getNumberOfSubtasks(), jobVertex.getVertexToShareInstancesWith() != null ? true
-//					: false, 0, jobVertex.getConfiguration(), signature,
-//				invokableClass);
-//		} catch (Throwable t) {
-//			throw new GraphConversionException(t);
-//		}
-//
-//		// Register input and output vertices separately
-//		if (jobVertex instanceof AbstractJobInputVertex) {
-//
-//			final AbstractJobInputVertex jobInputVertex = (AbstractJobInputVertex) jobVertex;
-//			
-//			if (jobVertex instanceof InputFormatVertex) {
-//				try {
-//					// get a handle to the user code class loader
-//					ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID());
-//					
-//					((InputFormatVertex) jobVertex).initializeInputFormatFromTaskConfig(cl);
-//				}
-//				catch (Throwable t) {
-//					throw new GraphConversionException("Could not deserialize input format.", t);
-//				}
-//			}
-//			
-//			final Class<? extends InputSplit> inputSplitType = jobInputVertex.getInputSplitType();
-//			
-//			InputSplit[] inputSplits;
-//
-//			try {
-//				inputSplits = jobInputVertex.getInputSplits(jobVertex.getNumberOfSubtasks());
-//			}
-//			catch (Throwable t) {
-//				throw new GraphConversionException("Cannot compute input splits for " + groupVertex.getName(), t);
-//			}
-//
-//			if (inputSplits == null) {
-//				inputSplits = new InputSplit[0];
-//			}
-//			
-//			LOG.info("Job input vertex " + jobVertex.getName() + " generated " + inputSplits.length + " input splits");
-//
-//			// assign input splits and type
-//			groupVertex.setInputSplits(inputSplits);
-//			groupVertex.setInputSplitType(inputSplitType);
-//		}
-//
-//		if (jobVertex instanceof OutputFormatVertex){
-//			final OutputFormatVertex jobOutputVertex = (OutputFormatVertex) jobVertex;
-//			
-//			try {
-//				// get a handle to the user code class loader
-//				ClassLoader cl = LibraryCacheManager.getClassLoader(jobVertex.getJobGraph().getJobID());
-//				jobOutputVertex.initializeOutputFormatFromTaskConfig(cl);
-//			}
-//			catch (Throwable t) {
-//				throw new GraphConversionException("Could not deserialize output format.", t);
-//			}
-//
-//			OutputFormat<?> outputFormat = jobOutputVertex.getOutputFormat();
-//			if (outputFormat != null && outputFormat instanceof InitializeOnMaster){
-//				try {
-//					((InitializeOnMaster) outputFormat).initializeGlobal(jobVertex.getNumberOfSubtasks());
-//				}
-//				catch (Throwable t) {
-//					throw new GraphConversionException(t);
-//				}
-//			}
-//		}
-//
-//		// Add group vertex to initial execution stage
-//		initialExecutionStage.addStageMember(groupVertex);
-//
-//		final ExecutionVertex ev = new ExecutionVertex(this, groupVertex, jobVertex.getNumberOfForwardConnections(),
-//			jobVertex.getNumberOfBackwardConnections());
-//
-//		// Assign initial instance to vertex (may be overwritten later on when user settings are applied)
-//		ev.setAllocatedResource(new AllocatedResource(DummyInstance.createDummyInstance(), null));
-//
-//		return ev;
-//	}
-//
-//	/**
-//	 * Returns the number of input vertices registered with this execution graph.
-//	 * 
-//	 * @return the number of input vertices registered with this execution graph
-//	 */
-//	public int getNumberOfInputVertices() {
-//		return this.stages.get(0).getNumberOfInputExecutionVertices();
-//	}
-//
-//	/**
-//	 * Returns the number of input vertices for the given stage.
-//	 * 
-//	 * @param stage
-//	 *        the index of the execution stage
-//	 * @return the number of input vertices for the given stage
-//	 */
-//	public int getNumberOfInputVertices(int stage) {
-//		if (stage >= this.stages.size()) {
-//			return 0;
-//		}
-//
-//		return this.stages.get(stage).getNumberOfInputExecutionVertices();
-//	}
-//
-//	/**
-//	 * Returns the number of output vertices registered with this execution graph.
-//	 * 
-//	 * @return the number of output vertices registered with this execution graph
-//	 */
-//	public int getNumberOfOutputVertices() {
-//		return this.stages.get(0).getNumberOfOutputExecutionVertices();
-//	}
-//
-//	/**
-//	 * Returns the number of output vertices for the given stage.
-//	 * 
-//	 * @param stage
-//	 *        the index of the execution stage
-//	 * @return the number of input vertices for the given stage
-//	 */
-//	public int getNumberOfOutputVertices(final int stage) {
-//		if (stage >= this.stages.size()) {
-//			return 0;
-//		}
-//
-//		return this.stages.get(stage).getNumberOfOutputExecutionVertices();
-//	}
-//
-//	/**
-//	 * Returns the input vertex with the specified index.
-//	 * 
-//	 * @param index
-//	 *        the index of the input vertex to return
-//	 * @return the input vertex with the specified index or <code>null</code> if no input vertex with such an index
-//	 *         exists
-//	 */
-//	public ExecutionVertex getInputVertex(final int index) {
-//		return this.stages.get(0).getInputExecutionVertex(index);
-//	}
-//
-//	/**
-//	 * Returns the output vertex with the specified index.
-//	 * 
-//	 * @param index
-//	 *        the index of the output vertex to return
-//	 * @return the output vertex with the specified index or <code>null</code> if no output vertex with such an index
-//	 *         exists
-//	 */
-//	public ExecutionVertex getOutputVertex(final int index) {
-//		return this.stages.get(0).getOutputExecutionVertex(index);
-//	}
-//
-//	/**
-//	 * Returns the input vertex with the specified index for the given stage
-//	 * 
-//	 * @param stage
-//	 *        the index of the stage
-//	 * @param index
-//	 *        the index of the input vertex to return
-//	 * @return the input vertex with the specified index or <code>null</code> if no input vertex with such an index
-//	 *         exists in that stage
-//	 */
-//	public ExecutionVertex getInputVertex(final int stage, final int index) {
-//		try {
-//			final ExecutionStage s = this.stages.get(stage);
-//			if (s == null) {
-//				return null;
-//			}
-//
-//			return s.getInputExecutionVertex(index);
-//
-//		} catch (ArrayIndexOutOfBoundsException e) {
-//			return null;
-//		}
-//	}
-//
-//	/**
-//	 * Returns the output vertex with the specified index for the given stage.
-//	 * 
-//	 * @param stage
-//	 *        the index of the stage
-//	 * @param index
-//	 *        the index of the output vertex to return
-//	 * @return the output vertex with the specified index or <code>null</code> if no output vertex with such an index
-//	 *         exists in that stage
-//	 */
-//	public ExecutionVertex getOutputVertex(final int stage, final int index) {
-//		try {
-//			final ExecutionStage s = this.stages.get(stage);
-//			if (s == null) {
-//				return null;
-//			}
-//
-//			return s.getOutputExecutionVertex(index);
-//
-//		} catch (ArrayIndexOutOfBoundsException e) {
-//			return null;
-//		}
-//	}
-//
-//	/**
-//	 * Identifies an execution by the specified channel ID and returns it.
-//	 * 
-//	 * @param id
-//	 *        the channel ID to identify the vertex with
-//	 * @return the execution vertex which has a channel with ID <code>id</code> or <code>null</code> if no such vertex
-//	 *         exists in the execution graph
-//	 */
-//	public ExecutionVertex getVertexByChannelID(final ChannelID id) {
-//		final ExecutionEdge edge = this.edgeMap.get(id);
-//		if (edge == null) {
-//			return null;
-//		}
-//
-//		if (id.equals(edge.getOutputChannelID())) {
-//			return edge.getOutputGate().getVertex();
-//		}
-//
-//		return edge.getInputGate().getVertex();
-//	}
-//
-//	/**
-//	 * Finds an {@link ExecutionEdge} by its ID and returns it.
-//	 * 
-//	 * @param id
-//	 *        the channel ID to identify the edge
-//	 * @return the edge whose ID matches <code>id</code> or <code>null</code> if no such edge is known
-//	 */
-//	public ExecutionEdge getEdgeByID(final ChannelID id) {
-//		return this.edgeMap.get(id);
-//	}
-//
-//	/**
-//	 * Registers an execution vertex with the execution graph.
-//	 * 
-//	 * @param vertex
-//	 *        the execution vertex to register
-//	 */
-//	void registerExecutionVertex(final ExecutionVertex vertex) {
-//		if (this.vertexMap.put(vertex.getID(), vertex) != null) {
-//			throw new IllegalStateException("There is already an execution vertex with ID " + vertex.getID()
-//				+ " registered");
-//		}
-//	}
-//
-//	/**
-//	 * Returns the execution vertex with the given vertex ID.
-//	 * 
-//	 * @param id
-//	 *        the vertex ID to retrieve the execution vertex
-//	 * @return the execution vertex matching the provided vertex ID or <code>null</code> if no such vertex could be
-//	 *         found
-//	 */
-//	public ExecutionVertex getVertexByID(final ExecutionVertexID id) {
-//		return this.vertexMap.get(id);
-//	}
-//
-//	/**
-//	 * Checks if the current execution stage has been successfully completed, i.e.
-//	 * all vertices in this stage have successfully finished their execution.
-//	 * 
-//	 * @return <code>true</code> if stage is completed, <code>false</code> otherwise
-//	 */
-//	private boolean isCurrentStageCompleted() {
-//		if (this.indexToCurrentExecutionStage >= this.stages.size()) {
-//			return true;
-//		}
-//
-//		final ExecutionGraphIterator it = new ExecutionGraphIterator(this, this.indexToCurrentExecutionStage, true,
-//			true);
-//		while (it.hasNext()) {
-//			final ExecutionVertex vertex = it.next();
-//			if (vertex.getExecutionState() != ExecutionState.FINISHED) {
-//				return false;
-//			}
-//		}
-//
-//		return true;
-//	}
-//
-//	/**
-//	 * Checks if the execution of execution graph is finished.
-//	 * 
-//	 * @return <code>true</code> if the execution of the graph is finished, <code>false</code> otherwise
-//	 */
-//	public boolean isExecutionFinished() {
-//		return (getJobStatus() == InternalJobStatus.FINISHED);
-//	}
-//
-//	/**
-//	 * Returns the job ID of the job configuration this execution graph was originally constructed from.
-//	 * 
-//	 * @return the job ID of the job configuration this execution graph was originally constructed from
-//	 */
-//	public JobID getJobID() {
-//		return this.jobID;
-//	}
-//
-//	/**
-//	 * Returns the index of the current execution stage.
-//	 * 
-//	 * @return the index of the current execution stage
-//	 */
-//	public int getIndexOfCurrentExecutionStage() {
-//		return this.indexToCurrentExecutionStage;
-//	}
-//
-//	/**
-//	 * Returns the stage which is currently executed.
-//	 * 
-//	 * @return the currently executed stage or <code>null</code> if the job execution is already completed
-//	 */
-//	public ExecutionStage getCurrentExecutionStage() {
-//
-//		try {
-//			return this.stages.get(this.indexToCurrentExecutionStage);
-//		} catch (ArrayIndexOutOfBoundsException e) {
-//			return null;
-//		}
-//	}
-//
-//	public void repairStages() {
-//
-//		final Map<ExecutionGroupVertex, Integer> stageNumbers = new HashMap<ExecutionGroupVertex, Integer>();
-//		ExecutionGroupVertexIterator it = new ExecutionGroupVertexIterator(this, true, -1);
-//
-//		while (it.hasNext()) {
-//
-//			final ExecutionGroupVertex groupVertex = it.next();
-//			int precedingNumber = 0;
-//			if (stageNumbers.containsKey(groupVertex)) {
-//				precedingNumber = stageNumbers.get(groupVertex).intValue();
-//			} else {
-//				stageNumbers.put(groupVertex, Integer.valueOf(precedingNumber));
-//			}
-//
-//			for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) {
-//
-//				final ExecutionGroupEdge edge = groupVertex.getForwardEdge(i);
-//				if (!stageNumbers.containsKey(edge.getTargetVertex())) {
-//					// Target vertex has not yet been discovered
-//					// Same stage as preceding vertex
-//					stageNumbers.put(edge.getTargetVertex(), Integer.valueOf(precedingNumber));
-//				} else {
-//					final int stageNumber = stageNumbers.get(edge.getTargetVertex()).intValue();
-//					if (stageNumber != precedingNumber) {
-//						stageNumbers.put(edge.getTargetVertex(), (int) Math.max(precedingNumber, stageNumber));
-//					}
-//				}
-//			}
-//		}
-//
-//		// Traverse the graph backwards (starting from the output vertices) to make sure vertices are allocated in a
-//		// stage as high as possible
-//		it = new ExecutionGroupVertexIterator(this, false, -1);
-//
-//		while (it.hasNext()) {
-//
-//			final ExecutionGroupVertex groupVertex = it.next();
-//			final int succeedingNumber = stageNumbers.get(groupVertex);
-//
-//			for (int i = 0; i < groupVertex.getNumberOfBackwardLinks(); i++) {
-//
-//				final ExecutionGroupEdge edge = groupVertex.getBackwardEdge(i);
-//				final int stageNumber = stageNumbers.get(edge.getSourceVertex());
-//				if (stageNumber != succeedingNumber) {
-//					throw new IllegalStateException(edge.getSourceVertex() + " and " + edge.getTargetVertex()
-//						+ " are assigned to different stages");
-//				}
-//			}
-//		}
-//
-//		// Finally, assign the new stage numbers
-//		this.stages.clear();
-//		final Iterator<Map.Entry<ExecutionGroupVertex, Integer>> it2 = stageNumbers.entrySet().iterator();
-//		while (it2.hasNext()) {
-//
-//			final Map.Entry<ExecutionGroupVertex, Integer> entry = it2.next();
-//			final ExecutionGroupVertex groupVertex = entry.getKey();
-//			final int stageNumber = entry.getValue().intValue();
-//			// Prevent out of bounds exceptions
-//			while (this.stages.size() <= stageNumber) {
-//				this.stages.add(null);
-//			}
-//			ExecutionStage executionStage = this.stages.get(stageNumber);
-//			// If the stage not yet exists,
-//			if (executionStage == null) {
-//				executionStage = new ExecutionStage(this, stageNumber);
-//				this.stages.set(stageNumber, executionStage);
-//			}
-//
-//			executionStage.addStageMember(groupVertex);
-//			groupVertex.setExecutionStage(executionStage);
-//		}
-//	}
-//
-//	public void repairInstanceSharing() {
-//
-//		final Set<AllocatedResource> availableResources = new LinkedHashSet<AllocatedResource>();
-//
-//		final Iterator<ExecutionGroupVertex> it = new ExecutionGroupVertexIterator(this, true, -1);
-//		while (it.hasNext()) {
-//			final ExecutionGroupVertex groupVertex = it.next();
-//			if (groupVertex.getVertexToShareInstancesWith() == null) {
-//				availableResources.clear();
-//				groupVertex.repairInstanceSharing(availableResources);
-//			}
-//		}
-//	}
-//
-//	public void repairInstanceAssignment() {
-//
-//		Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-//		while (it.hasNext()) {
-//
-//			final ExecutionVertex sourceVertex = it.next();
-//
-//			for (int i = 0; i < sourceVertex.getNumberOfOutputGates(); ++i) {
-//
-//				final ExecutionGate outputGate = sourceVertex.getOutputGate(i);
-//				final ChannelType channelType = outputGate.getChannelType();
-//				if (channelType == ChannelType.IN_MEMORY) {
-//					final int numberOfOutputChannels = outputGate.getNumberOfEdges();
-//					for (int j = 0; j < numberOfOutputChannels; ++j) {
-//						final ExecutionEdge outputChannel = outputGate.getEdge(j);
-//						outputChannel.getInputGate().getVertex()
-//							.setAllocatedResource(sourceVertex.getAllocatedResource());
-//					}
-//				}
-//			}
-//		}
-//
-//		it = new ExecutionGraphIterator(this, false);
-//		while (it.hasNext()) {
-//
-//			final ExecutionVertex targetVertex = it.next();
-//
-//			for (int i = 0; i < targetVertex.getNumberOfInputGates(); ++i) {
-//
-//				final ExecutionGate inputGate = targetVertex.getInputGate(i);
-//				final ChannelType channelType = inputGate.getChannelType();
-//				if (channelType == ChannelType.IN_MEMORY) {
-//					final int numberOfInputChannels = inputGate.getNumberOfEdges();
-//					for (int j = 0; j < numberOfInputChannels; ++j) {
-//						final ExecutionEdge inputChannel = inputGate.getEdge(j);
-//						inputChannel.getOutputGate().getVertex()
-//							.setAllocatedResource(targetVertex.getAllocatedResource());
-//					}
-//				}
-//			}
-//		}
-//	}
-//
-//	public ChannelType getChannelType(final ExecutionVertex sourceVertex, final ExecutionVertex targetVertex) {
-//
-//		final ExecutionGroupVertex sourceGroupVertex = sourceVertex.getGroupVertex();
-//		final ExecutionGroupVertex targetGroupVertex = targetVertex.getGroupVertex();
-//
-//		final List<ExecutionGroupEdge> edges = sourceGroupVertex.getForwardEdges(targetGroupVertex);
-//		if (edges.size() == 0) {
-//			return null;
-//		}
-//
-//		// On a task level, the two vertices are connected
-//		final ExecutionGroupEdge edge = edges.get(0);
-//
-//		// Now lets see if these two concrete subtasks are connected
-//		final ExecutionGate outputGate = sourceVertex.getOutputGate(edge.getIndexOfOutputGate());
-//		for (int i = 0; i < outputGate.getNumberOfEdges(); ++i) {
-//
-//			final ExecutionEdge outputChannel = outputGate.getEdge(i);
-//			if (targetVertex == outputChannel.getInputGate().getVertex()) {
-//				return edge.getChannelType();
-//			}
-//		}
-//
-//		return null;
-//	}
-//
-//	/**
-//	 * Returns the job configuration that was originally attached to the job graph.
-//	 * 
-//	 * @return the job configuration that was originally attached to the job graph
-//	 */
-//	public Configuration getJobConfiguration() {
-//		return this.jobConfiguration;
-//	}
-//
-//	/**
-//	 * Checks whether the job represented by the execution graph has the status <code>FINISHED</code>.
-//	 * 
-//	 * @return <code>true</code> if the job has the status <code>CREATED</code>, <code>false</code> otherwise
-//	 */
-//	private boolean jobHasFinishedStatus() {
-//
-//		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-//
-//		while (it.hasNext()) {
-//
-//			if (it.next().getExecutionState() != ExecutionState.FINISHED) {
-//				return false;
-//			}
-//		}
-//
-//		return true;
-//	}
-//
-//	/**
-//	 * Checks whether the job represented by the execution graph has the status <code>SCHEDULED</code>.
-//	 * 
-//	 * @return <code>true</code> if the job has the status <code>SCHEDULED</code>, <code>false</code> otherwise
-//	 */
-//	private boolean jobHasScheduledStatus() {
-//
-//		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-//
-//		while (it.hasNext()) {
-//
-//			final ExecutionState s = it.next().getExecutionState();
-//			if (s != ExecutionState.CREATED && s != ExecutionState.SCHEDULED && s != ExecutionState.READY) {
-//				return false;
-//			}
-//		}
-//
-//		return true;
-//	}
-//
-//	/**
-//	 * Checks whether the job represented by the execution graph has the status <code>CANCELED</code> or
-//	 * <code>FAILED</code>.
-//	 * 
-//	 * @return <code>true</code> if the job has the status <code>CANCELED</code> or <code>FAILED</code>,
-//	 *         <code>false</code> otherwise
-//	 */
-//	private boolean jobHasFailedOrCanceledStatus() {
-//
-//		final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(this, true);
-//
-//		while (it.hasNext()) {
-//
-//			final ExecutionState state = it.next().getExecutionState();
-//
-//			if (state != ExecutionState.CANCELED && state != ExecutionState.FAILED && state != ExecutionState.FINISHED) {
-//				return false;
-//			}
-//		}
-//
-//		return true;
-//	}
-//
-//	private static InternalJobStatus determineNewJobStatus(final ExecutionGraph eg,
-//			final ExecutionState latestStateChange) {
-//
-//		final InternalJobStatus currentJobStatus = eg.getJobStatus();
-//
-//		switch (currentJobStatus) {
-//		case CREATED:
-//			if (eg.jobHasScheduledStatus()) {
-//				return InternalJobStatus.SCHEDULED;
-//			} else if (latestStateChange == ExecutionState.CANCELED) {
-//				if (eg.jobHasFailedOrCanceledStatus()) {
-//					return InternalJobStatus.CANCELED;
-//				}
-//			}else if(latestStateChange == ExecutionState.FAILED){
-//				return InternalJobStatus.FAILING;
-//			}
-//			break;
-//		case SCHEDULED:
-//			if (latestStateChange == ExecutionState.RUNNING) {
-//				return InternalJobStatus.RUNNING;
-//			} else if (latestStateChange == ExecutionState.CANCELED) {
-//				if (eg.jobHasFailedOrCanceledStatus()) {
-//					return InternalJobStatus.CANCELED;
-//				}
-//			}else if(latestStateChange == ExecutionState.FAILED){
-//				return InternalJobStatus.FAILING;
-//			}
-//			break;
-//		case RUNNING:
-//			if (latestStateChange == ExecutionState.CANCELED) {
-//				return InternalJobStatus.CANCELING;
-//			}
-//			if (latestStateChange == ExecutionState.FAILED) {
-//
-//				final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(eg, true);
-//				while (it.hasNext()) {
-//
-//					final ExecutionVertex vertex = it.next();
-//					if (vertex.getExecutionState() == ExecutionState.FAILED) {
-//						return InternalJobStatus.FAILING;
-//					}
-//				}
-//			}
-//			if (eg.jobHasFinishedStatus()) {
-//				return InternalJobStatus.FINISHED;
-//			}
-//			break;
-//		case FAILING:
-//			if (eg.jobHasFailedOrCanceledStatus()) {
-//				return InternalJobStatus.FAILED;
-//			}
-//			break;
-//		case FAILED:
-//			LOG.error("Received update of execute state in job status FAILED");
-//			break;
-//		case CANCELING:
-//			if (eg.jobHasFailedOrCanceledStatus()) {
-//				return InternalJobStatus.CANCELED;
-//			}
-//			break;
-//		case CANCELED:
-//			LOG.error("Received update of execute state in job status CANCELED: " + eg.getJobID());
-//			break;
-//		case FINISHED:
-//			LOG.error("Received update of execute state in job status FINISHED: " + eg.getJobID() + " "
-//				+ StringUtils.stringifyException(new Throwable()));
-//			break;
-//		}
-//
-//		return currentJobStatus;
-//	}
-//
-//	/**
-//	 * Returns the current status of the job
-//	 * represented by this execution graph.
-//	 * 
-//	 * @return the current status of the job
-//	 */
-//	public InternalJobStatus getJobStatus() {
-//
-//		return this.jobStatus.get();
-//	}
-//
-//
-//	@Override
-//	public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID,
-//			final ExecutionState newExecutionState, String optionalMessage) {
-//
-//		// Do not use the parameter newExecutionState here as it may already be out-dated
-//
-//		final ExecutionVertex vertex = getVertexByID(vertexID);
-//		if (vertex == null) {
-//			LOG.error("Cannot find execution vertex with the ID " + vertexID);
-//			return;
-//		}
-//
-//		final ExecutionState actualExecutionState = vertex.getExecutionState();
-//
-//		final InternalJobStatus newJobStatus = determineNewJobStatus(this, actualExecutionState);
-//
-//		if (actualExecutionState == ExecutionState.FINISHED) {
-//			// It is worth checking if the current stage has complete
-//			if (this.isCurrentStageCompleted()) {
-//				// Increase current execution stage
-//				++this.indexToCurrentExecutionStage;
-//
-//				if (this.indexToCurrentExecutionStage < this.stages.size()) {
-//					final Iterator<ExecutionStageListener> it = this.executionStageListeners.iterator();
-//					final ExecutionStage nextExecutionStage = getCurrentExecutionStage();
-//					while (it.hasNext()) {
-//						it.next().nextExecutionStageEntered(jobID, nextExecutionStage);
-//					}
-//				}
-//			}
-//		}
-//
-//		updateJobStatus(newJobStatus, optionalMessage);
-//	}
-//
-//	/**
-//	 * Updates the job status to given status and triggers the execution of the {@link JobStatusListener} objects.
-//	 * 
-//	 * @param newJobStatus
-//	 *        the new job status
-//	 * @param optionalMessage
-//	 *        an optional message providing details on the reasons for the state change
-//	 */
-//	public void updateJobStatus(final InternalJobStatus newJobStatus, String optionalMessage) {
-//
-//		// Check if the new job status equals the old one
-//		if (this.jobStatus.getAndSet(newJobStatus) == newJobStatus) {
-//			return;
-//		}
-//
-//		// The task caused the entire job to fail, save the error description
-//		if (newJobStatus == InternalJobStatus.FAILING) {
-//			this.errorDescription = optionalMessage;
-//		}
-//
-//		// If this is the final failure state change, reuse the saved error description
-//		if (newJobStatus == InternalJobStatus.FAILED) {
-//			optionalMessage = this.errorDescription;
-//		}
-//
-//		final Iterator<JobStatusListener> it = this.jobStatusListeners.iterator();
-//		while (it.hasNext()) {
-//			it.next().jobStatusHasChanged(this, newJobStatus, optionalMessage);
-//		}
-//	}
-//
-//	/**
-//	 * Registers a new {@link JobStatusListener} object with this execution graph.
-//	 * After being registered the object will receive notifications about changes
-//	 * of the job status. It is not possible to register the same listener object
-//	 * twice.
-//	 * 
-//	 * @param jobStatusListener
-//	 *        the listener object to register
-//	 */
-//	public void registerJobStatusListener(final JobStatusListener jobStatusListener) {
-//
-//		if (jobStatusListener == null) {
-//			throw new IllegalArgumentException("Argument jobStatusListener must not be null");
-//		}
-//
-//		this.jobStatusListeners.addIfAbsent(jobStatusListener);
-//	}
-//
-//	/**
-//	 * Unregisters the given {@link JobStatusListener} object. After having called this
-//	 * method, the object will no longer receive notifications about changes of the job
-//	 * status.
-//	 * 
-//	 * @param jobStatusListener
-//	 *        the listener object to unregister
-//	 */
-//	public void unregisterJobStatusListener(final JobStatusListener jobStatusListener) {
-//
-//		if (jobStatusListener == null) {
-//			throw new IllegalArgumentException("Argument jobStatusListener must not be null");
-//		}
-//
-//		this.jobStatusListeners.remove(jobStatusListener);
-//	}
-//
-//	/**
-//	 * Registers a new {@link ExecutionStageListener} object with this execution graph. After being registered the
-//	 * object will receive a notification whenever the job has entered its next execution stage. Note that a
-//	 * notification is not sent when the job has entered its initial execution stage.
-//	 * 
-//	 * @param executionStageListener
-//	 *        the listener object to register
-//	 */
-//	public void registerExecutionStageListener(final ExecutionStageListener executionStageListener) {
-//
-//		if (executionStageListener == null) {
-//			throw new IllegalArgumentException("Argument executionStageListener must not be null");
-//		}
-//
-//		this.executionStageListeners.addIfAbsent(executionStageListener);
-//	}
-//
-//	/**
-//	 * Unregisters the given {@link ExecutionStageListener} object. After having called this method, the object will no
-//	 * longer receiver notifications about the execution stage progress.
-//	 * 
-//	 * @param executionStageListener
-//	 *        the listener object to unregister
-//	 */
-//	public void unregisterExecutionStageListener(final ExecutionStageListener executionStageListener) {
-//
-//		if (executionStageListener == null) {
-//			throw new IllegalArgumentException("Argument executionStageListener must not be null");
-//		}
-//
-//		this.executionStageListeners.remove(executionStageListener);
-//	}
-//
-//	/**
-//	 * Returns the name of the original job graph.
-//	 * 
-//	 * @return the name of the original job graph, possibly <code>null</code>
-//	 */
-//	public String getJobName() {
-//		return this.jobName;
-//	}
-//	
-//	private void calculateConnectionIDs() {
-//		final Set<ExecutionGroupVertex> alreadyVisited = new HashSet<ExecutionGroupVertex>();
-//		final ExecutionStage lastStage = getStage(getNumberOfStages() - 1);
-//
-//		for (int i = 0; i < lastStage.getNumberOfStageMembers(); ++i) {
-//
-//			final ExecutionGroupVertex groupVertex = lastStage.getStageMember(i);
-//
-//			int currentConnectionID = 0;
-//
-//			if (groupVertex.isOutputVertex()) {
-//			currentConnectionID = groupVertex.calculateConnectionID(currentConnectionID, alreadyVisited);
-//			}
-//		}
-//	}
-//	
-//	/**
-//	 * Retrieves the number of required slots to run this execution graph
-//	 * @return
-//	 */
-//	public int getRequiredSlots(){
-//		int maxRequiredSlots = 0;
-//
-//		final Iterator<ExecutionStage> stageIterator = this.stages.iterator();
-//
-//		while(stageIterator.hasNext()){
-//			final ExecutionStage stage = stageIterator.next();
-//
-//			int requiredSlots = stage.getRequiredSlots();
-//
-//			if(requiredSlots > maxRequiredSlots){
-//				maxRequiredSlots = requiredSlots;
-//			}
-//		}
-//
-//		return maxRequiredSlots;
-//	}
-//}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
index dcad957..440566d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.executiongraph;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.io.InputSplitAssigner;
@@ -33,6 +32,8 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 
 import org.slf4j.Logger;
@@ -43,12 +44,13 @@ public class ExecutionJobVertex {
 	/** Use the same log for all ExecutionGraph classes */
 	private static final Logger LOG = ExecutionGraph.LOG;
 	
+	private final Object stateMonitor = new Object();
 	
 	private final ExecutionGraph graph;
 	
 	private final AbstractJobVertex jobVertex;
 	
-	private final ExecutionVertex2[] taskVertices;
+	private final ExecutionVertex[] taskVertices;
 
 	private final IntermediateResult[] producedDataSets;
 	
@@ -58,18 +60,21 @@ public class ExecutionJobVertex {
 	
 	private final int parallelism;
 	
-	
-	private final AtomicInteger numRunningTasks = new AtomicInteger(0);
-	
-	private final AtomicInteger numFinishedTasks = new AtomicInteger(0);
-	
-	private final AtomicInteger numCancelledOrFailedTasks = new AtomicInteger(0);
+	private final boolean[] finishedSubtasks;
+			
+	private int numSubtasksInFinalState;
 	
 	
 	private SlotSharingGroup slotSharingGroup;
 	
 	
 	public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex, int defaultParallelism) throws JobException {
+		this(graph, jobVertex, defaultParallelism, System.currentTimeMillis());
+	}
+	
+	public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex, int defaultParallelism, long createTimestamp)
+			throws JobException
+	{
 		if (graph == null || jobVertex == null) {
 			throw new NullPointerException();
 		}
@@ -81,7 +86,7 @@ public class ExecutionJobVertex {
 		int numTaskVertices = vertexParallelism > 0 ? vertexParallelism : defaultParallelism;
 		
 		this.parallelism = numTaskVertices;
-		this.taskVertices = new ExecutionVertex2[numTaskVertices];
+		this.taskVertices = new ExecutionVertex[numTaskVertices];
 		
 		this.inputs = new ArrayList<IntermediateResult>(jobVertex.getInputs().size());
 		
@@ -94,7 +99,7 @@ public class ExecutionJobVertex {
 		
 		// create all task vertices
 		for (int i = 0; i < numTaskVertices; i++) {
-			ExecutionVertex2 vertex = new ExecutionVertex2(this, i, this.producedDataSets);
+			ExecutionVertex vertex = new ExecutionVertex(this, i, this.producedDataSets, createTimestamp);
 			this.taskVertices[i] = vertex;
 		}
 		
@@ -122,6 +127,8 @@ public class ExecutionJobVertex {
 		catch (Throwable t) {
 			throw new JobException("Creating the input splits caused an error: " + t.getMessage(), t);
 		}
+		
+		this.finishedSubtasks = new boolean[parallelism];
 	}
 
 	public ExecutionGraph getGraph() {
@@ -144,7 +151,7 @@ public class ExecutionJobVertex {
 		return this.jobVertex.getID();
 	}
 	
-	public ExecutionVertex2[] getTaskVertices() {
+	public ExecutionVertex[] getTaskVertices() {
 		return taskVertices;
 	}
 	
@@ -168,6 +175,10 @@ public class ExecutionJobVertex {
 		return inputs;
 	}
 	
+	public boolean isInFinalState() {
+		return numSubtasksInFinalState == parallelism;
+	}
+	
 	//---------------------------------------------------------------------------------------------
 	
 	public void connectToPredecessors(Map<IntermediateDataSetID, IntermediateResult> intermediateDataSets) throws JobException {
@@ -204,26 +215,73 @@ public class ExecutionJobVertex {
 			int consumerIndex = ires.registerConsumer();
 			
 			for (int i = 0; i < parallelism; i++) {
-				ExecutionVertex2 ev = taskVertices[i];
+				ExecutionVertex ev = taskVertices[i];
 				ev.connectSource(num, ires, edge, consumerIndex);
 			}
 		}
 	}
 	
 	//---------------------------------------------------------------------------------------------
-	//  State, deployment, and recovery logic 
+	//  Actions
 	//---------------------------------------------------------------------------------------------
 	
-	void vertexSwitchedToRunning(int subtask) {
-		this.numRunningTasks.incrementAndGet();
+	public void scheduleAll(DefaultScheduler scheduler, boolean queued) throws NoResourceAvailableException {
+		for (ExecutionVertex ev : getTaskVertices()) {
+			ev.scheduleForExecution(scheduler, queued);
+		}
+	}
+
+	public void cancel() {
+		for (ExecutionVertex ev : getTaskVertices()) {
+			ev.cancel();
+		}
 	}
 	
-	void vertexFailed(int subtask) {
-		
+	public void fail(Throwable t) {
+		for (ExecutionVertex ev : getTaskVertices()) {
+			ev.fail(t);
+		}
+	}
+	
+	public void waitForAllVerticesToReachFinishingState() throws InterruptedException {
+		synchronized (stateMonitor) {
+			while (numSubtasksInFinalState < parallelism) {
+				stateMonitor.wait();
+			}
+		}
+	}
+	
+	//---------------------------------------------------------------------------------------------
+	//  Notifications
+	//---------------------------------------------------------------------------------------------
+	
+	void vertexFinished(int subtask) {
+		subtaskInFinalState(subtask);
 	}
 	
 	void vertexCancelled(int subtask) {
-		
+		subtaskInFinalState(subtask);
+	}
+	
+	void vertexFailed(int subtask, Throwable error) {
+		subtaskInFinalState(subtask);
+	}
+	
+	private void subtaskInFinalState(int subtask) {
+		synchronized (stateMonitor) {
+			if (!finishedSubtasks[subtask]) {
+				finishedSubtasks[subtask] = true;
+				numSubtasksInFinalState++;
+				
+				if (numSubtasksInFinalState == parallelism) {
+					// we are in our final state
+					stateMonitor.notifyAll();
+					
+					// tell the graph
+					graph.jobVertexInFinalState(this);
+				}
+			}
+		}
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -233,5 +291,4 @@ public class ExecutionJobVertex {
 	public void execute(Runnable action) {
 		this.graph.execute(action);
 	}
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/ae139f5a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java
deleted file mode 100644
index 31108a4..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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;
-
-public enum ExecutionState {
-
-	/** The job is scheduled, but nothing is running yet */
-	CREATED,
-	
-	/** At least some tasks of the job are running */
-	RUNNING,
-	
-	/** The execution has finished, no further tasks may be added to the job */
-	FINISHED,
-	
-	/** The execution has failed */
-	FAILED
-}


[21/63] [abbrv] Unify all job vertices to one type (rather than dedicated input/output types)

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
index e7d590d..d1f4ae0 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/danglingpagerank/CompensatableDanglingPageRank.java
@@ -33,10 +33,11 @@ import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.BuildSecondCachedMatchDriver;
 import org.apache.flink.runtime.operators.CoGroupDriver;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
@@ -118,7 +119,7 @@ public class CompensatableDanglingPageRank {
 		// --------------- the inputs ---------------------
 
 		// page rank input
-		JobInputVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(),
+		InputFormatInputVertex pageWithRankInput = JobGraphUtils.createInput(new ImprovedDanglingPageRankInputFormat(),
 			pageWithRankInputPath, "DanglingPageWithRankInput", jobGraph, degreeOfParallelism);
 		TaskConfig pageWithRankInputConfig = new TaskConfig(pageWithRankInput.getConfiguration());
 		pageWithRankInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -127,7 +128,7 @@ public class CompensatableDanglingPageRank {
 		pageWithRankInputConfig.setStubParameter("pageRank.numVertices", String.valueOf(numVertices));
 
 		// edges as adjacency list
-		JobInputVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(),
+		InputFormatInputVertex adjacencyListInput = JobGraphUtils.createInput(new ImprovedAdjacencyListInputFormat(),
 			adjacencyListInputPath, "AdjancencyListInput", jobGraph, degreeOfParallelism);
 		TaskConfig adjacencyListInputConfig = new TaskConfig(adjacencyListInput.getConfiguration());
 		adjacencyListInputConfig.addOutputShipStrategy(ShipStrategyType.PARTITION_HASH);
@@ -247,7 +248,7 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the output ---------------------
 
-		JobOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
+		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "FinalOutput", degreeOfParallelism);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		outputConfig.addInputToGroup(0);
 		outputConfig.setInputSerializer(recSerializer, 0);
@@ -256,10 +257,10 @@ public class CompensatableDanglingPageRank {
 		
 		// --------------- the auxiliaries ---------------------
 		
-		JobOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
+		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput",
 			degreeOfParallelism);
 
-		JobOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
+		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, degreeOfParallelism);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.addIterationAggregator(CompensatableDotProductCoGroup.AGGREGATOR_NAME, new PageRankStatsAggregator());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cb7039e3/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
index 7424de6..c365378 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NetworkStackThroughput.java
@@ -32,9 +32,9 @@ import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.JobTaskVertex;
+import org.apache.flink.runtime.jobgraph.SimpleInputVertex;
+import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.test.util.RecordAPITestBase;
 import org.junit.After;
@@ -99,7 +99,7 @@ public class NetworkStackThroughput {
 
 			JobGraph jobGraph = new JobGraph("Speed Test");
 
-			JobInputVertex producer = new JobInputVertex("Speed Test Producer", jobGraph);
+			SimpleInputVertex producer = new SimpleInputVertex("Speed Test Producer", jobGraph);
 			producer.setInvokableClass(SpeedTestProducer.class);
 			producer.setNumberOfSubtasks(numSubtasks);
 			producer.getConfiguration().setInteger(DATA_VOLUME_GB_CONFIG_KEY, dataVolumeGb);
@@ -112,7 +112,7 @@ public class NetworkStackThroughput {
 				forwarder.setNumberOfSubtasks(numSubtasks);
 			}
 
-			JobOutputVertex consumer = new JobOutputVertex("Speed Test Consumer", jobGraph);
+			SimpleOutputVertex consumer = new SimpleOutputVertex("Speed Test Consumer", jobGraph);
 			consumer.setInvokableClass(SpeedTestConsumer.class);
 			consumer.setNumberOfSubtasks(numSubtasks);
 			consumer.getConfiguration().setBoolean(IS_SLOW_RECEIVER_CONFIG_KEY, isSlowReceiver);


[51/63] [abbrv] git commit: Fix bug in topological sort

Posted by se...@apache.org.
Fix bug in topological sort


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/f229d5bc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/f229d5bc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/f229d5bc

Branch: refs/heads/master
Commit: f229d5bcb9d877cbac5eec4120bd4d9ed4548c8d
Parents: 8231b62
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 16 19:48:02 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:50 2014 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/jobgraph/JobGraph.java |  4 ++
 .../flink/runtime/jobgraph/JobGraphTest.java    | 39 ++++++++++++++++++++
 2 files changed, 43 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f229d5bc/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
index 85978fe..dc6eb04 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobGraph.java
@@ -280,6 +280,10 @@ public class JobGraph implements IOReadableWritable {
 				
 				// a vertex can be added, if it has no predecessors that are still in the 'remaining' set
 				AbstractJobVertex v = edge.getTarget();
+				if (!remaining.contains(v)) {
+					continue;
+				}
+				
 				boolean hasNewPredecessors = false;
 				
 				for (JobEdge e : v.getInputs()) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/f229d5bc/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
index 768ac82..18b16e7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
@@ -94,6 +94,8 @@ public class JobGraphTest {
 			JobGraph graph = new JobGraph("TestGraph", source1, source2, intermediate1, intermediate2, target1, target2);
 			List<AbstractJobVertex> sorted = graph.getVerticesSortedTopologicallyFromSources();
 			
+			assertEquals(6, sorted.size());
+			
 			assertBefore(source1, target1, sorted);
 			assertBefore(source1, target2, sorted);
 			assertBefore(source2, target2, sorted);
@@ -136,6 +138,8 @@ public class JobGraphTest {
 			JobGraph graph = new JobGraph("TestGraph", source1, source2, root, l11, l13, l12, l2);
 			List<AbstractJobVertex> sorted = graph.getVerticesSortedTopologicallyFromSources();
 			
+			assertEquals(7,  sorted.size());
+			
 			assertBefore(source1, root, sorted);
 			assertBefore(source2, root, sorted);
 			assertBefore(l11, root, sorted);
@@ -159,6 +163,41 @@ public class JobGraphTest {
 	}
 	
 	@Test
+	public void testTopologicalSort3() {
+		//             --> op1 --
+		//            /         \
+		//  (source) -           +-> op2 -> op3
+		//            \         /
+		//             ---------
+		
+		try {
+			AbstractJobVertex source = new AbstractJobVertex("source");
+			AbstractJobVertex op1 = new AbstractJobVertex("op4");
+			AbstractJobVertex op2 = new AbstractJobVertex("op2");
+			AbstractJobVertex op3 = new AbstractJobVertex("op3");
+			
+			op1.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE);
+			op2.connectNewDataSetAsInput(op1, DistributionPattern.POINTWISE);
+			op2.connectNewDataSetAsInput(source, DistributionPattern.POINTWISE);
+			op3.connectNewDataSetAsInput(op2, DistributionPattern.POINTWISE);
+			
+			JobGraph graph = new JobGraph("TestGraph", source, op1, op2, op3);
+			List<AbstractJobVertex> sorted = graph.getVerticesSortedTopologicallyFromSources();
+			
+			assertEquals(4,  sorted.size());
+			
+			assertBefore(source, op1, sorted);
+			assertBefore(source, op2, sorted);
+			assertBefore(op1, op2, sorted);
+			assertBefore(op2, op3, sorted);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
 	public void testTopoSortCyclicGraphNoSources() {
 		try {
 			AbstractJobVertex v1 = new AbstractJobVertex("1");


[29/63] [abbrv] Redesign Scheduler from pre-assignment to more flexible schedule-on-demand model

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
index a479bd3..2d30cf2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.taskmanager;
 
 import java.io.File;
@@ -55,8 +54,8 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry;
 import org.apache.flink.configuration.ConfigConstants;
@@ -75,6 +74,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
 import org.apache.flink.runtime.filecache.FileCache;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.ChannelManager;
 import org.apache.flink.runtime.io.network.InsufficientResourcesException;
@@ -95,7 +95,6 @@ import org.apache.flink.runtime.protocols.ChannelLookupProtocol;
 import org.apache.flink.runtime.protocols.InputSplitProviderProtocol;
 import org.apache.flink.runtime.protocols.JobManagerProtocol;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
-import org.apache.flink.runtime.taskmanager.transferenvelope.RegisterTaskManagerResult;
 import org.apache.flink.runtime.types.IntegerRecord;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.SerializableArrayList;
@@ -110,12 +109,19 @@ import org.apache.flink.util.StringUtils;
  */
 public class TaskManager implements TaskOperationProtocol {
 
-	private static final Logger LOG = LoggerFactory.getLogger(TaskManager.class);
+	private static final Log LOG = LogFactory.getLog(TaskManager.class);
 
-	private final static int FAILURE_RETURN_CODE = -1;
+	private static final int STARTUP_FAILURE_RETURN_CODE = 1;
+	
+	private static final int CRITICAL_ERROR_RETURN_CODE = 2;
 	
 	private static final int IPC_HANDLER_COUNT = 1;
 	
+	private static final int MAX_LOST_HEART_BEATS = 3;
+	
+	private static final int DELAY_AFTER_LOST_CONNECTION = 10000;
+	
+	
 	public final static String ARG_CONF_DIR = "tempDir";
 	
 	// --------------------------------------------------------------------------------------------
@@ -126,7 +132,6 @@ public class TaskManager implements TaskOperationProtocol {
 	
 	private final ExecutionMode executionMode;
 	
-	
 	private final JobManagerProtocol jobManager;
 
 	private final InputSplitProviderProtocol globalInputSplitProvider;
@@ -169,9 +174,12 @@ public class TaskManager implements TaskOperationProtocol {
 	
 	private final AtomicBoolean shutdownStarted = new AtomicBoolean(false);
 	
+	private volatile InstanceID registeredId;
+	
 	/** Stores whether the task manager has already been shut down. */
 	private volatile boolean shutdownComplete;
 	
+	
 	/**
 	 * All parameters are obtained from the 
 	 * {@link GlobalConfiguration}, which must be loaded prior to instantiating the task manager.
@@ -203,7 +211,7 @@ public class TaskManager implements TaskOperationProtocol {
 				jobManagerAddress = new InetSocketAddress(tmpAddress, port);
 			}
 			catch (UnknownHostException e) {
-				LOG.error("Could not resolve JobManager host name.");
+				LOG.fatal("Could not resolve JobManager host name.");
 				throw new Exception("Could not resolve JobManager host name: " + e.getMessage(), e);
 			}
 			
@@ -214,7 +222,7 @@ public class TaskManager implements TaskOperationProtocol {
 		try {
 			this.jobManager = RPC.getProxy(JobManagerProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
 		} catch (IOException e) {
-			LOG.error("Could not connect to the JobManager: " + e.getMessage(), e);
+			LOG.fatal("Could not connect to the JobManager: " + e.getMessage(), e);
 			throw new Exception("Failed to initialize connection to JobManager: " + e.getMessage(), e);
 		}
 		
@@ -245,7 +253,7 @@ public class TaskManager implements TaskOperationProtocol {
 				this.taskManagerServer = RPC.getServer(this, taskManagerAddress.getHostAddress(), ipcPort, IPC_HANDLER_COUNT);
 				this.taskManagerServer.start();
 			} catch (IOException e) {
-				LOG.error("Failed to start TaskManager server. " + e.getMessage(), e);
+				LOG.fatal("Failed to start TaskManager server. " + e.getMessage(), e);
 				throw new Exception("Failed to start taskmanager server. " + e.getMessage(), e);
 			}
 		}
@@ -254,7 +262,7 @@ public class TaskManager implements TaskOperationProtocol {
 		try {
 			this.globalInputSplitProvider = RPC.getProxy(InputSplitProviderProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
 		} catch (IOException e) {
-			LOG.error(e.getMessage(), e);
+			LOG.fatal(e.getMessage(), e);
 			throw new Exception("Failed to initialize connection to global input split provider: " + e.getMessage(), e);
 		}
 
@@ -262,7 +270,7 @@ public class TaskManager implements TaskOperationProtocol {
 		try {
 			this.lookupService = RPC.getProxy(ChannelLookupProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
 		} catch (IOException e) {
-			LOG.error(e.getMessage(), e);
+			LOG.fatal(e.getMessage(), e);
 			throw new Exception("Failed to initialize channel lookup protocol. " + e.getMessage(), e);
 		}
 
@@ -270,7 +278,7 @@ public class TaskManager implements TaskOperationProtocol {
 		try {
 			this.accumulatorProtocolProxy = RPC.getProxy(AccumulatorProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
 		} catch (IOException e) {
-			LOG.error("Failed to initialize accumulator protocol: " + e.getMessage(), e);
+			LOG.fatal("Failed to initialize accumulator protocol: " + e.getMessage(), e);
 			throw new Exception("Failed to initialize accumulator protocol: " + e.getMessage(), e);
 		}
 
@@ -324,17 +332,13 @@ public class TaskManager implements TaskOperationProtocol {
 							ConfigConstants.TASK_MANAGER_NET_NUM_OUT_THREADS_KEY,
 							ConfigConstants.DEFAULT_TASK_MANAGER_NET_NUM_OUT_THREADS);
 
-					int lowWaterMark = GlobalConfiguration.getInteger(
-							ConfigConstants.TASK_MANAGER_NET_NETTY_LOW_WATER_MARK,
-							ConfigConstants.DEFAULT_TASK_MANAGER_NET_NETTY_LOW_WATER_MARK);
-
-					int highWaterMark = GlobalConfiguration.getInteger(
-							ConfigConstants.TASK_MANAGER_NET_NETTY_HIGH_WATER_MARK,
-							ConfigConstants.DEFAULT_TASK_MANAGER_NET_NETTY_HIGH_WATER_MARK);
+					int closeAfterIdleForMs = GlobalConfiguration.getInteger(
+							ConfigConstants.TASK_MANAGER_NET_CLOSE_AFTER_IDLE_FOR_MS_KEY,
+							ConfigConstants.DEFAULT_TASK_MANAGER_NET_CLOSE_AFTER_IDLE_FOR_MS);
 
 					networkConnectionManager = new NettyConnectionManager(
 							localInstanceConnectionInfo.address(), localInstanceConnectionInfo.dataPort(),
-							bufferSize, numInThreads, numOutThreads, lowWaterMark, highWaterMark);
+							bufferSize, numInThreads, numOutThreads, closeAfterIdleForMs);
 					break;
 			}
 
@@ -391,7 +395,7 @@ public class TaskManager implements TaskOperationProtocol {
 				
 				this.memoryManager = new DefaultMemoryManager(memorySize, this.numberOfSlots, pageSize);
 			} catch (Throwable t) {
-				LOG.error("Unable to initialize memory manager with " + (memorySize >>> 20) + " megabytes of memory.", t);
+				LOG.fatal("Unable to initialize memory manager with " + (memorySize >>> 20) + " megabytes of memory.", t);
 				throw new Exception("Unable to initialize memory manager.", t);
 			}
 		}
@@ -400,15 +404,21 @@ public class TaskManager implements TaskOperationProtocol {
 
 		this.ioManager = new IOManager(tmpDirPaths);
 		
-		this.heartbeatThread = new Thread() {
-			@Override
-			public void run() {
-				runHeartbeatLoop();
-			}
-		};
-
-		this.heartbeatThread.setName("Heartbeat Thread");
-		this.heartbeatThread.start();
+		// start the heart beats
+		{
+			final long interval = GlobalConfiguration.getInteger(
+					ConfigConstants.TASK_MANAGER_HEARTBEAT_INTERVAL_KEY,
+					ConfigConstants.DEFAULT_TASK_MANAGER_HEARTBEAT_INTERVAL);
+			
+			this.heartbeatThread = new Thread() {
+				@Override
+				public void run() {
+					registerAndRunHeartbeatLoop(interval, MAX_LOST_HEART_BEATS);
+				}
+			};
+			this.heartbeatThread.setName("Heartbeat Thread");
+			this.heartbeatThread.start();
+		}
 
 		// --------------------------------------------------------------------
 		// Memory Usage
@@ -500,7 +510,7 @@ public class TaskManager implements TaskOperationProtocol {
 			line = parser.parse(options, args);
 		} catch (ParseException e) {
 			System.err.println("CLI Parsing failed. Reason: " + e.getMessage());
-			System.exit(FAILURE_RETURN_CODE);
+			System.exit(STARTUP_FAILURE_RETURN_CODE);
 		}
 
 		String configDir = line.getOptionValue(configDirOpt.getOpt(), null);
@@ -524,8 +534,8 @@ public class TaskManager implements TaskOperationProtocol {
 		try {
 			new TaskManager(ExecutionMode.CLUSTER);
 		} catch (Exception e) {
-			LOG.error("Taskmanager startup failed: " + e.getMessage(), e);
-			System.exit(FAILURE_RETURN_CODE);
+			LOG.fatal("Taskmanager startup failed: " + e.getMessage(), e);
+			System.exit(STARTUP_FAILURE_RETURN_CODE);
 		}
 		
 		// park the main thread to keep the JVM alive (all other threads may be daemon threads)
@@ -537,63 +547,7 @@ public class TaskManager implements TaskOperationProtocol {
 		}
 	}
 
-	/**
-	 * This method send the periodic heartbeats.
-	 */
-	private void runHeartbeatLoop() {
-		final long interval = GlobalConfiguration.getInteger(
-						ConfigConstants.TASK_MANAGER_HEARTBEAT_INTERVAL_KEY,
-						ConfigConstants.DEFAULT_TASK_MANAGER_HEARTBEAT_INTERVAL);
-
-		try {
-			while(!shutdownStarted.get()){
-				RegisterTaskManagerResult result  = this.jobManager.registerTaskManager(this
-								.localInstanceConnectionInfo,this.hardwareDescription,
-						new IntegerRecord(this.numberOfSlots));
-
-				if(result.getReturnCode() == RegisterTaskManagerResult.ReturnCode.SUCCESS){
-					break;
-				}
-
-				try{
-					Thread.sleep(50);
-				}catch(InterruptedException e){
-					if (!shutdownStarted.get()) {
-						LOG.error("TaskManager register task manager loop was interrupted without shutdown.");
-					}
-				}
-			}
-
-		} catch (IOException e) {
-			if(!shutdownStarted.get()){
-				LOG.error("Registering task manager caused an exception: " + e.getMessage(), e);
-			}
-			return;
-		}
 
-		while (!shutdownStarted.get()) {
-			// sleep until the next heart beat
-			try {
-				Thread.sleep(interval);
-			}
-			catch (InterruptedException e) {
-				if (!shutdownStarted.get()) {
-					LOG.error("TaskManager heart beat loop was interrupted without shutdown.");
-				}
-			}
-
-			// send heart beat
-			try {
-				this.jobManager.sendHeartbeat(this.localInstanceConnectionInfo);
-			} catch (IOException e) {
-				if (shutdownStarted.get()) {
-					break;
-				} else {
-					LOG.error("Sending the heart beat caused an exception: " + e.getMessage(), e);
-				}
-			}
-		}
-	}
 
 	
 	/**
@@ -979,7 +933,7 @@ public class TaskManager implements TaskOperationProtocol {
 				this.jobManager.updateTaskExecutionState(new TaskExecutionState(jobID, id, newExecutionState,
 					optionalDescription));
 			} catch (IOException e) {
-				LOG.error("Could not update task execution state.", e);
+				LOG.error(e);
 			}
 		}
 	}
@@ -1046,7 +1000,7 @@ public class TaskManager implements TaskOperationProtocol {
 				this.executorService.awaitTermination(5000L, TimeUnit.MILLISECONDS);
 			} catch (InterruptedException e) {
 				if (LOG.isDebugEnabled()) {
-					LOG.debug("Got interrupted while awaiting the termination of the executor service.", e);
+					LOG.debug(e);
 				}
 			}
 		}
@@ -1065,7 +1019,6 @@ public class TaskManager implements TaskOperationProtocol {
 
 	@Override
 	public void logBufferUtilization() {
-
 		this.channelManager.logBufferUtilization();
 	}
 
@@ -1090,38 +1043,8 @@ public class TaskManager implements TaskOperationProtocol {
 		this.channelManager.invalidateLookupCacheEntries(channelIDs);
 	}
 
-	/**
-	 * Checks, whether the given strings describe existing directories that are writable. If that is not
-	 * the case, an exception is raised.
-	 * 
-	 * @param tempDirs
-	 *        An array of strings which are checked to be paths to writable directories.
-	 * @throws Exception
-	 *         Thrown, if any of the mentioned checks fails.
-	 */
-	private static final void checkTempDirs(final String[] tempDirs) throws Exception {
-
-		for (int i = 0; i < tempDirs.length; ++i) {
-
-			final String dir = tempDirs[i];
-			if (dir == null) {
-				throw new Exception("Temporary file directory #" + (i + 1) + " is null.");
-			}
-
-			final File f = new File(dir);
-
-			if (!f.exists()) {
-				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' does not exist.");
-			}
-
-			if (!f.isDirectory()) {
-				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' is not a directory.");
-			}
-
-			if (!f.canWrite()) {
-				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' is not writable.");
-			}
-		}
+	public void cancelAndClearEverything() {
+		LOG.info("Cancelling all computations and discarding all cached data.");
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -1136,8 +1059,152 @@ public class TaskManager implements TaskOperationProtocol {
 		return this.executionMode;
 	}
 	
+	/**
+	 * Gets the ID under which the TaskManager is currently registered at its JobManager.
+	 * If the TaskManager has not been registered, yet, or if it lost contact, this is is null.
+	 * 
+	 * @return The ID under which the TaskManager is currently registered.
+	 */
+	public InstanceID getRegisteredId() {
+		return this.registeredId;
+	}
+	
+	/**
+	 * Checks if the TaskManager is properly registered and ready to receive work.
+	 * 
+	 * @return True, if the TaskManager is registered, false otherwise.
+	 */
+	public boolean isRegistered() {
+		return this.registeredId != null;
+	}
+	
 	// --------------------------------------------------------------------------------------------
-	//  Memory and Garbace Collection Debugging Utilities
+	//  Heartbeats
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * This method registers the TaskManager at the jobManager and send periodic heartbeats.
+	 */
+	private void registerAndRunHeartbeatLoop(long interval, int maxNonSuccessfulHeatbeats) {
+
+		while (!shutdownStarted.get()) {
+			InstanceID resultId = null;
+	
+			// try to register. We try as long as we need to, because it may be that the jobmanager is  not yet online
+			{
+				final long maxDelay = 10000;	// the maximal delay between registration attempts
+				final long reportingDelay = 5000;
+				long currentDelay = 100;		// initially, wait 100 msecs for the next registration attempt
+				
+				while (!shutdownStarted.get())
+				{
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Trying to register at Jobmanager...");
+					}
+					
+					try {
+						resultId = this.jobManager.registerTaskManager(this.localInstanceConnectionInfo,
+								this.hardwareDescription, this.numberOfSlots);
+					}
+					catch (IOException e) {
+						// this may be if the job manager was not yet online
+						// if this has happened for a while, report it. if it has just happened
+						// at the very beginning, this may not mean anything (JM still in startup)
+						if (currentDelay >= reportingDelay) {
+							LOG.error("Connection to JobManager failed.", e);
+						} else if (LOG.isDebugEnabled()) {
+							LOG.debug("Could not connect to JobManager.", e);
+						}
+					}
+					
+					// check if we were accepted
+					if (resultId != null) {
+						// success
+						this.registeredId = resultId;
+						break;
+					} else {
+						// this is bad. The job manager refused us. report and try again later
+						LOG.error("Registration attempt refused by JobManager.");
+					}
+		
+					try {
+						Thread.sleep(currentDelay);
+					}
+					catch (InterruptedException e) {
+						// may be due to shutdown
+						if (!shutdownStarted.get()) {
+							LOG.error("TaskManager's registration loop was interrupted without shutdown.");
+						}
+					}
+					
+					// increase the time between registration attempts, to not keep on pinging overly frequently
+					currentDelay = Math.min(2 * currentDelay, maxDelay);
+				}
+			}
+			
+			// registration complete, or shutdown
+			int successiveUnsuccessfulHeartbeats = 0;
+			
+			// the heart beat loop
+			while (!shutdownStarted.get()) {
+				// sleep until the next heart beat
+				try {
+					Thread.sleep(interval);
+				}
+				catch (InterruptedException e) {
+					if (!shutdownStarted.get()) {
+						LOG.error("TaskManager heart beat loop was interrupted without shutdown.");
+					}
+				}
+	
+				// send heart beat
+				try {
+					boolean accepted = this.jobManager.sendHeartbeat(resultId);
+					
+					if (accepted) {
+						// reset the unsuccessful heart beats
+						successiveUnsuccessfulHeartbeats = 0;
+					} else {
+						successiveUnsuccessfulHeartbeats++;
+						LOG.error("JobManager rejected heart beat.");
+					}
+				}
+				catch (IOException e) {
+					if (!shutdownStarted.get()) {
+						successiveUnsuccessfulHeartbeats++;
+						LOG.error("Sending the heart beat failed on I/O error: " + e.getMessage(), e);
+					}
+				}
+				
+				if (successiveUnsuccessfulHeartbeats == maxNonSuccessfulHeatbeats) {
+					// we are done for, we cannot connect to the jobmanager any more
+					// or we are not welcome there any more
+					// what to do now? Wait for a while and try to reconnect
+					LOG.error("TaskManager has lost connection to JobManager.");
+					
+					// mark us as disconnected and abort all computation
+					this.registeredId = null;
+					cancelAndClearEverything();
+					
+					// wait for a while, then attempt to register again
+					try {
+						Thread.sleep(DELAY_AFTER_LOST_CONNECTION);
+					}
+					catch (InterruptedException e) {
+						if (!shutdownStarted.get()) {
+							LOG.error("TaskManager heart beat loop was interrupted without shutdown.");
+						}
+					}
+					
+					// leave the heart beat loop
+					break;
+				}
+			} // end heart beat loop
+		} // end while not shutdown
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Memory and Garbage Collection Debugging Utilities
 	// --------------------------------------------------------------------------------------------
 
 	private String getMemoryUsageStatsAsString(MemoryMXBean memoryMXBean) {
@@ -1175,4 +1242,55 @@ public class TaskManager implements TaskOperationProtocol {
 
 		return str.toString();
 	}
+	
+	
+	// --------------------------------------------------------------------------------------------
+	// Miscellaneous Utilities
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Checks, whether the given strings describe existing directories that are writable. If that is not
+	 * the case, an exception is raised.
+	 * 
+	 * @param tempDirs An array of strings which are checked to be paths to writable directories.
+	 * @throws Exception Thrown, if any of the mentioned checks fails.
+	 */
+	private static final void checkTempDirs(final String[] tempDirs) throws Exception {
+		for (int i = 0; i < tempDirs.length; ++i) {
+			final String dir = tempDirs[i];
+			if (dir == null) {
+				throw new Exception("Temporary file directory #" + (i + 1) + " is null.");
+			}
+
+			final File f = new File(dir);
+
+			if (!f.exists()) {
+				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' does not exist.");
+			}
+
+			if (!f.isDirectory()) {
+				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' is not a directory.");
+			}
+
+			if (!f.canWrite()) {
+				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' is not writable.");
+			}
+		}
+	}
+	
+	public static class EmergencyShutdownExceptionHandler implements Thread.UncaughtExceptionHandler {
+
+		private final TaskManager tm;
+		
+		public EmergencyShutdownExceptionHandler(TaskManager tm) {
+			this.tm = tm;
+		}
+		
+		@Override
+		public void uncaughtException(Thread t, Throwable e) {
+			LOG.fatal("Thread " + t.getName() + " caused an unrecoverable exception.", e);
+			tm.shutdown();
+		}
+		
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/transferenvelope/RegisterTaskManagerResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/transferenvelope/RegisterTaskManagerResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/transferenvelope/RegisterTaskManagerResult.java
deleted file mode 100644
index 27fae1c..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/transferenvelope/RegisterTaskManagerResult.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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.taskmanager.transferenvelope;
-
-import java.io.IOException;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.util.EnumUtils;
-
-public class RegisterTaskManagerResult implements IOReadableWritable {
-	public enum ReturnCode{
-		SUCCESS, FAILURE
-	};
-
-	public RegisterTaskManagerResult(){
-		this.returnCode = ReturnCode.SUCCESS;
-	}
-
-	public RegisterTaskManagerResult(ReturnCode returnCode){
-		this.returnCode = returnCode;
-	}
-
-	private ReturnCode returnCode;
-
-	public ReturnCode getReturnCode() { return this.returnCode; }
-
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-		EnumUtils.writeEnum(out, this.returnCode);
-	}
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-		this.returnCode = EnumUtils.readEnum(in, ReturnCode.class);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
index 290326c..02c814c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
@@ -55,7 +55,6 @@ import org.apache.flink.runtime.protocols.InputSplitProviderProtocol;
 import org.apache.flink.runtime.protocols.JobManagerProtocol;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskManager;
-import org.apache.flink.runtime.taskmanager.transferenvelope.RegisterTaskManagerResult;
 import org.apache.flink.runtime.types.IntegerRecord;
 
 import org.junit.After;
@@ -214,16 +213,6 @@ public class LocalInstanceManagerTest {
 		}
 
 		@Override
-		public void sendHeartbeat(InstanceConnectionInfo instanceConnectionInfo) {}
-
-		@Override
-		public RegisterTaskManagerResult registerTaskManager(InstanceConnectionInfo instanceConnectionInfo,
-				HardwareDescription hardwareDescription, IntegerRecord numberOfSlots)
-		{
-			return new RegisterTaskManagerResult(RegisterTaskManagerResult.ReturnCode.SUCCESS);
-		}
-
-		@Override
 		public void updateTaskExecutionState(TaskExecutionState taskExecutionState) {}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java
index 76616b8..3a24f97 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java
@@ -16,185 +16,165 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobmanager.scheduler;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
-import java.io.IOException;
-import java.util.List;
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 
-import org.apache.flink.api.common.io.GenericInputFormat;
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.executiongraph.GraphConversionException;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
-import org.apache.flink.runtime.jobmanager.scheduler.SchedulingException;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.util.StringUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobgraph.JobID;
 import org.junit.Test;
 
 /**
  * This class checks the functionality of the {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler} class
  */
-@SuppressWarnings("serial")
 public class DefaultSchedulerTest {
 
-
-	public static final class InputTask extends AbstractInvokable {
-
-		@Override
-		public void registerInputOutput() {
-			new RecordWriter<StringRecord>(this);
-		}
-
-		@Override
-		public void invoke() throws Exception {}
-
-	}
-
-	public static final class OutputTask extends AbstractInvokable {
-
-		@Override
-		public void registerInputOutput() {
-			new RecordReader<StringRecord>(this, StringRecord.class);
-		}
-
-		@Override
-		public void invoke() throws Exception {}
-
-	}
-
-	public static final class DummyInputFormat extends GenericInputFormat<IntValue> {
-
-		@Override
-		public boolean reachedEnd() throws IOException {
-			return true;
-		}
-
-		@Override
-		public IntValue nextRecord(IntValue reuse) throws IOException {
-			return null;
-		}
-	}
-
-	public static final class DummyOutputFormat implements OutputFormat<IntValue> {
-
-		@Override
-		public void configure(Configuration parameters) {}
-
-		@Override
-		public void open(int taskNumber, int numTasks) {}
-
-		@Override
-		public void writeRecord(IntValue record) {}
-
-		@Override
-		public void close() {}
-	}
-
-	/**
-	 * Constructs a sample execution graph consisting of two vertices connected by a channel of the given type.
-	 * 
-	 * @param channelType
-	 *        the channel type to connect the vertices with
-	 * @return a sample execution graph
-	 */
-	private ExecutionGraph createExecutionGraph(ChannelType channelType) {
-
-		final JobGraph jobGraph = new JobGraph("Job Graph");
-
-		final JobInputVertex inputVertex = new JobInputVertex("Input 1", jobGraph);
-		inputVertex.setInvokableClass(InputTask.class);
-		inputVertex.setInputFormat(new DummyInputFormat());
-		inputVertex.setNumberOfSubtasks(1);
-
-		final JobOutputVertex outputVertex = new JobOutputVertex("Output 1", jobGraph);
-		outputVertex.setInvokableClass(OutputTask.class);
-		outputVertex.setOutputFormat(new DummyOutputFormat());
-		outputVertex.setNumberOfSubtasks(1);
-
+	private int portNum = 10000;
+	
+	@Test
+	public void testAddAndRemoveInstance() {
 		try {
-			inputVertex.connectTo(outputVertex, channelType);
-		} catch (JobGraphDefinitionException e) {
-			fail(StringUtils.stringifyException(e));
+			DefaultScheduler scheduler = new DefaultScheduler();
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			Instance i3 = getRandomInstance(2);
+			
+			assertEquals(0, scheduler.getNumberOfAvailableInstances());
+			scheduler.newInstanceAvailable(i1);
+			assertEquals(1, scheduler.getNumberOfAvailableInstances());
+			scheduler.newInstanceAvailable(i2);
+			assertEquals(2, scheduler.getNumberOfAvailableInstances());
+			scheduler.newInstanceAvailable(i3);
+			assertEquals(3, scheduler.getNumberOfAvailableInstances());
+			
+			// cannot add available instance again
+			try {
+				scheduler.newInstanceAvailable(i2);
+				fail("Scheduler accepted instance twice");
+			}
+			catch (IllegalArgumentException e) {
+				// bueno!
+			}
+			
+			// some instances die
+			assertEquals(3, scheduler.getNumberOfAvailableInstances());
+			scheduler.instanceDied(i2);
+			assertEquals(2, scheduler.getNumberOfAvailableInstances());
+			
+			// try to add a dead instance
+			try {
+				scheduler.newInstanceAvailable(i2);
+				fail("Scheduler accepted dead instance");
+			}
+			catch (IllegalArgumentException e) {
+				// stimmt
+				
+			}
+						
+			scheduler.instanceDied(i1);
+			assertEquals(1, scheduler.getNumberOfAvailableInstances());
+			scheduler.instanceDied(i3);
+			assertEquals(0, scheduler.getNumberOfAvailableInstances());
+			
+			assertFalse(i1.isAlive());
+			assertFalse(i2.isAlive());
+			assertFalse(i3.isAlive());
 		}
-
-		try {
-			LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
-			return new ExecutionGraph(jobGraph, 1);
-
-		} catch (GraphConversionException e) {
-			fail(StringUtils.stringifyException(e));
-		} catch (IOException e) {
-			fail(StringUtils.stringifyException(e));
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
 		}
-
-		return null;
 	}
-
-	/**
-	 * Checks the behavior of the scheduleJob() method with a job consisting of two tasks connected via an in-memory
-	 * channel.
-	 */
+	
+	
 	@Test
-	public void testScheduleJobWithInMemoryChannel() {
-
-		final TestInstanceManager tim = new TestInstanceManager();
-		final TestDeploymentManager tdm = new TestDeploymentManager();
-		final DefaultScheduler scheduler = new DefaultScheduler(tdm, tim);
-
-		final ExecutionGraph executionGraph = createExecutionGraph(ChannelType.IN_MEMORY);
-
+	public void testAssignToSlots() {
 		try {
+			final JobID jobId = new JobID();
+			
+			DefaultScheduler scheduler = new DefaultScheduler();
+
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			
+			ResourceId id1 = new ResourceId();
+			ResourceId id2 = new ResourceId();
+			ResourceId id3 = new ResourceId();
+			ResourceId id4 = new ResourceId();
+			ResourceId id5 = new ResourceId();
+			ResourceId id6 = new ResourceId();
+			
+			AllocatedSlot s1 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id1), true);
+			AllocatedSlot s2 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id2), true);
+			AllocatedSlot s3 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id3), true);
+			AllocatedSlot s4 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id4), true);
+			AllocatedSlot s5 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id5), true);
+			AllocatedSlot s6 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id6), true);
+			
+			// no more slots available, the next call should throw an exception
 			try {
-				scheduler.scheduleJob(executionGraph);
-			} catch (SchedulingException e) {
-				fail(StringUtils.stringifyException(e));
-			}
-
-			// Wait for the deployment to complete
-			tdm.waitForDeployment();
-
-			assertEquals(executionGraph.getJobID(), tdm.getIDOfLastDeployedJob());
-			final List<ExecutionVertex> listOfDeployedVertices = tdm.getListOfLastDeployedVertices();
-			assertNotNull(listOfDeployedVertices);
-			// Vertices connected via in-memory channels must be deployed in a single cycle.
-			assertEquals(2, listOfDeployedVertices.size());
-
-			// Check if the release of the allocated resources works properly by simulating the vertices' life cycle
-			assertEquals(0, tim.getNumberOfReleaseMethodCalls());
-
-			// Simulate vertex life cycle
-			for (final ExecutionVertex vertex : listOfDeployedVertices) {
-				vertex.updateExecutionState(ExecutionState.STARTING);
-				vertex.updateExecutionState(ExecutionState.RUNNING);
-				vertex.updateExecutionState(ExecutionState.FINISHING);
-				vertex.updateExecutionState(ExecutionState.FINISHED);
+				scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), new ResourceId()), true);
+				fail("Scheduler accepted scheduling request without available resource.");
 			}
-
-			assertEquals(1, tim.getNumberOfReleaseMethodCalls());
-		} finally {
-			try {
-				LibraryCacheManager.unregister(executionGraph.getJobID());
-			} catch (IOException ioe) {
-				// Ignore exception here
+			catch (NoResourceAvailableException e) {
+				// expected
 			}
+			
+			// schedule something into the same slots as before
+			AllocatedSlot s1s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id1), true);
+			AllocatedSlot s2s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id2), true);
+			AllocatedSlot s3s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id3), true);
+			AllocatedSlot s4s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id4), true);
+			AllocatedSlot s5s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id5), true);
+			AllocatedSlot s6s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id6), true);
+			
+			assertEquals(s1, s1s);
+			assertEquals(s2, s2s);
+			assertEquals(s3, s3s);
+			assertEquals(s4, s4s);
+			assertEquals(s5, s5s);
+			assertEquals(s6, s6s);
 		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private Instance getRandomInstance(int numSlots) {
+		InetAddress address;
+		try {
+			address = InetAddress.getByName("127.0.0.1");
+		} catch (UnknownHostException e) {
+			throw new RuntimeException("Test could not create IP address for localhost loopback.");
+		}
+		
+		int ipcPort = portNum++;
+		int dataPort = portNum++;
+		
+		InstanceConnectionInfo ci = new InstanceConnectionInfo(address, ipcPort, dataPort);
+		
+		final long GB = 1024L*1024*1024;
+		HardwareDescription resources = new HardwareDescription(4, 4*GB, 3*GB, 2*GB);
+		
+		return new Instance(ci, new InstanceID(), resources, numSlots);
+	}
+	
+	private ExecutionVertex2 getDummyVertex() {
+		return new ExecutionVertex2();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueueTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueueTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueueTest.java
new file mode 100644
index 0000000..4a2a0cb
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueueTest.java
@@ -0,0 +1,128 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+/**
+ * Test for the basic functionality of the {@link LifoSetQueue}.
+ */
+public class LifoSetQueueTest {
+
+	@Test
+	public void testSizeAddPollAndPeek() {
+		try {
+			LifoSetQueue<Integer> queue = new LifoSetQueue<Integer>();
+			
+			// empty queue
+			assertEquals(0, queue.size());
+			assertNull(queue.poll());
+			assertNull(queue.peek());
+			
+			// add some elements
+			assertTrue(queue.add(1));
+			assertTrue(queue.offer(2));
+			assertTrue(queue.offer(3));
+			assertEquals(3, queue.size());
+			
+			assertEquals(3, queue.peek().intValue());
+			
+			// prevent duplicates. note that the methods return true, because no capacity constraint is violated
+			assertTrue(queue.add(1));
+			assertTrue(queue.offer(1));
+			assertTrue(queue.add(3));
+			assertTrue(queue.offer(3));
+			assertTrue(queue.add(2));
+			assertTrue(queue.offer(2));
+			assertEquals(3, queue.size());
+			
+			// peek and poll some elements
+			assertEquals(3, queue.peek().intValue());
+			assertEquals(3, queue.size());
+			assertEquals(3, queue.poll().intValue());
+			assertEquals(2, queue.size());
+			assertEquals(2, queue.peek().intValue());
+			assertEquals(2, queue.size());
+			assertEquals(2, queue.poll().intValue());
+			assertEquals(1, queue.size());
+			assertEquals(1, queue.peek().intValue());
+			assertEquals(1, queue.size());
+			assertEquals(1, queue.poll().intValue());
+			assertEquals(0, queue.size());
+			assertTrue(queue.isEmpty());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getClass().getSimpleName() + " : " + e.getMessage());
+		}
+	}
+	
+	/**
+	 * Remove is tricky, because it goes through the iterator and calls remove() on the iterator.
+	 */
+	@Test
+	public void testRemove() {
+		try {
+			LifoSetQueue<String> queue = new LifoSetQueue<String>();
+			queue.add("1");
+			queue.add("2");
+			queue.add("3");
+			queue.add("4");
+			queue.add("5");
+			queue.add("6");
+			queue.add("7");
+			
+			assertEquals(7, queue.size());
+			assertEquals("7", queue.peek());
+			
+			// remove non-existing
+			assertFalse(queue.remove("8"));
+			
+			// remove the last
+			assertTrue(queue.remove("7"));
+			// remove the first
+			assertTrue(queue.remove("1"));
+			// remove in the middle
+			assertTrue(queue.remove("3"));
+			
+			assertEquals(4, queue.size());
+			
+			// check that we can re-add the removed elements
+			assertTrue(queue.add("1"));
+			assertTrue(queue.add("7"));
+			assertTrue(queue.add("3"));
+			assertEquals(7, queue.size());
+			
+			// check the order
+			assertEquals("3", queue.poll());
+			assertEquals("7", queue.poll());
+			assertEquals("1", queue.poll());
+			assertEquals("6", queue.poll());
+			assertEquals("5", queue.poll());
+			assertEquals("4", queue.poll());
+			assertEquals("2", queue.poll());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getClass().getSimpleName() + " : " + e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestDeploymentManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestDeploymentManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestDeploymentManager.java
deleted file mode 100644
index 81a7d60..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestDeploymentManager.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/**
- * 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.jobmanager.scheduler;
-
-import java.util.List;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobmanager.DeploymentManager;
-
-/**
- * This class provides an implementation of the {@DeploymentManager} interface which is used during
- * the unit tests.
- * <p>
- * This class is thread-safe.
- * 
- */
-public class TestDeploymentManager implements DeploymentManager {
-
-	/**
-	 * The ID of the job to be deployed.
-	 */
-	private volatile JobID jobID = null;
-
-	/**
-	 * The list of vertices to be deployed.
-	 */
-	private volatile List<ExecutionVertex> verticesToBeDeployed = null;
-
-	/**
-	 * Auxiliary object to synchronize on.
-	 */
-	private final Object synchronizationObject = new Object();
-
-
-	@Override
-	public void deploy(final JobID jobID, final Instance instance,
-			final List<ExecutionVertex> verticesToBeDeployed) {
-
-		this.jobID = jobID;
-		this.verticesToBeDeployed = verticesToBeDeployed;
-
-		synchronized (this.synchronizationObject) {
-			this.synchronizationObject.notify();
-		}
-	}
-
-	/**
-	 * Returns the ID of the last deployed job.
-	 */
-	JobID getIDOfLastDeployedJob() {
-
-		return this.jobID;
-	}
-
-	/**
-	 * Returns a list of the last deployed vertices.
-	 * 
-	 * @return a list of the last deployed vertices
-	 */
-	List<ExecutionVertex> getListOfLastDeployedVertices() {
-
-		return this.verticesToBeDeployed;
-	}
-
-	/**
-	 * Clears the internal state of the test deployment manager.
-	 */
-	void clear() {
-
-		this.jobID = null;
-		this.verticesToBeDeployed = null;
-	}
-
-	/**
-	 * Wait for the scheduler to complete the deployment.
-	 */
-	void waitForDeployment() {
-
-		while (this.jobID == null) {
-			synchronized (this.synchronizationObject) {
-				try {
-					this.synchronizationObject.wait(50);
-				} catch (InterruptedException e) {
-					// Ignore exception
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestInstanceManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestInstanceManager.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestInstanceManager.java
deleted file mode 100644
index 9286def..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/TestInstanceManager.java
+++ /dev/null
@@ -1,194 +0,0 @@
-/**
- * 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.jobmanager.scheduler;
-
-import java.net.Inet4Address;
-import java.net.UnknownHostException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.instance.*;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.topology.NetworkNode;
-import org.apache.flink.runtime.topology.NetworkTopology;
-import org.apache.flink.util.StringUtils;
-
-/**
- * A dummy implementation of an {@link org.apache.flink.runtime.instance.InstanceManager}.
- */
-public final class TestInstanceManager implements InstanceManager {
-
-	/**
-	 * Counts the number of times the method releaseAllocatedResource is called.
-	 */
-	private volatile int numberOfReleaseCalls = 0;
-
-	/**
-	 * The instance listener.
-	 */
-	private volatile InstanceListener instanceListener = null;
-
-	/**
-	 * The list of resources allocated to a job.
-	 */
-	private final List<AllocatedResource> allocatedResources;
-
-	/**
-	 * The test instance
-	 */
-	private final TestInstance testInstance;
-
-	/**
-	 * Test implementation of {@link org.apache.flink.runtime.instance.Instance}.
-	 * 
-	 */
-	private static final class TestInstance extends Instance {
-
-		/**
-		 * Constructs a new test instance.
-		 * 
-		 * @param instanceConnectionInfo
-		 *        the instance connection information
-		 * @param parentNode
-		 *        the parent node in the network topology
-		 * @param networkTopology
-		 *        the network topology
-		 * @param hardwareDescription
-		 *        the hardware description
-		 * @param numberSlots
-		 * 		  the number of slots available on the instance
-		 */
-		public TestInstance(final InstanceConnectionInfo instanceConnectionInfo,
-				final NetworkNode parentNode, final NetworkTopology networkTopology,
-				final HardwareDescription hardwareDescription, int numberSlots) {
-			super(instanceConnectionInfo, parentNode, networkTopology, hardwareDescription, numberSlots);
-		}
-	}
-
-	/**
-	 * Constructs a new test instance manager
-	 */
-	public TestInstanceManager() {
-
-		final HardwareDescription hd = HardwareDescriptionFactory.construct(1, 1L, 1L);
-
-		this.allocatedResources = new ArrayList<AllocatedResource>();
-		try {
-			final InstanceConnectionInfo ici = new InstanceConnectionInfo(Inet4Address.getLocalHost(), 1, 1);
-			final NetworkTopology nt = new NetworkTopology();
-			this.testInstance = new TestInstance(ici, nt.getRootNode(), nt, hd, 1);
-			this.allocatedResources.add(new AllocatedResource(testInstance, new AllocationID()));
-		} catch (UnknownHostException e) {
-			throw new RuntimeException(StringUtils.stringifyException(e));
-		}
-	}
-
-
-	@Override
-	public void requestInstance(final JobID jobID, final Configuration conf,
-								int requiredSlots) throws InstanceException {
-
-		if (this.instanceListener == null) {
-			throw new InstanceException("instanceListener not registered with TestInstanceManager");
-		}
-
-		final InstanceListener il = this.instanceListener;
-
-		final Runnable runnable = new Runnable() {
-
-			/**
-			 * {@inheritDoc}
-			 */
-			@Override
-			public void run() {
-				il.resourcesAllocated(jobID, allocatedResources);
-			}
-		};
-
-		new Thread(runnable).start();
-	}
-
-
-	@Override
-	public void releaseAllocatedResource(final AllocatedResource allocatedResource) throws InstanceException {
-		++this.numberOfReleaseCalls;
-	}
-
-	/**
-	 * Returns the number of times the method releaseAllocatedResource has been called.
-	 * 
-	 * @return the number of times the method releaseAllocatedResource has been called
-	 */
-	int getNumberOfReleaseMethodCalls() {
-		return this.numberOfReleaseCalls;
-	}
-
-
-	@Override
-	public void reportHeartBeat(final InstanceConnectionInfo instanceConnectionInfo) {
-		throw new IllegalStateException("reportHeartBeat called on TestInstanceManager");
-	}
-
-	@Override
-	public void registerTaskManager(final InstanceConnectionInfo instanceConnectionInfo,
-									final HardwareDescription hardwareDescription, int numberSlots){
-		throw new IllegalStateException("registerTaskManager called on TestInstanceManager.");
-	}
-
-	@Override
-	public NetworkTopology getNetworkTopology(final JobID jobID) {
-		throw new IllegalStateException("getNetworkTopology called on TestInstanceManager");
-	}
-
-
-	@Override
-	public void setInstanceListener(final InstanceListener instanceListener) {
-
-		this.instanceListener = instanceListener;
-	}
-
-	@Override
-	public Instance getInstanceByName(final String name) {
-		throw new IllegalStateException("getInstanceByName called on TestInstanceManager");
-	}
-
-	@Override
-	public void shutdown() {
-		throw new IllegalStateException("shutdown called on TestInstanceManager");
-	}
-
-	@Override
-	public int getNumberOfTaskManagers() {
-		throw new IllegalStateException("getNumberOfTaskTrackers called on TestInstanceManager");
-	}
-
-	@Override
-	public int getNumberOfSlots() {
-		return this.testInstance.getNumberOfSlots();
-	}
-
-
-	@Override
-	public Map<InstanceConnectionInfo, Instance> getInstances() {
-		throw new IllegalStateException("getInstances called on TestInstanceManager");
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
index 5b76d53..2b2c81d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
@@ -1,17 +1,20 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.jobmanager.splitassigner;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
index ddad0d3..fdc1cf4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
@@ -1,17 +1,20 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.jobmanager.splitassigner;
 


[15/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
index 82600d2..2d00f40 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptor.java
@@ -16,81 +16,63 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.deployment;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.util.SerializableArrayList;
-import org.apache.flink.util.StringUtils;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.types.StringValue;
 
 /**
  * A task deployment descriptor contains all the information necessary to deploy a task on a task manager.
- * <p>
- * This class is not thread-safe in general.
- * 
  */
 public final class TaskDeploymentDescriptor implements IOReadableWritable {
 
-	/**
-	 * The ID of the job the tasks belongs to.
-	 */
+	/** The ID of the job the tasks belongs to. */
 	private final JobID jobID;
 
-	/**
-	 * The task's execution vertex ID.
-	 */
-	private final ExecutionVertexID vertexID;
+	/** The task's job vertex ID. */
+	private final JobVertexID vertexID;
+	
+	/** The ID referencing the attempt to execute the task. */
+	private final ExecutionAttemptID executionId;
 
-	/**
-	 * The task's name.
-	 */
+	/** The task's name. */
 	private String taskName;
 
-	/**
-	 * The task's index in the subtask group.
-	 */
+	/** The task's index in the subtask group. */
 	private int indexInSubtaskGroup;
 
-	/**
-	 * The current number of subtasks.
-	 */
+	/** The current number of subtasks. */
 	private int currentNumberOfSubtasks;
 
-	/**
-	 * The configuration of the job the task belongs to.
-	 */
+	/** The configuration of the job the task belongs to. */
 	private Configuration jobConfiguration;
 
-	/**
-	 * The task's configuration object.
-	 */
+	/** The task's configuration object. */
 	private Configuration taskConfiguration;
 
+	/** The name of the class containing the task code to be executed. */
+	private String invokableClassName;
 
-	/**
-	 * The class containing the task code to be executed.
-	 */
-	private Class<? extends AbstractInvokable> invokableClass;
+	/** The list of output gate deployment descriptors. */
+	private List<GateDeploymentDescriptor> outputGates;
 
-	/**
-	 * The list of output gate deployment descriptors.
-	 */
-	private final SerializableArrayList<GateDeploymentDescriptor> outputGates;
-
-	/**
-	 * The list of input gate deployment descriptors.
-	 */
-	private final SerializableArrayList<GateDeploymentDescriptor> inputGates;
+	/** The list of input gate deployment descriptors. */
+	private List<GateDeploymentDescriptor> inputGates;
+	
+	private String[] requiredJarFiles;
+	
+	private int targetSlotNumber;
 
 	/**
 	 * Constructs a task deployment descriptor.
@@ -109,167 +91,58 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 *        the configuration of the job the task belongs to
 	 * @param taskConfiguration
 	 *        the task's configuration object
-	 * @param invokableClass
+	 * @param invokableClassName
 	 *        the class containing the task code to be executed
 	 * @param outputGates
 	 *        list of output gate deployment descriptors
-	 * @param inputGateIDs
-	 *        list of input gate deployment descriptors
 	 */
-	public TaskDeploymentDescriptor(final JobID jobID, final ExecutionVertexID vertexID, final String taskName,
-			final int indexInSubtaskGroup, final int currentNumberOfSubtasks, final Configuration jobConfiguration,
-			final Configuration taskConfiguration, 
-			final Class<? extends AbstractInvokable> invokableClass,
-			final SerializableArrayList<GateDeploymentDescriptor> outputGates,
-			final SerializableArrayList<GateDeploymentDescriptor> inputGates) {
-
-		if (jobID == null) {
-			throw new IllegalArgumentException("Argument jobID must not be null");
-		}
-
-		if (vertexID == null) {
-			throw new IllegalArgumentException("Argument vertexID must not be null");
-		}
-
-		if (taskName == null) {
-			throw new IllegalArgumentException("Argument taskName must not be null");
-		}
-
-		if (indexInSubtaskGroup < 0) {
-			throw new IllegalArgumentException("Argument indexInSubtaskGroup must not be smaller than zero");
+	public TaskDeploymentDescriptor(JobID jobID, JobVertexID vertexID, ExecutionAttemptID execuionId,
+			String taskName, int indexInSubtaskGroup, int currentNumberOfSubtasks, 
+			Configuration jobConfiguration, Configuration taskConfiguration,
+			String invokableClassName,
+			List<GateDeploymentDescriptor> outputGates,
+			List<GateDeploymentDescriptor> inputGates,
+			String[] requiredJarFiles,
+			int targetSlotNumber)
+	{
+		if (jobID == null || vertexID == null || execuionId == null || taskName == null || indexInSubtaskGroup < 0 ||
+				currentNumberOfSubtasks <= indexInSubtaskGroup || jobConfiguration == null ||
+				taskConfiguration == null || invokableClassName == null || outputGates == null || inputGates == null)
+		{
+			throw new IllegalArgumentException();
 		}
-
-		if (currentNumberOfSubtasks < indexInSubtaskGroup) {
-			throw new IllegalArgumentException(
-				"Argument currentNumberOfSubtasks must not be smaller than argument indexInSubtaskGroup");
-		}
-
-		if (jobConfiguration == null) {
-			throw new IllegalArgumentException("Argument jobConfiguration must not be null");
-		}
-
-		if (taskConfiguration == null) {
-			throw new IllegalArgumentException("Argument taskConfiguration must not be null");
-		}
-
-		if (invokableClass == null) {
-			throw new IllegalArgumentException("Argument invokableClass must not be null");
-		}
-
-		if (outputGates == null) {
-			throw new IllegalArgumentException("Argument outputGates must not be null");
-		}
-
-		if (inputGates == null) {
-			throw new IllegalArgumentException("Argument inputGates must not be null");
-		}
-
+		
 		this.jobID = jobID;
 		this.vertexID = vertexID;
+		this.executionId = execuionId;
 		this.taskName = taskName;
 		this.indexInSubtaskGroup = indexInSubtaskGroup;
 		this.currentNumberOfSubtasks = currentNumberOfSubtasks;
 		this.jobConfiguration = jobConfiguration;
 		this.taskConfiguration = taskConfiguration;
-		this.invokableClass = invokableClass;
+		this.invokableClassName = invokableClassName;
 		this.outputGates = outputGates;
 		this.inputGates = inputGates;
+		this.requiredJarFiles = requiredJarFiles == null ? new String[0] : requiredJarFiles;
+		this.targetSlotNumber = targetSlotNumber;
 	}
 
 	/**
 	 * Default constructor for serialization/deserialization.
 	 */
 	public TaskDeploymentDescriptor() {
-
 		this.jobID = new JobID();
-		this.vertexID = new ExecutionVertexID();
-		this.taskName = null;
-		this.indexInSubtaskGroup = 0;
-		this.currentNumberOfSubtasks = 0;
+		this.vertexID = new JobVertexID();
+		this.executionId = new ExecutionAttemptID();
 		this.jobConfiguration = new Configuration();
 		this.taskConfiguration = new Configuration();
-		this.invokableClass = null;
-		this.outputGates = new SerializableArrayList<GateDeploymentDescriptor>();
-		this.inputGates = new SerializableArrayList<GateDeploymentDescriptor>();
-	}
-
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-
-		this.jobID.write(out);
-		this.vertexID.write(out);
-		StringRecord.writeString(out, this.taskName);
-		out.writeInt(this.indexInSubtaskGroup);
-		out.writeInt(this.currentNumberOfSubtasks);
-
-		// Write out the names of the required jar files
-		final String[] requiredJarFiles = LibraryCacheManager.getRequiredJarFiles(this.jobID);
-
-		out.writeInt(requiredJarFiles.length);
-		for (int i = 0; i < requiredJarFiles.length; i++) {
-			StringRecord.writeString(out, requiredJarFiles[i]);
-		}
-
-		// Write out the name of the invokable class
-		if (this.invokableClass == null) {
-			throw new IOException("this.invokableClass is null");
-		}
-
-		StringRecord.writeString(out, this.invokableClass.getName());
-
-		this.jobConfiguration.write(out);
-		this.taskConfiguration.write(out);
-
-		this.outputGates.write(out);
-		this.inputGates.write(out);
+		this.outputGates = Collections.emptyList();
+		this.inputGates = Collections.emptyList();
+		this.requiredJarFiles = new String[0];
 	}
 
 
-	@SuppressWarnings("unchecked")
-	@Override
-	public void read(final DataInputView in) throws IOException {
-
-		this.jobID.read(in);
-		this.vertexID.read(in);
-		this.taskName = StringRecord.readString(in);
-		this.indexInSubtaskGroup = in.readInt();
-		this.currentNumberOfSubtasks = in.readInt();
-
-		// Read names of required jar files
-		final String[] requiredJarFiles = new String[in.readInt()];
-		for (int i = 0; i < requiredJarFiles.length; i++) {
-			requiredJarFiles[i] = StringRecord.readString(in);
-		}
-
-		// Now register data with the library manager
-		LibraryCacheManager.register(this.jobID, requiredJarFiles);
-
-		// Get ClassLoader from Library Manager
-		final ClassLoader cl = LibraryCacheManager.getClassLoader(this.jobID);
-
-		// Read the name of the invokable class;
-		final String invokableClassName = StringRecord.readString(in);
-
-		if (invokableClassName == null) {
-			throw new IOException("invokableClassName is null");
-		}
-
-		try {
-			this.invokableClass = (Class<? extends AbstractInvokable>) Class.forName(invokableClassName, true, cl);
-		} catch (ClassNotFoundException cnfe) {
-			throw new IOException("Class " + invokableClassName + " not found in one of the supplied jar files: "
-				+ StringUtils.stringifyException(cnfe));
-		}
-
-		this.jobConfiguration = new Configuration(cl);
-		this.jobConfiguration.read(in);
-		this.taskConfiguration = new Configuration(cl);
-		this.taskConfiguration.read(in);
 
-		this.outputGates.read(in);
-		this.inputGates.read(in);
-	}
 
 	/**
 	 * Returns the ID of the job the tasks belongs to.
@@ -277,7 +150,6 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 * @return the ID of the job the tasks belongs to
 	 */
 	public JobID getJobID() {
-
 		return this.jobID;
 	}
 
@@ -286,10 +158,13 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 * 
 	 * @return the task's execution vertex ID
 	 */
-	public ExecutionVertexID getVertexID() {
-
+	public JobVertexID getVertexID() {
 		return this.vertexID;
 	}
+	
+	public ExecutionAttemptID getExecutionId() {
+		return executionId;
+	}
 
 	/**
 	 * Returns the task's name.
@@ -297,7 +172,6 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 * @return the task's name
 	 */
 	public String getTaskName() {
-
 		return this.taskName;
 	}
 
@@ -307,7 +181,6 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 * @return the task's index in the subtask group
 	 */
 	public int getIndexInSubtaskGroup() {
-
 		return this.indexInSubtaskGroup;
 	}
 
@@ -317,7 +190,6 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 * @return the current number of subtasks
 	 */
 	public int getCurrentNumberOfSubtasks() {
-
 		return this.currentNumberOfSubtasks;
 	}
 
@@ -327,7 +199,6 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 * @return the configuration of the job the tasks belongs to
 	 */
 	public Configuration getJobConfiguration() {
-
 		return this.jobConfiguration;
 	}
 
@@ -337,61 +208,97 @@ public final class TaskDeploymentDescriptor implements IOReadableWritable {
 	 * @return the task's configuration object
 	 */
 	public Configuration getTaskConfiguration() {
-
 		return this.taskConfiguration;
 	}
 
 	/**
-	 * Returns the class containing the task code to be executed.
+	 * Returns the name of the class containing the task code to be executed.
 	 * 
-	 * @return the class containing the task code to be executed
+	 * @return The name of the class containing the task code to be executed
 	 */
-	public Class<? extends AbstractInvokable> getInvokableClass() {
-
-		return this.invokableClass;
+	public String getInvokableClassName() {
+		return this.invokableClassName;
 	}
 
-	/**
-	 * Returns the number of output gate deployment descriptors contained in this task deployment descriptor.
-	 * 
-	 * @return the number of output gate deployment descriptors
-	 */
-	public int getNumberOfOutputGateDescriptors() {
-
-		return this.outputGates.size();
+	public List<GateDeploymentDescriptor> getOutputGates() {
+		return outputGates;
 	}
-
-	/**
-	 * Returns the output gate descriptor with the given index
-	 * 
-	 * @param index
-	 *        the index if the output gate descriptor to return
-	 * @return the output gate descriptor with the given index
-	 */
-	public GateDeploymentDescriptor getOutputGateDescriptor(final int index) {
-
-		return this.outputGates.get(index);
+	
+	public List<GateDeploymentDescriptor> getInputGates() {
+		return inputGates;
 	}
-
-	/**
-	 * Returns the number of output gate deployment descriptors contained in this task deployment descriptor.
-	 * 
-	 * @return the number of output gate deployment descriptors
-	 */
-	public int getNumberOfInputGateDescriptors() {
-
-		return this.inputGates.size();
+	
+	// --------------------------------------------------------------------------------------------
+	//  Serialization
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public void write(final DataOutputView out) throws IOException {
+		jobID.write(out);
+		vertexID.write(out);
+		executionId.write(out);
+		
+		StringValue.writeString(taskName, out);
+		StringValue.writeString(invokableClassName, out);
+		
+		out.writeInt(indexInSubtaskGroup);
+		out.writeInt(currentNumberOfSubtasks);
+		out.writeInt(targetSlotNumber);
+
+		jobConfiguration.write(out);
+		taskConfiguration.write(out);
+
+		writeGateList(inputGates, out);
+		writeGateList(outputGates, out);
+				
+		out.writeInt(requiredJarFiles.length);
+		for (int i = 0; i < requiredJarFiles.length; i++) {
+			StringValue.writeString(requiredJarFiles[i], out);
+		}
 	}
 
-	/**
-	 * Returns the input gate descriptor with the given index
-	 * 
-	 * @param index
-	 *        the index if the input gate descriptor to return
-	 * @return the input gate descriptor with the given index
-	 */
-	public GateDeploymentDescriptor getInputGateDescriptor(final int index) {
-
-		return this.inputGates.get(index);
+	@Override
+	public void read(DataInputView in) throws IOException {
+		jobID.read(in);
+		vertexID.read(in);
+		executionId.read(in);
+		
+		taskName = StringValue.readString(in);
+		invokableClassName = StringValue.readString(in);
+		
+		indexInSubtaskGroup = in.readInt();
+		currentNumberOfSubtasks = in.readInt();
+		targetSlotNumber = in.readInt();
+		
+		jobConfiguration.read(in);
+		taskConfiguration.read(in);
+
+		inputGates = readGateList(in);
+		outputGates = readGateList(in);
+
+		String[] jarFiles = new String[in.readInt()];
+		for (int i = 0; i < jarFiles.length; i++) {
+			jarFiles[i] = StringValue.readString(in);
+		}
+	}
+	
+	private static final void writeGateList(List<GateDeploymentDescriptor> list, DataOutputView out) throws IOException {
+		out.writeInt(list.size());
+		for (GateDeploymentDescriptor gdd : list) {
+			gdd.write(out);
+		}
+	}
+	
+	private static final List<GateDeploymentDescriptor> readGateList(DataInputView in) throws IOException {
+		final int len = in.readInt();
+		ArrayList<GateDeploymentDescriptor> list = new ArrayList<GateDeploymentDescriptor>(len);
+		
+		for (int i = 0; i < len; i++) {
+			GateDeploymentDescriptor gdd = new GateDeploymentDescriptor();
+			gdd.read(in);
+			list.add(gdd);
+		}
+		
+		return list;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java
index 61970da..d7e4712 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/AbstractEvent.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.job;
 
 import java.io.IOException;
@@ -30,29 +29,20 @@ import org.apache.flink.core.memory.DataOutputView;
 
 /**
  * An abstract event is transmitted from the job manager to the
- * job client in order to inform the user about the job progress.
- * 
+ * job client in order to inform the user about the job progress
  */
 public abstract class AbstractEvent implements IOReadableWritable {
 
-	/**
-	 * Static variable that points to the current global sequence number
-	 */
+	/** Static variable that points to the current global sequence number */
 	private static final AtomicLong GLOBAL_SEQUENCE_NUMBER = new AtomicLong(0);
 
-	/**
-	 * Auxiliary object which helps to convert a {@link Date} object to the given string representation.
-	 */
+	/** Auxiliary object which helps to convert a {@link Date} object to the given string representation. */
 	private static final SimpleDateFormat DATA_FORMATTER = new SimpleDateFormat("MM/dd/yyyy HH:mm:ss");
 
-	/**
-	 * The timestamp of the event.
-	 */
+	/** The timestamp of the event. */
 	private long timestamp = -1;
 
-	/**
-	 * The sequence number of the event.
-	 */
+	/** The sequence number of the event. */
 	private long sequenceNumber = -1;
 
 	/**
@@ -61,7 +51,7 @@ public abstract class AbstractEvent implements IOReadableWritable {
 	 * @param timestamp
 	 *        the timestamp of the event.
 	 */
-	public AbstractEvent(final long timestamp) {
+	public AbstractEvent(long timestamp) {
 		this.timestamp = timestamp;
 		this.sequenceNumber = GLOBAL_SEQUENCE_NUMBER.incrementAndGet();
 	}
@@ -75,23 +65,17 @@ public abstract class AbstractEvent implements IOReadableWritable {
 	 * is required for the deserialization process and is not
 	 * supposed to be called directly.
 	 */
-	public AbstractEvent() {
-	}
+	public AbstractEvent() {}
 
 
 	@Override
-	public void read(final DataInputView in) throws IOException {
-
-		// Read the timestamp
+	public void read(DataInputView in) throws IOException {
 		this.timestamp = in.readLong();
 		this.sequenceNumber = in.readLong();
 	}
 
-
 	@Override
-	public void write(final DataOutputView out) throws IOException {
-
-		// Write the timestamp
+	public void write(DataOutputView out) throws IOException {
 		out.writeLong(this.timestamp);
 		out.writeLong(this.sequenceNumber);
 	}
@@ -113,18 +97,14 @@ public abstract class AbstractEvent implements IOReadableWritable {
 	 *        the timestamp in milliseconds since the beginning of "the epoch"
 	 * @return the string unified representation of the timestamp
 	 */
-	public static String timestampToString(final long timestamp) {
-
+	public static String timestampToString(long timestamp) {
 		return DATA_FORMATTER.format(new Date(timestamp));
-
 	}
 
 
 	@Override
 	public boolean equals(final Object obj) {
-
 		if (obj instanceof AbstractEvent) {
-
 			final AbstractEvent abstractEvent = (AbstractEvent) obj;
 			if (this.timestamp == abstractEvent.getTimestamp()) {
 				return true;
@@ -137,7 +117,6 @@ public abstract class AbstractEvent implements IOReadableWritable {
 
 	@Override
 	public int hashCode() {
-
-		return (int) (this.timestamp % Integer.MAX_VALUE);
+		return (int) (this.timestamp ^ (this.timestamp >>> 32));
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
index bb55eac..15bae60 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ExecutionStateChangeEvent.java
@@ -23,13 +23,11 @@ import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.ExecutionState2;
 import org.apache.flink.runtime.managementgraph.ManagementVertexID;
-import org.apache.flink.runtime.util.EnumUtils;
 
 /**
- * An {@link ExecutionStateChangeEvent} can be used to notify other objects about an execution state change of a vertex.
- * 
+ * An {@link ExecutionStateChangeEvent} can be used to notify other objects about an execution state change of a vertex. 
  */
 public final class ExecutionStateChangeEvent extends AbstractEvent implements ManagementEvent {
 
@@ -41,7 +39,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 	/**
 	 * The new execution state of the vertex this event refers to.
 	 */
-	private ExecutionState newExecutionState;
+	private ExecutionState2 newExecutionState;
 
 	/**
 	 * Constructs a new vertex event object.
@@ -53,8 +51,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 	 * @param newExecutionState
 	 *        the new execution state of the vertex this event refers to
 	 */
-	public ExecutionStateChangeEvent(final long timestamp, final ManagementVertexID managementVertexID,
-			final ExecutionState newExecutionState) {
+	public ExecutionStateChangeEvent(long timestamp, ManagementVertexID managementVertexID, ExecutionState2 newExecutionState) {
 		super(timestamp);
 		this.managementVertexID = managementVertexID;
 		this.newExecutionState = newExecutionState;
@@ -69,7 +66,7 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 		super();
 
 		this.managementVertexID = new ManagementVertexID();
-		this.newExecutionState = ExecutionState.CREATED;
+		this.newExecutionState = ExecutionState2.CREATED;
 	}
 
 	/**
@@ -86,28 +83,24 @@ public final class ExecutionStateChangeEvent extends AbstractEvent implements Ma
 	 * 
 	 * @return the new execution state of the vertex this event refers to
 	 */
-	public ExecutionState getNewExecutionState() {
+	public ExecutionState2 getNewExecutionState() {
 		return this.newExecutionState;
 	}
 
 
 	@Override
-	public void read(final DataInputView in) throws IOException {
-
+	public void read(DataInputView in) throws IOException {
 		super.read(in);
-
 		this.managementVertexID.read(in);
-		this.newExecutionState = EnumUtils.readEnum(in, ExecutionState.class);
+		this.newExecutionState = ExecutionState2.values()[in.readInt()];
 	}
 
 
 	@Override
-	public void write(final DataOutputView out) throws IOException {
-
+	public void write(DataOutputView out) throws IOException {
 		super.write(out);
-
 		this.managementVertexID.write(out);
-		EnumUtils.writeEnum(out, this.newExecutionState);
+		out.writeInt(this.newExecutionState.ordinal());
 	}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java
index 210fb54..2f32686 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/JobEvent.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.job;
 
 import java.io.IOException;
@@ -30,18 +29,13 @@ import org.apache.flink.runtime.util.EnumUtils;
 /**
  * A job event object is used by the job manager to inform a client about
  * changes of the job's status.
- * 
  */
 public class JobEvent extends AbstractEvent {
 
-	/**
-	 * The current status of the job.
-	 */
+	/** The current status of the job. */
 	private JobStatus currentJobStatus;
 
-	/**
-	 * An optional message attached to the event, possibly <code>null</code>.
-	 */
+	/** An optional message attached to the event, possibly <code>null</code>. */
 	private String optionalMessage = null;
 
 	/**
@@ -68,8 +62,7 @@ public class JobEvent extends AbstractEvent {
 	 */
 	public JobEvent() {
 		super();
-
-		this.currentJobStatus = JobStatus.SCHEDULED;
+		this.currentJobStatus = JobStatus.CREATED;
 	}
 
 
@@ -111,20 +104,17 @@ public class JobEvent extends AbstractEvent {
 	 * @return the optional message, possibly <code>null</code>.
 	 */
 	public String getOptionalMessage() {
-
 		return this.optionalMessage;
 	}
 
 
 	public String toString() {
-
 		return timestampToString(getTimestamp()) + ":\tJob execution switched to status " + this.currentJobStatus;
 	}
 
 
 	@Override
 	public boolean equals(final Object obj) {
-
 		if (!super.equals(obj)) {
 			return false;
 		}
@@ -154,7 +144,6 @@ public class JobEvent extends AbstractEvent {
 
 	@Override
 	public int hashCode() {
-
 		return super.hashCode();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ManagementEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ManagementEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ManagementEvent.java
index 4b33f47..695ae5d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ManagementEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/ManagementEvent.java
@@ -16,14 +16,10 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.job;
 
 /**
  * This is a marker interface only. It marks events which
- * should only be accessible via the {@link ExtendedManagementProtocol}.
- * 
+ * should only be accessible via the {@link org.apache.flink.runtime.protocols.ExtendedManagementProtocol}.
  */
-public interface ManagementEvent {
-
-}
+public interface ManagementEvent {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexAssignmentEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexAssignmentEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexAssignmentEvent.java
deleted file mode 100644
index 0e048dd..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexAssignmentEvent.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/**
- * 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.event.job;
-
-import java.io.IOException;
-
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.managementgraph.ManagementVertexID;
-
-/**
- * A {@link VertexAssignmentEvent} can be used to notify other objects about changes in the assignment of vertices to
- * instances.
- * 
- */
-public final class VertexAssignmentEvent extends AbstractEvent implements ManagementEvent {
-
-	/**
-	 * The ID identifies the vertex this events refers to.
-	 */
-	private ManagementVertexID managementVertexID;
-
-	/**
-	 * The name of the instance the vertex is now assigned to.
-	 */
-	private String instanceName;
-
-	/**
-	 * Constructs a new event.
-	 * 
-	 * @param timestamp
-	 *        the timestamp of the event
-	 * @param managementVertexID
-	 *        identifies the vertex this event refers to
-	 * @param instanceName
-	 *        the name of the instance the vertex is now assigned to
-	 */
-	public VertexAssignmentEvent(final long timestamp, final ManagementVertexID managementVertexID,
-			final String instanceName) {
-		super(timestamp);
-
-		this.managementVertexID = managementVertexID;
-		this.instanceName = instanceName;
-	}
-
-	/**
-	 * Constructor for serialization/deserialization. Should not be called on other occasions.
-	 */
-	public VertexAssignmentEvent() {
-		super();
-
-		this.managementVertexID = new ManagementVertexID();
-	}
-
-	/**
-	 * Returns the ID of the vertex this event refers to.
-	 * 
-	 * @return the ID of the vertex this event refers to
-	 */
-	public ManagementVertexID getVertexID() {
-		return this.managementVertexID;
-	}
-
-	/**
-	 * Returns the name of the instance the vertex is now assigned to.
-	 * 
-	 * @return the name of the instance the vertex is now assigned to
-	 */
-	public String getInstanceName() {
-		return this.instanceName;
-	}
-
-	@Override
-	public void read(final DataInputView in) throws IOException {
-
-		super.read(in);
-
-		this.managementVertexID.read(in);
-		this.instanceName = StringRecord.readString(in);
-	}
-
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-
-		super.write(out);
-
-		this.managementVertexID.write(out);
-		StringRecord.writeString(out, this.instanceName);
-	}
-
-
-	@Override
-	public boolean equals(final Object obj) {
-
-		if (!super.equals(obj)) {
-			return false;
-		}
-
-		if (!(obj instanceof VertexAssignmentEvent)) {
-			return false;
-		}
-
-		final VertexAssignmentEvent vae = (VertexAssignmentEvent) obj;
-
-		if (!this.managementVertexID.equals(vae.getVertexID())) {
-			return false;
-		}
-
-		if (this.instanceName == null) {
-			if (vae.getInstanceName() != null) {
-				return false;
-			}
-		} else {
-			if (!this.instanceName.equals(vae.getInstanceName())) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public int hashCode() {
-
-		if (this.managementVertexID != null) {
-			return this.managementVertexID.hashCode();
-		}
-
-		return super.hashCode();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
index 69ab16d..fb3b247 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/job/VertexEvent.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.job;
 
 import java.io.IOException;
@@ -24,46 +23,31 @@ import java.io.IOException;
 import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.ExecutionState2;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
-import org.apache.flink.runtime.util.EnumUtils;
 
 /**
- * Vertex events are transmitted from the job manager
- * to the job client in order to inform the user about
+ * Vertex events are transmitted from the job manager to the job client in order to inform the user about
  * changes in terms of a tasks execution state.
- * 
  */
 public class VertexEvent extends AbstractEvent {
 
-	/**
-	 * The ID of the job vertex this event belongs to.
-	 */
+	/** The ID of the job vertex this event belongs to. */
 	private JobVertexID jobVertexID;
 
-	/**
-	 * The name of the job vertex this event belongs to.
-	 */
+	/** The name of the job vertex this event belongs to. */
 	private String jobVertexName;
 
-	/**
-	 * The number of subtasks the corresponding vertex has been split into at runtime.
-	 */
+	/** The number of subtasks the corresponding vertex has been split into at runtime. */
 	private int totalNumberOfSubtasks;
 
-	/**
-	 * The index of the subtask that this event belongs to.
-	 */
+	/** The index of the subtask that this event belongs to. */
 	private int indexOfSubtask;
 
-	/**
-	 * The current execution state of the subtask this event belongs to.
-	 */
-	private ExecutionState currentExecutionState;
+	/** The current execution state of the subtask this event belongs to. */
+	private ExecutionState2 currentExecutionState;
 
-	/**
-	 * An optional more detailed description of the event.
-	 */
+	/** An optional more detailed description of the event. */
 	private String description;
 
 	/**
@@ -84,10 +68,12 @@ public class VertexEvent extends AbstractEvent {
 	 * @param description
 	 *        an optional description
 	 */
-	public VertexEvent(final long timestamp, final JobVertexID jobVertexID, final String jobVertexName,
-			final int totalNumberOfSubtasks, final int indexOfSubtask, final ExecutionState currentExecutionState,
-			final String description) {
+	public VertexEvent(long timestamp, JobVertexID jobVertexID, String jobVertexName,
+			int totalNumberOfSubtasks, int indexOfSubtask, ExecutionState2 currentExecutionState,
+			String description)
+	{
 		super(timestamp);
+		
 		this.jobVertexID = jobVertexID;
 		this.jobVertexName = jobVertexName;
 		this.totalNumberOfSubtasks = totalNumberOfSubtasks;
@@ -108,7 +94,7 @@ public class VertexEvent extends AbstractEvent {
 		this.jobVertexName = null;
 		this.totalNumberOfSubtasks = -1;
 		this.indexOfSubtask = -1;
-		this.currentExecutionState = ExecutionState.CREATED;
+		this.currentExecutionState = ExecutionState2.CREATED;
 		this.description = null;
 	}
 
@@ -122,7 +108,7 @@ public class VertexEvent extends AbstractEvent {
 		this.jobVertexName = StringRecord.readString(in);
 		this.totalNumberOfSubtasks = in.readInt();
 		this.indexOfSubtask = in.readInt();
-		this.currentExecutionState = EnumUtils.readEnum(in, ExecutionState.class);
+		this.currentExecutionState = ExecutionState2.values()[in.readInt()];
 		this.description = StringRecord.readString(in);
 	}
 
@@ -136,7 +122,7 @@ public class VertexEvent extends AbstractEvent {
 		StringRecord.writeString(out, this.jobVertexName);
 		out.writeInt(this.totalNumberOfSubtasks);
 		out.writeInt(this.indexOfSubtask);
-		EnumUtils.writeEnum(out, this.currentExecutionState);
+		out.writeInt(this.currentExecutionState.ordinal());
 		StringRecord.writeString(out, this.description);
 	}
 
@@ -183,7 +169,7 @@ public class VertexEvent extends AbstractEvent {
 	 * 
 	 * @return the current execution state of the subtask this event belongs to
 	 */
-	public ExecutionState getCurrentExecutionState() {
+	public ExecutionState2 getCurrentExecutionState() {
 		return currentExecutionState;
 	}
 
@@ -261,7 +247,6 @@ public class VertexEvent extends AbstractEvent {
 
 	@Override
 	public int hashCode() {
-
 		return super.hashCode();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java b/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java
index e18b1eb..22c0620 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/event/task/AbstractEvent.java
@@ -16,17 +16,13 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.task;
 
 import org.apache.flink.core.io.IOReadableWritable;
 
 /**
  * This type of event can be used to exchange notification messages between
- * different {@link TaskManager} objects at runtime using the communication
- * channels Nephele has established between different tasks.
- * 
+ * different {@link org.apache.flink.runtime.taskmanager.TaskManager} objects
+ * at runtime using the communication channels.
  */
-public abstract class AbstractEvent implements IOReadableWritable {
-
-}
+public abstract class AbstractEvent implements IOReadableWritable {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
index 51ba96c..06c9fb6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
@@ -82,20 +82,6 @@ public interface Environment {
 	int getIndexInSubtaskGroup();
 
 	/**
-	 * Sends a notification that objects that a new user thread has been started to the execution observer.
-	 *
-	 * @param userThread the user thread which has been started
-	 */
-	void userThreadStarted(Thread userThread);
-
-	/**
-	 * Sends a notification that a user thread has finished to the execution observer.
-	 *
-	 * @param userThread the user thread which has finished
-	 */
-	void userThreadFinished(Thread userThread);
-
-	/**
 	 * Returns the input split provider assigned to this environment.
 	 *
 	 * @return the input split provider or <code>null</code> if no such provider has been assigned to this environment.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java
new file mode 100644
index 0000000..9b39851
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionAttempt.java
@@ -0,0 +1,100 @@
+/**
+ * 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.execution;
+
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+/**
+ * An attempt to execute a task for a {@link ExecutionVertex2}.
+ */
+public class ExecutionAttempt implements java.io.Serializable {
+	
+	private static final long serialVersionUID = 1L;
+	
+
+	private final JobVertexID vertexId;
+	
+	private final int subtaskIndex;
+	
+	private final ExecutionAttemptID executionId;
+	
+	private final int attempt;
+
+	// --------------------------------------------------------------------------------------------
+	
+	public ExecutionAttempt(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, int attempt) {
+		if (vertexId == null || executionId == null || subtaskIndex < 0 || attempt < 1) {
+			throw new IllegalArgumentException();
+		}
+		
+		this.vertexId = vertexId;
+		this.subtaskIndex = subtaskIndex;
+		this.executionId = executionId;
+		this.attempt = attempt;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public JobVertexID getVertexId() {
+		return vertexId;
+	}
+	
+	public int getSubtaskIndex() {
+		return subtaskIndex;
+	}
+	
+	public ExecutionAttemptID getExecutionId() {
+		return executionId;
+	}
+	
+	public int getAttempt() {
+		return attempt;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public int hashCode() {
+		return vertexId.hashCode() +
+				executionId.hashCode() +
+				31 * subtaskIndex +
+				17 * attempt;
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj instanceof ExecutionAttempt) {
+			ExecutionAttempt other = (ExecutionAttempt) obj;
+			return this.executionId.equals(other.executionId) &&
+					this.vertexId.equals(other.vertexId) &&
+					this.subtaskIndex == other.subtaskIndex &&
+					this.attempt == other.attempt;
+		} else {
+			return false;
+		}
+	}
+	
+	@Override
+	public String toString() {
+		return String.format("ExecutionAttempt (vertex=%s, subtask=%d, executionAttemptId=%s, attempt=%d)",
+				vertexId, subtaskIndex, executionId, attempt);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
index ded630f..b08c847 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
@@ -18,55 +18,16 @@
 
 package org.apache.flink.runtime.execution;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 /**
- * This interface must be implemented by classes which should be able to receive notifications about
+ * Implementing this interface allows classes to receive notifications about
  * changes of a task's execution state.
  */
 public interface ExecutionListener {
 
-	/**
-	 * Called when the execution state of the associated task has changed. It is important to point out that multiple
-	 * execution listeners can be invoked as a reaction to a state change, according to their priority. As a result, the
-	 * value of <code>newExecutionState</code> may be out-dated by the time a particular execution listener is called.
-	 * To determine the most recent state of the respective task, it is recommended to store a reference on the
-	 * execution that represents it and then call <code>getExecutionState()</code> on the vertex within this method.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the task belongs to
-	 * @param vertexID
-	 *        the ID of the task whose state has changed
-	 * @param newExecutionState
-	 *        the execution state the task has just switched to
-	 * @param optionalMessage
-	 *        an optional message providing further information on the state change
-	 */
-	void executionStateChanged(JobID jobID, ExecutionVertexID vertexID, ExecutionState newExecutionState,
-			String optionalMessage);
-
-	/**
-	 * Called when the user task has started a new thread.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the task belongs to
-	 * @param vertexID
-	 *        the ID of the task that started of new thread
-	 * @param userThread
-	 *        the user thread which has been started
-	 */
-	void userThreadStarted(JobID jobID, ExecutionVertexID vertexID, Thread userThread);
-
-	/**
-	 * Called when a thread spawn by a user task has finished.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the task belongs to
-	 * @param vertexID
-	 *        the ID of the task whose thread has finished
-	 * @param userThread
-	 *        the user thread which has finished
-	 */
-	void userThreadFinished(JobID jobID, ExecutionVertexID vertexID, Thread userThread);
+	void executionStateChanged(JobID jobID, JobVertexID vertexId, int subtask, ExecutionAttemptID executionId,
+			ExecutionState2 newExecutionState, String optionalMessage);
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
index c3b9b72..20a6180 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionObserver.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.execution;
 
 public interface ExecutionObserver {
@@ -29,23 +28,7 @@ public interface ExecutionObserver {
 	 * @param optionalMessage
 	 *        an optional message providing further information on the state change
 	 */
-	void executionStateChanged(ExecutionState newExecutionState, String optionalMessage);
-
-	/**
-	 * Called when the user task has started a new thread.
-	 * 
-	 * @param userThread
-	 *        the user thread which has been started
-	 */
-	void userThreadStarted(Thread userThread);
-
-	/**
-	 * Called when a thread spawn by a user task has finished.
-	 * 
-	 * @param userThread
-	 *        the user thread which has finished
-	 */
-	void userThreadFinished(Thread userThread);
+	void executionStateChanged(ExecutionState2 newExecutionState, String optionalMessage);
 
 	/**
 	 * Returns whether the task has been canceled.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java
new file mode 100644
index 0000000..c2b2070
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionState2.java
@@ -0,0 +1,38 @@
+/**
+ * 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.execution;
+
+public enum ExecutionState2 {
+
+	CREATED,
+	
+	SCHEDULED,
+	
+	DEPLOYING,
+	
+	RUNNING,
+	
+	FINISHED,
+	
+	CANCELING,
+	
+	CANCELED,
+	
+	FAILED
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionStateTransition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionStateTransition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionStateTransition.java
deleted file mode 100644
index 98557f9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionStateTransition.java
+++ /dev/null
@@ -1,118 +0,0 @@
-/**
- * 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.execution;
-
-import static org.apache.flink.runtime.execution.ExecutionState.CANCELED;
-import static org.apache.flink.runtime.execution.ExecutionState.CANCELING;
-import static org.apache.flink.runtime.execution.ExecutionState.FAILED;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class is a utility class to check the consistency of Nephele's execution state model.
- * 
- */
-public final class ExecutionStateTransition {
-
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(ExecutionStateTransition.class);
-
-	/**
-	 * Private constructor to prevent instantiation of object.
-	 */
-	private ExecutionStateTransition() {
-	}
-
-	/**
-	 * Checks the transition of the execution state and outputs an error in case of an unexpected state transition.
-	 * 
-	 * @param jobManager
-	 *        <code>true</code> to indicate the method is called by the job manager,
-	 *        <code>false/<code> to indicate it is called by a task manager
-	 * @param taskName
-	 *        the name of the task whose execution has changed
-	 * @param oldState
-	 *        the old execution state
-	 * @param newState
-	 *        the new execution state
-	 */
-	public static void checkTransition(boolean jobManager, String taskName, ExecutionState oldState, ExecutionState newState) {
-
-		LOG.info((jobManager ? "JM: " : "TM: ") + "ExecutionState set from " + oldState + " to " + newState + " for task " + taskName);
-
-		boolean unexpectedStateChange = true;
-
-		// This is the regular life cycle of a task
-		if (oldState == ExecutionState.CREATED && newState == ExecutionState.SCHEDULED) {
-			unexpectedStateChange = false;
-		}
-		else if (oldState == ExecutionState.SCHEDULED && newState == ExecutionState.ASSIGNED) {
-			unexpectedStateChange = false;
-		}
-		else if (oldState == ExecutionState.ASSIGNED && newState == ExecutionState.READY) {
-			unexpectedStateChange = false;
-		}
-		else if (oldState == ExecutionState.READY && newState == ExecutionState.STARTING) {
-			unexpectedStateChange = false;
-		}
-		else if (oldState == ExecutionState.STARTING && newState == ExecutionState.RUNNING) {
-			unexpectedStateChange = false;
-		}
-		else if (oldState == ExecutionState.RUNNING && newState == ExecutionState.FINISHING) {
-			unexpectedStateChange = false;
-		}
-		else if (oldState == ExecutionState.FINISHING && newState == ExecutionState.FINISHED) {
-			unexpectedStateChange = false;
-		}
-
-		// A vertex might skip the SCHEDULED state if its resource has been allocated in a previous stage.
-		else if (oldState == ExecutionState.CREATED && newState == ExecutionState.ASSIGNED) {
-			unexpectedStateChange = false;
-		}
-
-		// This transition can appear if a task in a stage which is not yet executed gets canceled.
-		else if (oldState == ExecutionState.SCHEDULED && newState == ExecutionState.CANCELING) {
-			unexpectedStateChange = false;
-		}
-
-		// This transition can appear if a task in a stage which is not yet executed gets canceled.
-		else if (oldState == ExecutionState.ASSIGNED && newState == ExecutionState.CANCELING) {
-			unexpectedStateChange = false;
-		}
-
-		// This transition can appear if a task is canceled that is not yet running on the task manager.
-		else if (oldState == ExecutionState.READY && newState == ExecutionState.CANCELING) {
-			unexpectedStateChange = false;
-		}
-
-		// -------------- error cases --------------
-		else if (newState == FAILED || newState == CANCELED || newState == CANCELING) {
-			// any state may fail or cancel itself
-			unexpectedStateChange = false;
-		}
-
-		if (unexpectedStateChange) {
-			LOG.error("Unexpected state change: " + oldState + " -> " + newState);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
index 6bfaf2a..cb7b290 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/RuntimeEnvironment.java
@@ -16,16 +16,29 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.execution;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.Buffer;
 import org.apache.flink.runtime.io.network.bufferprovider.BufferAvailabilityListener;
@@ -43,177 +56,139 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.protocols.AccumulatorProtocol;
+import org.apache.flink.runtime.taskmanager.Task;
 import org.apache.flink.util.StringUtils;
 
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.FutureTask;
+import com.google.common.base.Preconditions;
+
 
-/**
- * The user code of every Nephele task runs inside a <code>RuntimeEnvironment</code> object. The environment provides
- * important services to the task. It keeps track of setting up the communication channels and provides access to input
- * splits, memory manager, etc.
- * <p/>
- * This class is thread-safe.
- */
 public class RuntimeEnvironment implements Environment, BufferProvider, LocalBufferPoolOwner, Runnable {
 
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(RuntimeEnvironment.class);
+	/** The log object used for debugging. */
+	private static final Log LOG = LogFactory.getLog(RuntimeEnvironment.class);
 
-	/**
-	 * The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds).
-	 */
+	/** The interval to sleep in case a communication channel is not yet entirely set up (in milliseconds). */
 	private static final int SLEEPINTERVAL = 100;
+	
+	// --------------------------------------------------------------------------------------------
 
-	/**
-	 * List of output gates created by the task.
-	 */
-	private final List<OutputGate> outputGates = new CopyOnWriteArrayList<OutputGate>();
-
-	/**
-	 * List of input gates created by the task.
-	 */
-	private final List<InputGate<? extends IOReadableWritable>> inputGates = new CopyOnWriteArrayList<InputGate<? extends IOReadableWritable>>();
-
-	/**
-	 * Queue of unbound input gate IDs which are required for deserializing an environment in the course of an RPC
-	 * call.
-	 */
-	private final Queue<GateID> unboundInputGateIDs = new ArrayDeque<GateID>();
-
-	/**
-	 * The memory manager of the current environment (currently the one associated with the executing TaskManager).
-	 */
-	private final MemoryManager memoryManager;
-
-	/**
-	 * The io manager of the current environment (currently the one associated with the executing TaskManager).
-	 */
-	private final IOManager ioManager;
+	/** The task that owns this environment */
+	private final Task owner;
+	
+	
+	/** The job configuration encapsulated in the environment object. */
+	private final Configuration jobConfiguration;
 
-	/**
-	 * Class of the task to run in this environment.
-	 */
+	/** The task configuration encapsulated in the environment object. */
+	private final Configuration taskConfiguration;
+	
+	
+	/** ClassLoader for all user code classes */
+	private final ClassLoader userCodeClassLoader;
+	
+	/** Class of the task to run in this environment. */
 	private final Class<? extends AbstractInvokable> invokableClass;
 
-	/**
-	 * Instance of the class to be run in this environment.
-	 */
+	/** Instance of the class to be run in this environment. */
 	private final AbstractInvokable invokable;
+	
+	
+	/** List of output gates created by the task. */
+	private final ArrayList<OutputGate> outputGates = new ArrayList<OutputGate>();
 
-	/**
-	 * The ID of the job this task belongs to.
-	 */
-	private final JobID jobID;
+	/** List of input gates created by the task. */
+	private final ArrayList<InputGate<? extends IOReadableWritable>> inputGates = new ArrayList<InputGate<? extends IOReadableWritable>>();
 
-	/**
-	 * The job configuration encapsulated in the environment object.
-	 */
-	private final Configuration jobConfiguration;
+	/** Unbound input gate IDs which are required for deserializing an environment in the course of an RPC call. */
+	private final Queue<GateID> unboundInputGateIDs = new ArrayDeque<GateID>();
 
-	/**
-	 * The task configuration encapsulated in the environment object.
-	 */
-	private final Configuration taskConfiguration;
+	/** The memory manager of the current environment (currently the one associated with the executing TaskManager). */
+	private final MemoryManager memoryManager;
 
-	/**
-	 * The input split provider that can be queried for new input splits.
-	 */
+	/** The I/O manager of the current environment (currently the one associated with the executing TaskManager). */
+	private final IOManager ioManager;
+
+	/** The input split provider that can be queried for new input splits.  */
 	private final InputSplitProvider inputSplitProvider;
 
-	/**
-	 * The observer object for the task's execution.
-	 */
-	private volatile ExecutionObserver executionObserver = null;
 	
-	/**
-	 * The thread executing the task in the environment.
-	 */
+	/** The thread executing the task in the environment. */
 	private volatile Thread executingThread;
 
 	/**
 	 * The RPC proxy to report accumulators to JobManager
 	 */
-	private AccumulatorProtocol accumulatorProtocolProxy = null;
-
-	/**
-	 * The index of this subtask in the subtask group.
-	 */
-	private final int indexInSubtaskGroup;
-
-	/**
-	 * The current number of subtasks the respective task is split into.
-	 */
-	private final int currentNumberOfSubtasks;
-
-	/**
-	 * The name of the task running in this environment.
-	 */
-	private final String taskName;
+	private final AccumulatorProtocol accumulatorProtocolProxy;
 
+	private final Map<String,FutureTask<Path>> cacheCopyTasks = new HashMap<String, FutureTask<Path>>();
+	
 	private LocalBufferPool outputBufferPool;
-
-	private final Map<String,FutureTask<Path>> cacheCopyTasks;
 	
-	private volatile boolean canceled;
+	private AtomicBoolean canceled = new AtomicBoolean();
+
+
+	public RuntimeEnvironment(Task owner, TaskDeploymentDescriptor tdd,
+							ClassLoader userCodeClassLoader,
+							MemoryManager memoryManager, IOManager ioManager,
+							InputSplitProvider inputSplitProvider,
+							AccumulatorProtocol accumulatorProtocolProxy)
+		throws Exception
+	{
+		Preconditions.checkNotNull(owner);
+		Preconditions.checkNotNull(memoryManager);
+		Preconditions.checkNotNull(ioManager);
+		Preconditions.checkNotNull(inputSplitProvider);
+		Preconditions.checkNotNull(accumulatorProtocolProxy);
+		Preconditions.checkNotNull(userCodeClassLoader);
+		
+		this.owner = owner;
 
-	/**
-	 * Constructs a runtime environment from a task deployment description.
-	 * 
-	 * @param tdd
-	 *        the task deployment description
-	 * @param memoryManager
-	 *        the task manager's memory manager component
-	 * @param ioManager
-	 *        the task manager's I/O manager component
-	 * @param inputSplitProvider
-	 *        the input split provider for this environment
-	 * @throws Exception
-	 *         thrown if an error occurs while instantiating the invokable class
-	 */
-	public RuntimeEnvironment(final TaskDeploymentDescriptor tdd,
-							final MemoryManager memoryManager, final IOManager ioManager,
-							final InputSplitProvider inputSplitProvider,
-							AccumulatorProtocol accumulatorProtocolProxy, Map<String, FutureTask<Path>> cpTasks) throws Exception {
-
-		this.jobID = tdd.getJobID();
-		this.taskName = tdd.getTaskName();
-		this.invokableClass = tdd.getInvokableClass();
-		this.jobConfiguration = tdd.getJobConfiguration();
-		this.taskConfiguration = tdd.getTaskConfiguration();
-		this.indexInSubtaskGroup = tdd.getIndexInSubtaskGroup();
-		this.currentNumberOfSubtasks = tdd.getCurrentNumberOfSubtasks();
 		this.memoryManager = memoryManager;
 		this.ioManager = ioManager;
 		this.inputSplitProvider = inputSplitProvider;
 		this.accumulatorProtocolProxy = accumulatorProtocolProxy;
-		this.cacheCopyTasks = cpTasks;
 
-		this.invokable = this.invokableClass.newInstance();
+		// load and instantiate the invokable class
+		this.userCodeClassLoader = userCodeClassLoader;
+		try {
+			final String className = tdd.getInvokableClassName();
+			this.invokableClass = Class.forName(className, true, userCodeClassLoader).asSubclass(AbstractInvokable.class);
+		}
+		catch (Throwable t) {
+			throw new Exception("Could not load invokable class.", t);
+		}
+		
+		try {
+			this.invokable = this.invokableClass.newInstance();
+		}
+		catch (Throwable t) {
+			throw new Exception("Could not instantiate the invokable class.", t);
+		}
+		
+		this.jobConfiguration = tdd.getJobConfiguration();
+		this.taskConfiguration = tdd.getTaskConfiguration();
+		
 		this.invokable.setEnvironment(this);
 		this.invokable.registerInputOutput();
 
-		int numOutputGates = tdd.getNumberOfOutputGateDescriptors();
-
-		for (int i = 0; i < numOutputGates; ++i) {
-			this.outputGates.get(i).initializeChannels(tdd.getOutputGateDescriptor(i));
+		List<GateDeploymentDescriptor> inGates = tdd.getInputGates();
+		List<GateDeploymentDescriptor> outGates = tdd.getOutputGates();
+		
+		
+		if (this.inputGates.size() != inGates.size()) {
+			throw new Exception("The number of readers created in 'registerInputOutput()' "
+					+ "is different than the number of connected incoming edges in the job graph.");
 		}
-
-		int numInputGates = tdd.getNumberOfInputGateDescriptors();
-
-		for (int i = 0; i < numInputGates; i++) {
-			this.inputGates.get(i).initializeChannels(tdd.getInputGateDescriptor(i));
+		if (this.outputGates.size() != outGates.size()) {
+			throw new Exception("The number of writers created in 'registerInputOutput()' "
+					+ "is different than the number of connected outgoing edges in the job graph.");
+		}
+		
+		for (int i = 0; i < inGates.size(); i++) {
+			this.inputGates.get(i).initializeChannels(inGates.get(i));
+		}
+		for (int i = 0; i < outGates.size(); i++) {
+			this.outputGates.get(i).initializeChannels(outGates.get(i));
 		}
 	}
 
@@ -228,7 +203,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	@Override
 	public JobID getJobID() {
-		return this.jobID;
+		return this.owner.getJobID();
 	}
 
 	@Override
@@ -246,30 +221,24 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	@Override
 	public void run() {
-		if (invokable == null) {
-			LOG.error("ExecutionEnvironment has no Invokable set");
-		}
 
-		// Now the actual program starts to run
-		changeExecutionState(ExecutionState.RUNNING, null);
-
-		// If the task has been canceled in the mean time, do not even start it
-		if (this.executionObserver.isCanceled()) {
-			changeExecutionState(ExecutionState.CANCELED, null);
+		// quick fail in case the task was cancelled while the tread was started
+		if (owner.isCanceled()) {
+			owner.cancelingDone();
 			return;
 		}
-
+		
 		try {
-			ClassLoader cl = LibraryCacheManager.getClassLoader(jobID);
-			Thread.currentThread().setContextClassLoader(cl);
+			Thread.currentThread().setContextClassLoader(userCodeClassLoader);
 			this.invokable.invoke();
 
 			// Make sure, we enter the catch block when the task has been canceled
-			if (this.executionObserver.isCanceled()) {
-				throw new InterruptedException();
+			if (this.owner.isCanceled()) {
+				throw new CancelTaskException();
 			}
-		} catch (Throwable t) {
-			if (!this.executionObserver.isCanceled()) {
+		}
+		catch (Throwable t) {
+			if (!this.owner.isCanceled()) {
 
 				// Perform clean up when the task failed and has been not canceled by the user
 				try {
@@ -282,19 +251,16 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			// Release all resources that may currently be allocated by the individual channels
 			releaseAllChannelResources();
 
-			if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
-				changeExecutionState(ExecutionState.CANCELED, null);
+			if (this.owner.isCanceled() || t instanceof CancelTaskException) {
+				this.owner.cancelingDone();
 			}
 			else {
-				changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
+				this.owner.markFailed(t);
 			}
 
 			return;
 		}
 
-		// Task finished running, but there may be unconsumed output data in some of the channels
-		changeExecutionState(ExecutionState.FINISHING, null);
-
 		try {
 			// If there is any unclosed input gate, close it and propagate close operation to corresponding output gate
 			closeInputGates();
@@ -307,16 +273,16 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 			// Now we wait until all output channels have written out their data and are closed
 			waitForOutputChannelsToBeClosed();
-		} catch (Throwable t) {
-
+		}
+		catch (Throwable t) {
 			// Release all resources that may currently be allocated by the individual channels
 			releaseAllChannelResources();
 
-			if (this.executionObserver.isCanceled() || t instanceof CancelTaskException) {
-				changeExecutionState(ExecutionState.CANCELED, null);
+			if (this.owner.isCanceled() || t instanceof CancelTaskException) {
+				this.owner.cancelingDone();
 			}
 			else {
-				changeExecutionState(ExecutionState.FAILED, StringUtils.stringifyException(t));
+				this.owner.markFailed(t);
 			}
 
 			return;
@@ -326,7 +292,9 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 		releaseAllChannelResources();
 
 		// Finally, switch execution state to FINISHED and report to job manager
-		changeExecutionState(ExecutionState.FINISHED, null);
+		if (!owner.markAsFinished()) {
+			owner.markFailed(new Exception());
+		}
 	}
 
 	@Override
@@ -403,12 +371,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 		synchronized (this) {
 
 			if (this.executingThread == null) {
-				if (this.taskName == null) {
-					this.executingThread = new Thread(this);
-				}
-				else {
-					this.executingThread = new Thread(this, getTaskNameWithIndex());
-				}
+				String name = owner.getTaskNameWithSubtasks();
+				this.executingThread = new Thread(this, name);
 			}
 
 			return this.executingThread;
@@ -416,9 +380,11 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	}
 	
 	public void cancelExecution() {
-		canceled = true;
+		if (!canceled.compareAndSet(false, true)) {
+			return;
+		}
 
-		LOG.info("Canceling " + getTaskNameWithIndex());
+		LOG.info("Canceling " + owner.getTaskNameWithSubtasks());
 
 		// Request user code to shut down
 		if (this.invokable != null) {
@@ -429,6 +395,8 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 			}
 		}
 		
+		final Thread executingThread = this.executingThread;
+		
 		// interrupt the running thread and wait for it to die
 		executingThread.interrupt();
 		
@@ -442,10 +410,10 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 		
 		// Continuously interrupt the user thread until it changed to state CANCELED
 		while (executingThread != null && executingThread.isAlive()) {
-			LOG.warn("Task " + getTaskName() + " did not react to cancelling signal. Sending repeated interrupt.");
+			LOG.warn("Task " + owner.getTaskNameWithSubtasks() + " did not react to cancelling signal. Sending repeated interrupt.");
 
 			if (LOG.isDebugEnabled()) {
-				StringBuilder bld = new StringBuilder("Task ").append(getTaskName()).append(" is stuck in method:\n");
+				StringBuilder bld = new StringBuilder("Task ").append(owner.getTaskNameWithSubtasks()).append(" is stuck in method:\n");
 				
 				StackTraceElement[] stack = executingThread.getStackTrace();
 				for (StackTraceElement e : stack) {
@@ -465,12 +433,11 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	/**
 	 * Blocks until all output channels are closed.
 	 *
-	 * @throws IOException          thrown if an error occurred while closing the output channels
 	 * @throws InterruptedException thrown if the thread waiting for the channels to be closed is interrupted
 	 */
 	private void waitForOutputChannelsToBeClosed() throws InterruptedException {
 		// Make sure, we leave this method with an InterruptedException when the task has been canceled
-		if (this.executionObserver.isCanceled()) {
+		if (this.owner.isCanceled()) {
 			return;
 		}
 
@@ -487,10 +454,10 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	 */
 	private void waitForInputChannelsToBeClosed() throws IOException, InterruptedException {
 		// Wait for disconnection of all output gates
-		while (!canceled) {
+		while (!canceled.get()) {
 
 			// Make sure, we leave this method with an InterruptedException when the task has been canceled
-			if (this.executionObserver.isCanceled()) {
+			if (this.owner.isCanceled()) {
 				throw new InterruptedException();
 			}
 
@@ -554,41 +521,17 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 
 	@Override
 	public int getCurrentNumberOfSubtasks() {
-		return this.currentNumberOfSubtasks;
+		return owner.getNumberOfSubtasks();
 	}
 
 	@Override
 	public int getIndexInSubtaskGroup() {
-		return this.indexInSubtaskGroup;
-	}
-
-	private void changeExecutionState(final ExecutionState newExecutionState, final String optionalMessage) {
-		if (this.executionObserver != null) {
-			this.executionObserver.executionStateChanged(newExecutionState, optionalMessage);
-		}
+		return owner.getSubtaskIndex();
 	}
 
 	@Override
 	public String getTaskName() {
-		return this.taskName;
-	}
-
-	/**
-	 * Returns the name of the task with its index in the subtask group and the total number of subtasks.
-	 *
-	 * @return the name of the task with its index in the subtask group and the total number of subtasks
-	 */
-	public String getTaskNameWithIndex() {
-		return String.format("%s (%d/%d)", this.taskName, getIndexInSubtaskGroup() + 1, getCurrentNumberOfSubtasks());
-	}
-
-	/**
-	 * Sets the execution observer for this environment.
-	 *
-	 * @param executionObserver the execution observer for this environment
-	 */
-	public void setExecutionObserver(final ExecutionObserver executionObserver) {
-		this.executionObserver = executionObserver;
+		return owner.getTaskName();
 	}
 
 	@Override
@@ -596,20 +539,6 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 		return this.inputSplitProvider;
 	}
 
-	@Override
-	public void userThreadStarted(final Thread userThread) {
-		if (this.executionObserver != null) {
-			this.executionObserver.userThreadStarted(userThread);
-		}
-	}
-
-	@Override
-	public void userThreadFinished(final Thread userThread) {
-		if (this.executionObserver != null) {
-			this.executionObserver.userThreadFinished(userThread);
-		}
-	}
-
 	/**
 	 * Releases the allocated resources (particularly buffer) of input and output channels attached to this task. This
 	 * method should only be called after the respected task has stopped running.
@@ -742,6 +671,10 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 		return accumulatorProtocolProxy;
 	}
 
+	public void addCopyTasksForCacheFile(Map<String, FutureTask<Path>> copyTasks) {
+		this.cacheCopyTasks.putAll(copyTasks);
+	}
+	
 	public void addCopyTaskForCacheFile(String name, FutureTask<Path> copyTask) {
 		this.cacheCopyTasks.put(name, copyTask);
 	}
@@ -809,7 +742,7 @@ public class RuntimeEnvironment implements Environment, BufferProvider, LocalBuf
 	@Override
 	public void logBufferUtilization() {
 		LOG.info(String.format("\t%s: %d available, %d requested, %d designated",
-				getTaskNameWithIndex(),
+				owner.getTaskNameWithSubtasks(),
 				this.outputBufferPool.numAvailableBuffers(),
 				this.outputBufferPool.numRequestedBuffers(),
 				this.outputBufferPool.numDesignatedBuffers()));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
index 55bf3f0..1076ede 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheManager.java
@@ -271,7 +271,7 @@ public final class LibraryCacheManager {
 	 * 
 	 * @param id
 	 *        the ID of the job to be registered.
-	 * @param clientPaths
+	 * @param requiredJarFiles
 	 *        the client path's of the required libraries
 	 * @throws IOException
 	 *         thrown if the library cache manager could not be instantiated or one of the requested libraries is not in
@@ -290,7 +290,7 @@ public final class LibraryCacheManager {
 	 * 
 	 * @param id
 	 *        the ID of the job to be registered.
-	 * @param clientPaths
+	 * @param requiredJarFiles
 	 *        the client path's of the required libraries
 	 * @throws IOException
 	 *         thrown if one of the requested libraries is not in the cache
@@ -438,8 +438,6 @@ public final class LibraryCacheManager {
 	 *        the ID of the job to return the class loader for
 	 * @return the class loader of requested vertex or <code>null</code> if no class loader has been registered with the
 	 *         given ID.
-	 * @throws IOException
-	 *         thrown if the library cache manager could not be instantiated
 	 */
 	private ClassLoader getClassLoaderInternal(final JobID id) {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheUpdate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheUpdate.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheUpdate.java
index 4310bfb..2cfbe58 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheUpdate.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/librarycache/LibraryCacheUpdate.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.execution.librarycache;
 
 import java.io.IOException;
@@ -26,15 +25,12 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
 /**
- * This class is used to encapsulate the transmission of a library file in a Nephele RPC call.
- * 
+ * This class is used to encapsulate the transmission of a library file in a RPC call.
  */
 public class LibraryCacheUpdate implements IOReadableWritable {
 
-	/**
-	 * The name of the library file that is transmitted with this object.
-	 */
-	private String libraryFileName = null;
+	/** The name of the library file that is transmitted with this object. */
+	private String libraryFileName;
 
 	/**
 	 * Constructs a new library cache update object.
@@ -42,32 +38,32 @@ public class LibraryCacheUpdate implements IOReadableWritable {
 	 * @param libraryFileName
 	 *        the name of the library that should be transported within this object.
 	 */
-	public LibraryCacheUpdate(final String libraryFileName) {
+	public LibraryCacheUpdate(String libraryFileName) {
+		if (libraryFileName == null) {
+			throw new IllegalArgumentException("libraryFileName must not be null");
+		}
+		
 		this.libraryFileName = libraryFileName;
 	}
 
 	/**
 	 * Constructor used to reconstruct the object at the receiver of the RPC call.
 	 */
-	public LibraryCacheUpdate() {
-	}
+	public LibraryCacheUpdate() {}
 
 
 	@Override
-	public void read(final DataInputView in) throws IOException {
-
+	public void read(DataInputView in) throws IOException {
 		LibraryCacheManager.readLibraryFromStream(in);
 	}
 
 
 	@Override
-	public void write(final DataOutputView out) throws IOException {
-
+	public void write(DataOutputView out) throws IOException {
 		if (this.libraryFileName == null) {
 			throw new IOException("libraryFileName is null");
 		}
 
 		LibraryCacheManager.writeLibraryToStream(this.libraryFileName, out);
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java
new file mode 100644
index 0000000..84781cb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/AllVerticesIterator.java
@@ -0,0 +1,71 @@
+/**
+ * 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 java.util.Iterator;
+import java.util.NoSuchElementException;
+
+class AllVerticesIterator implements Iterator<ExecutionVertex2> {
+
+	private final Iterator<ExecutionJobVertex> jobVertices;
+	
+	private ExecutionVertex2[] currVertices;
+	
+	private int currPos;
+	
+	
+	public AllVerticesIterator(Iterator<ExecutionJobVertex> jobVertices) {
+		this.jobVertices = jobVertices;
+	}
+	
+	
+	@Override
+	public boolean hasNext() {
+		while (true) {
+			if (currVertices != null) {
+				if (currPos < currVertices.length) {
+					return true;
+				} else {
+					currVertices = null;
+				}
+			}
+			else if (jobVertices.hasNext()) {
+				currVertices = jobVertices.next().getTaskVertices();
+				currPos = 0;
+			}
+			else {
+				return false;
+			}
+		}
+	}
+	
+	@Override
+	public ExecutionVertex2 next() {
+		if (hasNext()) {
+			return currVertices[currPos++];
+		} else {
+			throw new NoSuchElementException();
+		}
+	}
+	
+	@Override
+	public void remove() {
+		throw new UnsupportedOperationException();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DistributionPatternProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DistributionPatternProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DistributionPatternProvider.java
deleted file mode 100644
index 610b294..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DistributionPatternProvider.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/**
- * 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 org.apache.flink.runtime.jobgraph.DistributionPattern;
-
-public final class DistributionPatternProvider {
-
-	/**
-	 * Checks if two subtasks of different tasks should be wired.
-	 * 
-	 * @param pattern
-	 *        the distribution pattern that should be used
-	 * @param nodeLowerStage
-	 *        the index of the producing task's subtask
-	 * @param nodeUpperStage
-	 *        the index of the consuming task's subtask
-	 * @param sizeSetLowerStage
-	 *        the number of subtasks of the producing task
-	 * @param sizeSetUpperStage
-	 *        the number of subtasks of the consuming task
-	 * @return <code>true</code> if a wire between the two considered subtasks should be created, <code>false</code>
-	 *         otherwise
-	 */
-	public static boolean createWire(final DistributionPattern pattern, final int nodeLowerStage,
-			final int nodeUpperStage, final int sizeSetLowerStage, final int sizeSetUpperStage) {
-
-		switch (pattern) {
-		case BIPARTITE:
-			return true;
-
-		case POINTWISE:
-			if (sizeSetLowerStage < sizeSetUpperStage) {
-				if (nodeLowerStage == (nodeUpperStage % sizeSetLowerStage)) {
-					return true;
-				}
-			} else {
-				if ((nodeLowerStage % sizeSetUpperStage) == nodeUpperStage) {
-					return true;
-				}
-			}
-
-			return false;
-
-			/*
-			 * case STAR:
-			 * if (sizeSetLowerStage > sizeSetUpperStage) {
-			 * int groupNumber = nodeLowerStage / Math.max(sizeSetLowerStage / sizeSetUpperStage, 1);
-			 * if (nodeUpperStage == groupNumber) {
-			 * return true;
-			 * }
-			 * } else {
-			 * int groupNumber = nodeUpperStage / Math.max(sizeSetUpperStage / sizeSetLowerStage, 1);
-			 * if (nodeLowerStage == groupNumber) {
-			 * return true;
-			 * }
-			 * }
-			 * return false;
-			 */
-
-		default:
-			// this will never happen.
-			throw new IllegalStateException("No Match for Distribution Pattern found.");
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java
new file mode 100644
index 0000000..18e57f0
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionAttemptID.java
@@ -0,0 +1,30 @@
+/**
+ * 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 org.apache.flink.runtime.AbstractID;
+
+/**
+ * Unique identifier for the attempt to execute a tasks. Multiple attempts happen
+ * in cases of failures and recovery.
+ */
+public class ExecutionAttemptID extends AbstractID {
+	
+	private static final long serialVersionUID = -1169683445778281344L;
+}


[50/63] [abbrv] git commit: Add co-location-constraints as a special case of slot-shared scheduling

Posted by se...@apache.org.
Add co-location-constraints as a special case of slot-shared scheduling


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/8231b62f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/8231b62f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/8231b62f

Branch: refs/heads/master
Commit: 8231b62ff42aae53ca3a7b552980838ccab824ab
Parents: caa4ebe
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 21:15:07 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:50 2014 +0200

----------------------------------------------------------------------
 .../plantranslate/NepheleJobGraphGenerator.java |   6 +-
 .../flink/runtime/executiongraph/Execution.java |  15 +-
 .../executiongraph/ExecutionJobVertex.java      |  28 ++--
 .../runtime/executiongraph/ExecutionVertex.java |  16 ++
 .../runtime/jobgraph/AbstractJobVertex.java     |  75 ++++++++-
 .../scheduler/CoLocationConstraint.java         |  49 ++----
 .../jobmanager/scheduler/CoLocationGroup.java   |  87 ++++++++++
 .../scheduler/NoResourceAvailableException.java |   6 +-
 .../jobmanager/scheduler/ScheduledUnit.java     |   6 +-
 .../runtime/jobmanager/scheduler/Scheduler.java | 168 ++++++++++++-------
 .../scheduler/SlotSharingGroupAssignment.java   |  28 +++-
 .../ExecutionGraphConstructionTest.java         | 116 +++++++++++++
 .../jobmanager/CoLocationConstraintITCase.java  |   3 +-
 .../runtime/jobmanager/SlotSharingITCase.java   |   4 -
 .../ScheduleWithCoLocationHintTest.java         | 147 ++++++++++++----
 .../scheduler/SchedulerSlotSharingTest.java     |  64 +++++++
 16 files changed, 663 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index 1783e1d..39647d2 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.compiler.plantranslate;
 
 import java.util.ArrayList;
@@ -124,6 +123,7 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 	
 	private IterationPlanNode currentIteration;	// hack: as long as no nesting is possible, remember the enclosing iteration
 	
+	
 	// ------------------------------------------------------------------------
 
 	/**
@@ -357,6 +357,10 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 				// store the id of the iterations the step functions participate in
 				IterationDescriptor descr = this.iterations.get(this.currentIteration);
 				new TaskConfig(vertex.getConfiguration()).setIterationId(descr.getId());
+				
+				// make sure tasks inside iterations are co-located with the head
+				AbstractJobVertex headVertex = this.iterations.get(this.currentIteration).getHeadTask();
+				vertex.setStrictlyCoLocatedWith(headVertex);
 			}
 	
 			// store in the map

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index c290883..f6b6a9b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -34,11 +34,13 @@ import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFuture;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotAllocationFutureAction;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.taskmanager.TaskOperationResult;
 import org.apache.flink.util.ExceptionUtils;
 import org.slf4j.Logger;
@@ -162,9 +164,19 @@ public class Execution {
 			throw new NullPointerException();
 		}
 		
+		final SlotSharingGroup sharingGroup = vertex.getJobVertex().getSlotSharingGroup();
+		final CoLocationConstraint locationConstraint = vertex.getLocationConstraint();
+		
+		// sanity check
+		if (locationConstraint != null && sharingGroup == null) {
+			throw new RuntimeException("Trying to schedule with co-location constraint but without slot sharing allowed.");
+		}
+				
 		if (transitionState(CREATED, SCHEDULED)) {
 			
-			ScheduledUnit toSchedule = new ScheduledUnit(this, vertex.getJobVertex().getSlotSharingGroup());
+			ScheduledUnit toSchedule = locationConstraint == null ?
+				new ScheduledUnit(this, sharingGroup) :
+				new ScheduledUnit(this, sharingGroup, locationConstraint);
 		
 			// IMPORTANT: To prevent leaks of cluster resources, we need to make sure that slots are returned
 			//     in all cases where the deployment failed. we use many try {} finally {} clauses to assure that
@@ -484,7 +496,6 @@ public class Execution {
 					}
 				}
 				
-				
 				if (!isCallback && (current == RUNNING || current == DEPLOYING)) {
 					if (LOG.isDebugEnabled()) {
 						LOG.debug("Sending out cancel request, to remove task execution from TaskManager.");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
index 1884ce0..6a75610 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -32,10 +32,10 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
-
 import org.slf4j.Logger;
 
 
@@ -62,10 +62,12 @@ public class ExecutionJobVertex {
 	
 	private final boolean[] finishedSubtasks;
 			
-	private int numSubtasksInFinalState;
+	private volatile int numSubtasksInFinalState;
+	
 	
+	private final SlotSharingGroup slotSharingGroup;
 	
-	private SlotSharingGroup slotSharingGroup;
+	private final CoLocationGroup coLocationGroup;
 	
 	
 	public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex, int defaultParallelism) throws JobException {
@@ -90,6 +92,15 @@ public class ExecutionJobVertex {
 		
 		this.inputs = new ArrayList<IntermediateResult>(jobVertex.getInputs().size());
 		
+		// take the sharing group
+		this.slotSharingGroup = jobVertex.getSlotSharingGroup();
+		this.coLocationGroup = jobVertex.getCoLocationGroup();
+		
+		// setup the coLocation group
+		if (coLocationGroup != null && slotSharingGroup == null) {
+			throw new JobException("Vertex uses a co-location constraint without using slot sharing");
+		}
+		
 		// create the intermediate results
 		this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()];
 		for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) {
@@ -110,9 +121,6 @@ public class ExecutionJobVertex {
 			}
 		}
 		
-		// take the sharing group
-		this.slotSharingGroup = jobVertex.getSlotSharingGroup();
-		
 		// set up the input splits, if the vertex has any
 		try {
 			@SuppressWarnings("unchecked")
@@ -163,14 +171,14 @@ public class ExecutionJobVertex {
 		return splitAssigner;
 	}
 	
-	public void setSlotSharingGroup(SlotSharingGroup slotSharingGroup) {
-		this.slotSharingGroup = slotSharingGroup;
-	}
-	
 	public SlotSharingGroup getSlotSharingGroup() {
 		return slotSharingGroup;
 	}
 	
+	public CoLocationGroup getCoLocationGroup() {
+		return coLocationGroup;
+	}
+	
 	public List<IntermediateResult> getInputs() {
 		return inputs;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index fcd21af..22a3a64 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -38,6 +38,8 @@ import org.apache.flink.runtime.jobgraph.DistributionPattern;
 import org.apache.flink.runtime.jobgraph.JobEdge;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 
@@ -64,6 +66,8 @@ public class ExecutionVertex {
 	
 	private final List<Execution> priorExecutions;
 	
+	private final CoLocationConstraint locationConstraint;
+	
 	private volatile Execution currentExecution;	// this field must never be null
 	
 	// --------------------------------------------------------------------------------------------
@@ -87,6 +91,14 @@ public class ExecutionVertex {
 		this.priorExecutions = new CopyOnWriteArrayList<Execution>();
 		
 		this.currentExecution = new Execution(this, 0, createTimestamp);
+		
+		// create a co-location scheduling hint, if necessary
+		CoLocationGroup clg = jobVertex.getCoLocationGroup();
+		if (clg != null) {
+			this.locationConstraint = clg.getLocationConstraint(subTaskIndex);
+		} else {
+			this.locationConstraint = null;
+		}
 	}
 	
 	
@@ -129,6 +141,10 @@ public class ExecutionVertex {
 		return inputEdges[input];
 	}
 	
+	public CoLocationConstraint getLocationConstraint() {
+		return locationConstraint;
+	}
+	
 	public Execution getCurrentExecutionAttempt() {
 		return currentExecution;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
index d2462ba..899210f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/AbstractJobVertex.java
@@ -25,6 +25,7 @@ import org.apache.commons.lang3.Validate;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplitSource;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 
 /**
@@ -68,8 +69,9 @@ public class AbstractJobVertex implements java.io.Serializable {
 	/** Optionally, a sharing group that allows subtasks from different job vertices to run concurrently in one slot */
 	private SlotSharingGroup slotSharingGroup;
 	
-//	private AbstractJobVertex coLocatedWith
-
+	/** The group inside which the vertex subtasks share slots */
+	private CoLocationGroup coLocationGroup;
+	
 	// --------------------------------------------------------------------------------------------
 
 	/**
@@ -227,6 +229,12 @@ public class AbstractJobVertex implements java.io.Serializable {
 		return this.inputs;
 	}
 	
+	/**
+	 * Associates this vertex with a slot sharing group for scheduling. Different vertices in the same
+	 * slot sharing group can run one subtask each in the same slot.
+	 * 
+	 * @param grp The slot sharing group to associate the vertex with.
+	 */
 	public void setSlotSharingGroup(SlotSharingGroup grp) {
 		if (this.slotSharingGroup != null) {
 			this.slotSharingGroup.removeVertexFromGroup(id);
@@ -238,10 +246,73 @@ public class AbstractJobVertex implements java.io.Serializable {
 		}
 	}
 	
+	/**
+	 * Gets the slot sharing group that this vertex is associated with. Different vertices in the same
+	 * slot sharing group can run one subtask each in the same slot. If the vertex is not associated with
+	 * a slot sharing group, this method returns {@code null}.
+	 * 
+	 * @return The slot sharing group to associate the vertex with, or {@code null}, if not associated with one.
+	 */
 	public SlotSharingGroup getSlotSharingGroup() {
 		return slotSharingGroup;
 	}
 	
+	/**
+	 * Tells this vertex to strictly co locate its subtasks with the subtasks of the given vertex.
+	 * Strict co-location implies that the n'th subtask of this vertex will run on the same parallel computing
+	 * instance (TaskManager) as the n'th subtask of the given vertex.
+	 * 
+	 * NOTE: Co-location is only possible between vertices in a slot sharing group.
+	 * 
+	 * NOTE: This vertex must (transitively) depend on the vertex to be co-located with. That means that the
+	 * respective vertex must be a (transitive) input of this vertex.
+	 * 
+	 * @param strictlyCoLocatedWith The vertex whose subtasks to co-locate this vertex's subtasks with.
+	 * 
+	 * @throws IllegalArgumentException Thrown, if this vertex and the vertex to co-locate with are not in a common
+	 *                                  slot sharing group.
+	 * 
+	 * @see #setSlotSharingGroup(SlotSharingGroup)
+	 */
+	public void setStrictlyCoLocatedWith(AbstractJobVertex strictlyCoLocatedWith) {
+		if (this.slotSharingGroup == null || this.slotSharingGroup != strictlyCoLocatedWith.slotSharingGroup) {
+			throw new IllegalArgumentException();
+		}
+		
+		CoLocationGroup thisGroup = this.coLocationGroup;
+		CoLocationGroup otherGroup = strictlyCoLocatedWith.coLocationGroup;
+		
+		if (otherGroup == null) {
+			if (thisGroup == null) {
+				CoLocationGroup group = new CoLocationGroup(this, strictlyCoLocatedWith);
+				this.coLocationGroup = group;
+				strictlyCoLocatedWith.coLocationGroup = group;
+			}
+			else {
+				thisGroup.addVertex(strictlyCoLocatedWith);
+				strictlyCoLocatedWith.coLocationGroup = thisGroup;
+			}
+		}
+		else {
+			if (thisGroup == null) {
+				otherGroup.addVertex(this);
+				this.coLocationGroup = otherGroup;
+			}
+			else {
+				// both had yet distinct groups, we need to merge them
+				thisGroup.mergeInto(otherGroup);
+			}
+		}
+	}
+	
+	public CoLocationGroup getCoLocationGroup() {
+		return coLocationGroup;
+	}
+	
+	public void updateCoLocationGroup(CoLocationGroup group) {
+		this.coLocationGroup = group;
+	}
+	
 	// --------------------------------------------------------------------------------------------
 
 	public IntermediateDataSet createAndAddResultDataSet() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
index 26332c8..64f7ffc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
@@ -18,47 +18,30 @@
 
 package org.apache.flink.runtime.jobmanager.scheduler;
 
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
-
-import org.apache.flink.runtime.instance.AllocatedSlot;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.instance.Instance;
 
 public class CoLocationConstraint {
 	
-	private static final AtomicReferenceFieldUpdater<CoLocationConstraint, SharedSlot> UPDATER =
-			AtomicReferenceFieldUpdater.newUpdater(CoLocationConstraint.class, SharedSlot.class, "slot");
-	
-	private volatile SharedSlot slot;
-
+	private volatile Instance location;
 	
-	public boolean isUnassigned() {
-		return slot == null;
-	}
-	
-	public SharedSlot getSlot() {
-		return slot;
-	}
 	
-	public SharedSlot swapInNewSlot(AllocatedSlot newSlot) {
-		SharedSlot newShared = new SharedSlot(newSlot);
+	public void setLocation(Instance location) {
+		if (location == null) {
+			throw new IllegalArgumentException();
+		}
 		
-		// atomic swap/release-other to prevent resource leaks
-		while (true) {
-			SharedSlot current = this.slot;
-			if (UPDATER.compareAndSet(this, current, newShared)) {
-				if (current != null) {
-					current.rease();
-				}
-				return newShared;
-			}
+		if (this.location == null) {
+			this.location = location;
+		} else {
+			throw new IllegalStateException("The constraint has already been assigned a location.");
 		}
 	}
 	
-	public SubSlot allocateSubSlot(JobVertexID jid) {
-		if (this.slot == null) {
-			throw new IllegalStateException("Location constraint has not yet been assigned a slot.");
-		}
-		
-		return slot.allocateSubSlot(jid);
+	public Instance getLocation() {
+		return location;
+	}
+	
+	public boolean isUnassigned() {
+		return this.location == null;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java
new file mode 100644
index 0000000..2398334
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java
@@ -0,0 +1,87 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+
+import com.google.common.base.Preconditions;
+
+public class CoLocationGroup implements java.io.Serializable {
+	
+	private static final long serialVersionUID = -2605819490401895297L;
+
+	
+	private final List<AbstractJobVertex> vertices = new ArrayList<AbstractJobVertex>();
+	
+	private transient ArrayList<CoLocationConstraint> constraints;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public CoLocationGroup() {}
+	
+	public CoLocationGroup(AbstractJobVertex... vertices) {
+		for (AbstractJobVertex v : vertices) {
+			this.vertices.add(v);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void addVertex(AbstractJobVertex vertex) {
+		Preconditions.checkNotNull(vertex);
+		this.vertices.add(vertex);
+	}
+	
+	public void mergeInto(CoLocationGroup other) {
+		Preconditions.checkNotNull(other);
+		
+		for (AbstractJobVertex v : this.vertices) {
+			v.updateCoLocationGroup(other);
+		}
+		
+		// move vertex membership
+		other.vertices.addAll(this.vertices);
+		this.vertices.clear();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public CoLocationConstraint getLocationConstraint(int subtask) {
+		ensureConstraints(subtask + 1);
+		return constraints.get(subtask);
+	}
+	
+	public void ensureConstraints(int num) {
+		if (constraints == null) {
+			constraints = new ArrayList<CoLocationConstraint>(num);
+		} else {
+			constraints.ensureCapacity(num);
+		}
+		
+		if (num > constraints.size()) {
+			constraints.ensureCapacity(num);
+			for (int i = constraints.size(); i < num; i++) {
+				constraints.add(new CoLocationConstraint());
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
index 0fa866a..70e0dc7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
@@ -25,11 +25,13 @@ public class NoResourceAvailableException extends JobException {
 	private static final long serialVersionUID = -2249953165298717803L;
 
 	public NoResourceAvailableException() {
-		super("Not enough resources available for the Job.");
+		super("Not enough free slots available to run the job. "
+				+ "You can decrease the operator parallelism or increase the number of slots per TaskManager in the configuration.");
 	}
 	
 	public NoResourceAvailableException(ScheduledUnit unit) {
-		super("No resource available to schedule unit " + unit);
+		super("No resource available to schedule unit " + unit
+				+ ". You can decrease the operator parallelism or increase the number of slots per TaskManager in the configuration.");
 	}
 
 	public NoResourceAvailableException(String message) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
index 28fd916..8911605 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
@@ -49,11 +49,13 @@ public class ScheduledUnit {
 		this.locationConstraint = null;
 	}
 	
-	public ScheduledUnit(Execution task, CoLocationConstraint locationConstraint) {
+	public ScheduledUnit(Execution task, SlotSharingGroup sharingUnit, CoLocationConstraint locationConstraint) {
 		Preconditions.checkNotNull(task);
+		Preconditions.checkNotNull(sharingUnit);
+		Preconditions.checkNotNull(locationConstraint);
 		
 		this.vertexExecution = task;
-		this.sharingGroup = null;
+		this.sharingGroup = sharingUnit;
 		this.locationConstraint = locationConstraint;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index eb2c0a5..b9f83fc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.jobmanager.scheduler;
 
 import java.util.ArrayDeque;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Queue;
@@ -32,6 +33,7 @@ import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceDiedException;
 import org.apache.flink.runtime.instance.InstanceListener;
+import org.apache.flink.util.ExceptionUtils;
 
 /**
  * The scheduler is responsible for distributing the ready-to-run tasks and assigning them to instances and
@@ -57,11 +59,11 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 	private final Queue<QueuedTask> taskQueue = new ArrayDeque<QueuedTask>();
 	
 	
-	private int unconstrainedAssignments = 0;
+	private int unconstrainedAssignments;
 	
-	private int localizedAssignments = 0;
+	private int localizedAssignments;
 	
-	private int nonLocalizedAssignments = 0;
+	private int nonLocalizedAssignments;
 	
 	
 	public Scheduler() {
@@ -149,57 +151,57 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 	
 		synchronized (globalLock) {
 			
-			// 1)  === If the task has a strict co-schedule hint, obey it ===
-			
-			CoLocationConstraint locationConstraint = task.getLocationConstraint();
-			if (locationConstraint != null) {
-				// location constraints can never be scheduled in a queued fashion
-				if (queueIfNoResource) {
-					throw new IllegalArgumentException("A task with a location constraint was scheduled in a queued fashion.");
-				}
-				
-				// since we are inside the global lock scope, we can check, allocate, and assign
-				// in one atomic action. however, slots may die and be deallocated
-				
-				// (a) is the constraint has not yet has a slot, get one
-				if (locationConstraint.isUnassigned()) {
-					// try and get a slot
-					AllocatedSlot newSlot = getFreeSlotForTask(vertex);
-					if (newSlot == null) {
-						throw new NoResourceAvailableException();
-					}
-					SharedSlot sl = locationConstraint.swapInNewSlot(newSlot);
-					SubSlot slot = sl.allocateSubSlot(vertex.getJobvertexId());
-					
-					updateLocalityCounters(newSlot.getLocality());
-					return slot;
-				}
-				else {
-					// try to get a subslot. returns null, if the location's slot has been released
-					// in the meantime
-					SubSlot slot = locationConstraint.allocateSubSlot(vertex.getJobvertexId());
-					if (slot == null) {
-						// get a new slot. at the same instance!!!
-						Instance location = locationConstraint.getSlot().getAllocatedSlot().getInstance();
-						AllocatedSlot newSlot;
-						try {
-							newSlot = location.allocateSlot(vertex.getJobId());
-						} catch (InstanceDiedException e) {
-							throw new NoResourceAvailableException("The instance of the required location died.");
-						}
-						if (newSlot == null) {
-							throw new NoResourceAvailableException();
-						}
-						SharedSlot sharedSlot = locationConstraint.swapInNewSlot(newSlot);
-						slot = sharedSlot.allocateSubSlot(vertex.getJobvertexId());
-					}
-					
-					updateLocalityCounters(Locality.LOCAL);
-					return slot;
-				}
-			}
+//			// 1)  === If the task has a strict co-schedule hint, obey it ===
+//			
+//			CoLocationConstraint locationConstraint = task.getLocationConstraint();
+//			if (locationConstraint != null) {
+//				// location constraints can never be scheduled in a queued fashion
+//				if (queueIfNoResource) {
+//					throw new IllegalArgumentException("A task with a location constraint was scheduled in a queued fashion.");
+//				}
+//				
+//				// since we are inside the global lock scope, we can check, allocate, and assign
+//				// in one atomic action. however, slots may die and be deallocated
+//				
+//				// (a) is the constraint has not yet has a slot, get one
+//				if (locationConstraint.isUnassigned()) {
+//					// try and get a slot
+//					AllocatedSlot newSlot = getFreeSlotForTask(vertex);
+//					if (newSlot == null) {
+//						throw new NoResourceAvailableException();
+//					}
+//					SharedSlot sl = locationConstraint.swapInNewSlot(newSlot);
+//					SubSlot slot = sl.allocateSubSlot(vertex.getJobvertexId());
+//					
+//					updateLocalityCounters(newSlot.getLocality());
+//					return slot;
+//				}
+//				else {
+//					// try to get a subslot. returns null, if the location's slot has been released
+//					// in the meantime
+//					SubSlot slot = locationConstraint.allocateSubSlot(vertex.getJobvertexId());
+//					if (slot == null) {
+//						// get a new slot. at the same instance!!!
+//						Instance location = locationConstraint.getSlot().getAllocatedSlot().getInstance();
+//						AllocatedSlot newSlot;
+//						try {
+//							newSlot = location.allocateSlot(vertex.getJobId());
+//						} catch (InstanceDiedException e) {
+//							throw new NoResourceAvailableException("The instance of the required location died.");
+//						}
+//						if (newSlot == null) {
+//							throw new NoResourceAvailableException();
+//						}
+//						SharedSlot sharedSlot = locationConstraint.swapInNewSlot(newSlot);
+//						slot = sharedSlot.allocateSubSlot(vertex.getJobvertexId());
+//					}
+//					
+//					updateLocalityCounters(Locality.LOCAL);
+//					return slot;
+//				}
+//			}
 		
-			// 2)  === If the task has a slot sharing group, schedule with shared slots ===
+			// 1)  === If the task has a slot sharing group, schedule with shared slots ===
 			
 			SlotSharingGroup sharingUnit = task.getSlotSharingGroup();
 			if (sharingUnit != null) {
@@ -209,40 +211,79 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 				}
 				
 				final SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
+				final CoLocationConstraint constraint = task.getLocationConstraint();
 				
 				AllocatedSlot newSlot = null;
-				AllocatedSlot slotFromGroup = assignment.getSlotForTask(vertex.getJobvertexId(), vertex, true);
 				
+				// get a slot from the group. obey location constraints, if existing and assigned
+				AllocatedSlot slotFromGroup;
+				if (constraint == null || constraint.isUnassigned()) {
+					slotFromGroup = assignment.getSlotForTask(vertex.getJobvertexId(), vertex);
+				}
+				else {
+					// this returns null, if the constraint cannot be fulfilled
+					slotFromGroup = assignment.getSlotForTask(vertex.getJobvertexId(), constraint);
+				}
+				
+				// the following needs to make sure any allocated slot is released in case of an error
 				try {
+					
+					// check whether the slot from the group is already what we want
 					if (slotFromGroup != null) {
 						// local (or unconstrained in the current group)
 						if (slotFromGroup.getLocality() != Locality.NON_LOCAL) {
+							
+							// attach to the locality constraint
+							if (constraint != null && constraint.isUnassigned()) {
+								constraint.setLocation(slotFromGroup.getInstance());
+							}
+							
 							updateLocalityCounters(slotFromGroup.getLocality());
 							return slotFromGroup;
 						}
 					}
 					
-					// get another new slot, since we could not place it into the group,
-					// (or we could not place it locally)
-					newSlot = getFreeSlotForTask(vertex);
+					final Iterable<Instance> locations = (constraint == null || constraint.isUnassigned()) ?
+							vertex.getPreferredLocations() : Collections.singleton(constraint.getLocation());
+					
+					// get a new slot, since we could not place it into the group, or we could not place it locally
+					newSlot = getFreeSlotForTask(vertex, locations);
+					
 					AllocatedSlot toUse;
 					
 					if (newSlot == null) {
 						if (slotFromGroup == null) {
+							// both null
 							throw new NoResourceAvailableException();
 						} else {
 							toUse = slotFromGroup;
 						}
 					}
 					else if (slotFromGroup == null || newSlot.getLocality() == Locality.LOCAL) {
+						// new slot is preferable
+						if (slotFromGroup != null) {
+							slotFromGroup.releaseSlot();
+						}
+						
 						toUse = sharingUnit.getTaskAssignment().addSlotWithTask(newSlot, task.getJobVertexId());
-					} else {
+					}
+					else {
+						// both are available and potentially usable
+						newSlot.releaseSlot();
 						toUse = slotFromGroup;
 					}
 					
+					// assign to the co-location hint, if we have one and it is unassigned
+					if (constraint != null && constraint.isUnassigned()) {
+						constraint.setLocation(toUse.getInstance());
+					}
+					
 					updateLocalityCounters(toUse.getLocality());
 					return toUse;
 				}
+				catch (NoResourceAvailableException e) {
+					throw e;
+				}
 				catch (Throwable t) {
 					if (slotFromGroup != null) {
 						slotFromGroup.releaseSlot();
@@ -250,12 +291,14 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 					if (newSlot != null) {
 						newSlot.releaseSlot();
 					}
+					
+					ExceptionUtils.rethrow(t, "An error occurred while allocating a slot in a sharing group");
 				}
 			}
 		
-			// 3) === schedule without hints and sharing ===
+			// 2) === schedule without hints and sharing ===
 			
-			AllocatedSlot slot = getFreeSlotForTask(vertex);
+			AllocatedSlot slot = getFreeSlotForTask(vertex, vertex.getPreferredLocations());
 			if (slot != null) {
 				updateLocalityCounters(slot.getLocality());
 				return slot;
@@ -282,7 +325,7 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 	 * @param vertex The task to run. 
 	 * @return The instance to run the vertex on, it {@code null}, if no instance is available.
 	 */
-	protected AllocatedSlot getFreeSlotForTask(ExecutionVertex vertex) {
+	protected AllocatedSlot getFreeSlotForTask(ExecutionVertex vertex, Iterable<Instance> requestedLocations) {
 		
 		// we need potentially to loop multiple times, because there may be false positives
 		// in the set-with-available-instances
@@ -291,8 +334,7 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 				return null;
 			}
 			
-			Iterable<Instance> locationsIterable = vertex.getPreferredLocations();
-			Iterator<Instance> locations = locationsIterable == null ? null : locationsIterable.iterator();
+			Iterator<Instance> locations = requestedLocations == null ? null : requestedLocations.iterator();
 			
 			Instance instanceToUse = null;
 			Locality locality = Locality.UNCONSTRAINED;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
index 290381c..474fdbe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.jobmanager.scheduler;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
@@ -83,12 +84,35 @@ public class SlotSharingGroupAssignment {
 		}
 	}
 	
-	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex vertex, boolean localOnly) {
+	/**
+	 * Gets a slot suitable for the given task vertex. This method will prefer slots that are local
+	 * (with respect to {@link ExecutionVertex#getPreferredLocations()}), but will return non local
+	 * slots if no local slot is available. The method returns null, when no slot is available for the
+	 * given JobVertexID at all.
+	 * 
+	 * @param jid
+	 * @param vertex
+	 * 
+	 * @return A task vertex for a task with the given JobVertexID, or null, if none is available.
+	 */
+	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex vertex) {
 		synchronized (allSlots) {
-			return getSlotForTaskInternal(jid, vertex.getPreferredLocations(), localOnly);
+			return getSlotForTaskInternal(jid, vertex.getPreferredLocations(), false);
 		}
 	}
 	
+	
+	public AllocatedSlot getSlotForTask(JobVertexID jid, CoLocationConstraint constraint) {
+		if (constraint.isUnassigned()) {
+			throw new IllegalArgumentException("CoLocationConstraint is unassigned");
+		}
+		
+		synchronized (allSlots) {
+			return getSlotForTaskInternal(jid, Collections.singleton(constraint.getLocation()), true);
+		}
+	}
+	
+	
 	public boolean sharedSlotAvailableForJid(SharedSlot slot, JobVertexID jid, boolean lastSubSlot) {
 		if (slot == null || jid == null) {
 			throw new NullPointerException();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
index 642af31..2ea758c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.executiongraph;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.mockito.Mockito.mock;
@@ -30,8 +31,10 @@ import org.mockito.Matchers;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
@@ -46,6 +49,8 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 
 /**
  * This class contains test concerning the correct conversion from {@link JobGraph} to {@link ExecutionGraph} objects.
@@ -598,4 +603,115 @@ public class ExecutionGraphConstructionTest {
 			fail(e.getMessage());
 		}
 	}
+	
+	@Test
+	public void testCoLocationConstraintCreation() {
+		try {
+			final JobID jobId = new JobID();
+			final String jobName = "Co-Location Constraint Sample Job";
+			final Configuration cfg = new Configuration();
+			
+			// simple group of two, cyclic
+			AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+			AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+			v1.setParallelism(6);
+			v2.setParallelism(4);
+			
+			SlotSharingGroup sl1 = new SlotSharingGroup();
+			v1.setSlotSharingGroup(sl1);
+			v2.setSlotSharingGroup(sl1);
+			v2.setStrictlyCoLocatedWith(v1);
+			v1.setStrictlyCoLocatedWith(v2);
+			
+			// complex forked dependency pattern
+			AbstractJobVertex v3 = new AbstractJobVertex("vertex3");
+			AbstractJobVertex v4 = new AbstractJobVertex("vertex4");
+			AbstractJobVertex v5 = new AbstractJobVertex("vertex5");
+			AbstractJobVertex v6 = new AbstractJobVertex("vertex6");
+			AbstractJobVertex v7 = new AbstractJobVertex("vertex7");
+			v3.setParallelism(3);
+			v4.setParallelism(3);
+			v5.setParallelism(3);
+			v6.setParallelism(3);
+			v7.setParallelism(3);
+			
+			SlotSharingGroup sl2 = new SlotSharingGroup();
+			v3.setSlotSharingGroup(sl2);
+			v4.setSlotSharingGroup(sl2);
+			v5.setSlotSharingGroup(sl2);
+			v6.setSlotSharingGroup(sl2);
+			v7.setSlotSharingGroup(sl2);
+			
+			v4.setStrictlyCoLocatedWith(v3);
+			v5.setStrictlyCoLocatedWith(v4);
+			v6.setStrictlyCoLocatedWith(v3);
+			v3.setStrictlyCoLocatedWith(v7);
+			
+			// isolated vertex
+			AbstractJobVertex v8 = new AbstractJobVertex("vertex8");
+			v8.setParallelism(2);
+			
+			JobGraph jg = new JobGraph(jobId, jobName, v1, v2, v3, v4, v5, v6, v7, v8);
+			
+			ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+			eg.attachJobGraph(jg.getVerticesSortedTopologicallyFromSources());
+			
+			// check the v1 / v2 co location hints ( assumes parallelism(v1) >= parallelism(v2) )
+			{
+				ExecutionVertex[] v1s = eg.getJobVertex(v1.getID()).getTaskVertices();
+				ExecutionVertex[] v2s = eg.getJobVertex(v2.getID()).getTaskVertices();
+				
+				Set<CoLocationConstraint> all = new HashSet<CoLocationConstraint>();
+				
+				for (int i = 0; i < v2.getParallelism(); i++) {
+					assertNotNull(v1s[i].getLocationConstraint());
+					assertNotNull(v2s[i].getLocationConstraint());
+					assertTrue(v1s[i].getLocationConstraint() == v2s[i].getLocationConstraint());
+					all.add(v1s[i].getLocationConstraint());
+				}
+				
+				for (int i = v2.getParallelism(); i < v1.getParallelism(); i++) {
+					assertNotNull(v1s[i].getLocationConstraint());
+					all.add(v1s[i].getLocationConstraint());
+				}
+				
+				assertEquals("not all co location constraints are distinct", v1.getParallelism(), all.size());
+			}
+			
+			// check the v1 / v2 co location hints ( assumes parallelism(v1) >= parallelism(v2) )
+			{
+				ExecutionVertex[] v3s = eg.getJobVertex(v3.getID()).getTaskVertices();
+				ExecutionVertex[] v4s = eg.getJobVertex(v4.getID()).getTaskVertices();
+				ExecutionVertex[] v5s = eg.getJobVertex(v5.getID()).getTaskVertices();
+				ExecutionVertex[] v6s = eg.getJobVertex(v6.getID()).getTaskVertices();
+				ExecutionVertex[] v7s = eg.getJobVertex(v7.getID()).getTaskVertices();
+				
+				Set<CoLocationConstraint> all = new HashSet<CoLocationConstraint>();
+				
+				for (int i = 0; i < v3.getParallelism(); i++) {
+					assertNotNull(v3s[i].getLocationConstraint());
+					assertTrue(v3s[i].getLocationConstraint() == v4s[i].getLocationConstraint());
+					assertTrue(v4s[i].getLocationConstraint() == v5s[i].getLocationConstraint());
+					assertTrue(v5s[i].getLocationConstraint() == v6s[i].getLocationConstraint());
+					assertTrue(v6s[i].getLocationConstraint() == v7s[i].getLocationConstraint());
+					all.add(v3s[i].getLocationConstraint());
+				}
+				
+				assertEquals("not all co location constraints are distinct", v3.getParallelism(), all.size());
+			}
+			
+			// check the v8 has no co location hints
+			{
+				ExecutionVertex[] v8s = eg.getJobVertex(v8.getID()).getTaskVertices();
+				
+				for (int i = 0; i < v8.getParallelism(); i++) {
+					assertNull(v8s[i].getLocationConstraint());
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
index 9bda8ed..605c49f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/CoLocationConstraintITCase.java
@@ -38,7 +38,6 @@ import org.apache.flink.runtime.jobmanager.tasks.Sender;
 import org.junit.Test;
 
 public class CoLocationConstraintITCase {
-
 	
 	/**
 	 * This job runs in N slots with N senders and N receivers. Unless slot sharing is used, it cannot complete.
@@ -64,6 +63,8 @@ public class CoLocationConstraintITCase {
 			sender.setSlotSharingGroup(sharingGroup);
 			receiver.setSlotSharingGroup(sharingGroup);
 			
+			receiver.setStrictlyCoLocatedWith(sender);
+			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
 			final JobManager jm = startJobManager(NUM_TASKS);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java
index 98abc8d..70435f0 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/SlotSharingITCase.java
@@ -89,8 +89,6 @@ public class SlotSharingITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
-					
-					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;
@@ -163,8 +161,6 @@ public class SlotSharingITCase {
 				if (eg != null) {
 					eg.waitForJobEnd();
 					assertEquals(JobStatus.FINISHED, eg.getState());
-					
-					assertEquals(0, eg.getRegisteredExecutions().size());
 				}
 				else {
 					// already done, that was fast;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
index 0ee9346..0efe10b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
@@ -22,6 +22,7 @@ import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.g
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import org.apache.flink.runtime.instance.AllocatedSlot;
@@ -32,23 +33,21 @@ import org.junit.Test;
 public class ScheduleWithCoLocationHintTest {
 
 	@Test
-	public void schedule() {
+	public void scheduleAllSharedAndCoLocated() {
 		try {
 			JobVertexID jid1 = new JobVertexID();
 			JobVertexID jid2 = new JobVertexID();
 			
 			Scheduler scheduler = new Scheduler();
 			
-			Instance i1 = getRandomInstance(2);
-			Instance i2 = getRandomInstance(2);
-			Instance i3 = getRandomInstance(2);
-			
-			scheduler.newInstanceAvailable(i1);
-			scheduler.newInstanceAvailable(i2);
-			scheduler.newInstanceAvailable(i3);
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
 			
 			assertEquals(6, scheduler.getNumberOfAvailableSlots());
 			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
+			
 			CoLocationConstraint c1 = new CoLocationConstraint();
 			CoLocationConstraint c2 = new CoLocationConstraint();
 			CoLocationConstraint c3 = new CoLocationConstraint();
@@ -57,18 +56,18 @@ public class ScheduleWithCoLocationHintTest {
 			CoLocationConstraint c6 = new CoLocationConstraint();
 			
 			// schedule 4 tasks from the first vertex group
-			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), c1));
-			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 6), c2));
-			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 6), c3));
-			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 6), c4));
-			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 6), c1));
-			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 6), c2));
-			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 6), c3));
-			AllocatedSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 6), c5));
-			AllocatedSlot s9 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 6), c6));
-			AllocatedSlot s10 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 6), c4));
-			AllocatedSlot s11 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 6), c5));
-			AllocatedSlot s12 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 6), c6));
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 6), sharingGroup, c2));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 6), sharingGroup, c3));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 6), sharingGroup, c4));
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 6), sharingGroup, c1));
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 6), sharingGroup, c2));
+			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 6), sharingGroup, c3));
+			AllocatedSlot s8 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 4, 6), sharingGroup, c5));
+			AllocatedSlot s9 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 5, 6), sharingGroup, c6));
+			AllocatedSlot s10 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 6), sharingGroup, c4));
+			AllocatedSlot s11 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 4, 6), sharingGroup, c5));
+			AllocatedSlot s12 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 5, 6), sharingGroup, c6));
 
 			assertNotNull(s1);
 			assertNotNull(s2);
@@ -83,6 +82,20 @@ public class ScheduleWithCoLocationHintTest {
 			assertNotNull(s11);
 			assertNotNull(s12);
 			
+			// check that each slot got exactly two tasks
+			assertEquals(2, ((SubSlot) s1).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s2).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s3).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s4).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s5).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s6).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s7).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s8).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s9).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s10).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s11).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s12).getSharedSlot().getNumberOfAllocatedSubSlots());
+			
 			assertEquals(s1.getInstance(), s5.getInstance());
 			assertEquals(s2.getInstance(), s6.getInstance());
 			assertEquals(s3.getInstance(), s7.getInstance());
@@ -90,25 +103,31 @@ public class ScheduleWithCoLocationHintTest {
 			assertEquals(s8.getInstance(), s11.getInstance());
 			assertEquals(s9.getInstance(), s12.getInstance());
 			
-			assertEquals(c1.getSlot().getAllocatedSlot().getInstance(), s1.getInstance());
-			assertEquals(c2.getSlot().getAllocatedSlot().getInstance(), s2.getInstance());
-			assertEquals(c3.getSlot().getAllocatedSlot().getInstance(), s3.getInstance());
-			assertEquals(c4.getSlot().getAllocatedSlot().getInstance(), s4.getInstance());
-			assertEquals(c5.getSlot().getAllocatedSlot().getInstance(), s8.getInstance());
-			assertEquals(c6.getSlot().getAllocatedSlot().getInstance(), s9.getInstance());
+			assertEquals(c1.getLocation(), s1.getInstance());
+			assertEquals(c2.getLocation(), s2.getInstance());
+			assertEquals(c3.getLocation(), s3.getInstance());
+			assertEquals(c4.getLocation(), s4.getInstance());
+			assertEquals(c5.getLocation(), s8.getInstance());
+			assertEquals(c6.getLocation(), s9.getInstance());
 			
 			// check the scheduler's bookkeeping
 			assertEquals(0, scheduler.getNumberOfAvailableSlots());
 			
-			// the first assignments are unconstrained, co.-schedulings are constrained
+			// the first assignments are unconstrained, co.-scheduling is constrained
 			assertEquals(6, scheduler.getNumberOfLocalizedAssignments());
 			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
 			assertEquals(6, scheduler.getNumberOfUnconstrainedAssignments());
 			
 			// release some slots, be sure that new available ones come up
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s3.releaseSlot();
 			s4.releaseSlot();
+			s7.releaseSlot();
 			s10.releaseSlot();
-			assertEquals(1, scheduler.getNumberOfAvailableSlots());
+			s11.releaseSlot();
+			s12.releaseSlot();
+			assertTrue(scheduler.getNumberOfAvailableSlots() >= 1);
 			
 			AllocatedSlot single = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(new JobVertexID(), 0, 1)));
 			assertNotNull(single);
@@ -154,10 +173,11 @@ public class ScheduleWithCoLocationHintTest {
 			
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
 			CoLocationConstraint c1 = new CoLocationConstraint();
 			
-			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), c1));
-			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), c1));
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup, c1));
 			
 			AllocatedSlot sSolo = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 1)));
 			
@@ -167,7 +187,7 @@ public class ScheduleWithCoLocationHintTest {
 			s2.releaseSlot();
 			sSolo.releaseSlot();
 			
-			AllocatedSlot sNew = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), c1));
+			AllocatedSlot sNew = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1));
 			assertEquals(loc, sNew.getInstance());
 			
 			assertEquals(2, scheduler.getNumberOfLocalizedAssignments());
@@ -197,9 +217,10 @@ public class ScheduleWithCoLocationHintTest {
 			
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
 			CoLocationConstraint c1 = new CoLocationConstraint();
 			
-			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), c1));
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1));
 			s1.releaseSlot();
 			
 			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1)));
@@ -207,7 +228,7 @@ public class ScheduleWithCoLocationHintTest {
 			
 			
 			try {
-				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), c1));
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 1), sharingGroup, c1));
 				fail("Scheduled even though no resource was available.");
 			} catch (NoResourceAvailableException e) {
 				// expected
@@ -222,4 +243,64 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
+	
+	@Test
+	public void scheduleMixedCoLocationSlotSharing() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			JobVertexID jid3 = new JobVertexID();
+			JobVertexID jid4 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			scheduler.newInstanceAvailable(getRandomInstance(1));
+			scheduler.newInstanceAvailable(getRandomInstance(1));
+			scheduler.newInstanceAvailable(getRandomInstance(1));
+			scheduler.newInstanceAvailable(getRandomInstance(1));
+			
+			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+			
+			CoLocationConstraint clc1 = new CoLocationConstraint();
+			CoLocationConstraint clc2 = new CoLocationConstraint();
+			CoLocationConstraint clc3 = new CoLocationConstraint();
+			CoLocationConstraint clc4 = new CoLocationConstraint();
+			SlotSharingGroup shareGroup = new SlotSharingGroup();
+
+			// first wave
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), shareGroup));
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), shareGroup));
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), shareGroup));
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), shareGroup));
+			
+			// second wave
+			AllocatedSlot s21 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), shareGroup, clc1));
+			AllocatedSlot s22 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), shareGroup, clc2));
+			AllocatedSlot s23 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), shareGroup, clc3));
+			AllocatedSlot s24 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), shareGroup, clc4));
+			
+			// third wave
+			AllocatedSlot s31 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 1, 4), shareGroup, clc2));
+			AllocatedSlot s32 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 2, 4), shareGroup, clc3));
+			AllocatedSlot s33 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 3, 4), shareGroup, clc4));
+			AllocatedSlot s34 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid3, 0, 4), shareGroup, clc1));
+			
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 0, 4), shareGroup));
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 1, 4), shareGroup));
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 2, 4), shareGroup));
+			scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid4, 3, 4), shareGroup));
+			
+			assertEquals(s21.getInstance(), s34.getInstance());
+			assertEquals(s22.getInstance(), s31.getInstance());
+			assertEquals(s23.getInstance(), s32.getInstance());
+			assertEquals(s24.getInstance(), s33.getInstance());
+			
+			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(12, scheduler.getNumberOfUnconstrainedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8231b62f/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
index 9da993e..92a54b3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerSlotSharingTest.java
@@ -242,6 +242,70 @@ public class SchedulerSlotSharingTest {
 	}
 	
 	@Test
+	public void scheduleImmediatelyWithIntermediateTotallyEmptySharingGroup() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup(jid1, jid2);
+			
+			Scheduler scheduler = new Scheduler();
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			
+			// schedule 4 tasks from the first vertex group
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 4), sharingGroup));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 1, 4), sharingGroup));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 2, 4), sharingGroup));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 3, 4), sharingGroup));
+			
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s3.releaseSlot();
+			s4.releaseSlot();
+			
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+			
+			// schedule some tasks from the second ID group
+			AllocatedSlot s1_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 4), sharingGroup));
+			AllocatedSlot s2_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 1, 4), sharingGroup));
+			AllocatedSlot s3_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 2, 4), sharingGroup));
+			AllocatedSlot s4_2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 3, 4), sharingGroup));
+
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(4, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+			
+			s1_2.releaseSlot();
+			s2_2.releaseSlot();
+			s3_2.releaseSlot();
+			s4_2.releaseSlot();
+			
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+
+			// test that everything is released
+			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(8, scheduler.getNumberOfUnconstrainedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
 	public void scheduleImmediatelyWithTemprarilyEmptyVertexGroup() {
 		try {
 			JobVertexID jid1 = new JobVertexID();


[59/63] [abbrv] git commit: Fix test to be backwards compatible with Java 6

Posted by se...@apache.org.
Fix test to be backwards compatible with Java 6


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/cd699c56
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/cd699c56
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/cd699c56

Branch: refs/heads/master
Commit: cd699c562ec26d3909af08afb04de6080e4ad18f
Parents: 73ebd3e
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 15:18:12 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:02 2014 +0200

----------------------------------------------------------------------
 .../java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cd699c56/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
index ba08a9f..2da8b9f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -387,7 +387,7 @@ public class TaskManagerTest {
 	}
 	
 	public static TaskManager createTaskManager(JobManager jm) throws Exception {
-		InetAddress localhost = InetAddress.getLoopbackAddress();
+		InetAddress localhost = InetAddress.getLocalHost();
 		InetSocketAddress jmMockAddress = new InetSocketAddress(localhost, 55443);
 		
 		Configuration cfg = new Configuration();


[02/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
index 8b891a9..78c9288 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.util;
 
 import org.apache.flink.api.common.typeutils.TypeComparator;
@@ -30,7 +29,6 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
 import org.apache.flink.runtime.operators.hash.BuildFirstHashMatchIterator;
 import org.apache.flink.runtime.operators.hash.BuildSecondHashMatchIterator;
 import org.apache.flink.runtime.operators.sort.MergeMatchIterator;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/OneShotLatch.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/OneShotLatch.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/OneShotLatch.java
new file mode 100644
index 0000000..3f87040
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/OneShotLatch.java
@@ -0,0 +1,41 @@
+/**
+ * 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.taskmanager;
+
+final class OneShotLatch {
+	
+	private final Object lock = new Object();
+	
+	private boolean triggered;
+	
+	public void trigger() {
+		synchronized (lock) {
+			triggered = true;
+			lock.notifyAll();
+		}
+	}
+	
+	public void await() throws InterruptedException {
+		synchronized (lock) {
+			while (!triggered) {
+				lock.wait();
+			}
+		}
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
new file mode 100644
index 0000000..91dc5b3
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskExecutionStateTest.java
@@ -0,0 +1,72 @@
+/**
+ * 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.taskmanager;
+
+import static org.junit.Assert.*;
+
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.junit.Test;
+
+public class TaskExecutionStateTest {
+
+	@Test
+	public void testEqualsHashCode() {
+		try {
+			final JobID jid = new JobID();
+			final ExecutionAttemptID executionId = new ExecutionAttemptID();
+			final ExecutionState2 state = ExecutionState2.RUNNING;
+			final String description = "some test description";
+			
+			TaskExecutionState s1 = new TaskExecutionState(jid, executionId, state, description);
+			TaskExecutionState s2 = new TaskExecutionState(jid, executionId, state, description);
+			
+			assertEquals(s1.hashCode(), s2.hashCode());
+			assertEquals(s1, s2);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialization() {
+		try {
+			final JobID jid = new JobID();
+			final ExecutionAttemptID executionId = new ExecutionAttemptID();
+			final ExecutionState2 state = ExecutionState2.DEPLOYING;
+			final String description = "foo bar";
+			
+			TaskExecutionState original = new TaskExecutionState(jid, executionId, state, description);
+			
+			TaskExecutionState writableCopy = CommonTestUtils.createCopyWritable(original);
+			TaskExecutionState javaSerCopy = CommonTestUtils.createCopySerializable(original);
+			
+			assertEquals(original, writableCopy);
+			assertEquals(original, javaSerCopy);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
new file mode 100644
index 0000000..e7639e3
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerTest.java
@@ -0,0 +1,390 @@
+/**
+ * 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.taskmanager;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.runtime.ExecutionMode;
+import org.apache.flink.runtime.deployment.ChannelDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
+import org.apache.flink.runtime.io.network.api.RecordReader;
+import org.apache.flink.runtime.io.network.api.RecordWriter;
+import org.apache.flink.runtime.io.network.channels.ChannelID;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobmanager.JobManager;
+import org.apache.flink.runtime.types.IntegerRecord;
+import org.apache.flink.util.LogUtils;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+
+public class TaskManagerTest {
+
+	@BeforeClass
+	public static void reduceLogLevel() {
+		LogUtils.initializeDefaultTestConsoleLogger();
+	}
+	
+	@Test
+	public void testSetupTaskManager() {
+		try {
+			JobManager jobManager = getJobManagerMockBase();
+			
+			TaskManager tm = createTaskManager(jobManager);
+
+			JobID jid = new JobID();
+			JobVertexID vid = new JobVertexID();
+			ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, "TestTask", 2, 7,
+					new Configuration(), new Configuration(), TestInvokableCorrect.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(), 
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			LibraryCacheManager.register(jid, new String[0]);
+			
+			TaskOperationResult result = tm.submitTask(tdd);
+			assertTrue(result.isSuccess());
+			assertEquals(eid, result.getExecutionId());
+			assertEquals(vid, result.getVertexId());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testJobSubmissionAndCanceling() {
+		try {
+			JobManager jobManager = getJobManagerMockBase();
+			
+			TaskManager tm = createTaskManager(jobManager);
+
+			JobID jid1 = new JobID();
+			JobID jid2 = new JobID();
+			
+			JobVertexID vid1 = new JobVertexID();
+			JobVertexID vid2 = new JobVertexID();
+			
+			ExecutionAttemptID eid1 = new ExecutionAttemptID();
+			ExecutionAttemptID eid2 = new ExecutionAttemptID();
+			
+			TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid1, vid1, eid1, "TestTask1", 1, 5,
+					new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(), 
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid2, vid2, eid2, "TestTask2", 2, 7,
+					new Configuration(), new Configuration(), TestInvokableBlockingCancelable.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(), 
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			LibraryCacheManager.register(jid1, new String[0]);
+			LibraryCacheManager.register(jid2, new String[0]);
+			assertNotNull(LibraryCacheManager.getClassLoader(jid1));
+			assertNotNull(LibraryCacheManager.getClassLoader(jid2));
+			
+			TaskOperationResult result1 = tm.submitTask(tdd1);
+			TaskOperationResult result2 = tm.submitTask(tdd2);
+			
+			assertTrue(result1.isSuccess());
+			assertTrue(result2.isSuccess());
+			assertEquals(eid1, result1.getExecutionId());
+			assertEquals(eid2, result2.getExecutionId());
+			assertEquals(vid1, result1.getVertexId());
+			assertEquals(vid2, result2.getVertexId());
+			
+			Map<ExecutionAttemptID, Task> tasks = tm.getAllRunningTasks();
+			assertEquals(2, tasks.size());
+			
+			Task t1 = tasks.get(eid1);
+			Task t2 = tasks.get(eid2);
+			assertNotNull(t1);
+			assertNotNull(t2);
+			
+			assertEquals(ExecutionState2.RUNNING, t1.getExecutionState());
+			assertEquals(ExecutionState2.RUNNING, t2.getExecutionState());
+			
+			// cancel one task
+			assertTrue(tm.cancelTask(vid1, 1, eid1).isSuccess());
+			t1.getEnvironment().getExecutingThread().join();
+			assertEquals(ExecutionState2.CANCELED, t1.getExecutionState());
+			
+			tasks = tm.getAllRunningTasks();
+			assertEquals(1, tasks.size());
+			
+			// try to cancel a non existing task
+			assertFalse(tm.cancelTask(vid1, 1, eid1).isSuccess());
+			
+			// cancel the second task
+			assertTrue(tm.cancelTask(vid2, 2, eid2).isSuccess());
+			t2.getEnvironment().getExecutingThread().join();
+			assertEquals(ExecutionState2.CANCELED, t2.getExecutionState());
+			
+			tasks = tm.getAllRunningTasks();
+			assertEquals(0, tasks.size());
+			
+			// the class loaders should be de-registered
+			assertNull(LibraryCacheManager.getClassLoader(jid1));
+			assertNull(LibraryCacheManager.getClassLoader(jid2));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testGateChannelEdgeMismatch() {
+		try {
+			JobManager jobManager = getJobManagerMockBase();
+			
+			TaskManager tm = createTaskManager(jobManager);
+
+			JobID jid = new JobID();;
+			
+			JobVertexID vid1 = new JobVertexID();
+			JobVertexID vid2 = new JobVertexID();
+			
+			ExecutionAttemptID eid1 = new ExecutionAttemptID();
+			ExecutionAttemptID eid2 = new ExecutionAttemptID();
+			
+			TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1,
+					new Configuration(), new Configuration(), Sender.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7,
+					new Configuration(), new Configuration(), Receiver.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			LibraryCacheManager.register(jid, new String[0]);
+			LibraryCacheManager.register(jid, new String[0]);
+			assertNotNull(LibraryCacheManager.getClassLoader(jid));
+			
+			assertFalse(tm.submitTask(tdd1).isSuccess());
+			assertFalse(tm.submitTask(tdd2).isSuccess());
+			
+			Map<ExecutionAttemptID, Task> tasks = tm.getAllRunningTasks();
+			assertEquals(0, tasks.size());
+			
+			// the class loaders should be de-registered
+			assertNull(LibraryCacheManager.getClassLoader(jid));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testRunJobWithForwardChannel() {
+		try {
+			JobID jid = new JobID();
+			
+			JobVertexID vid1 = new JobVertexID();
+			JobVertexID vid2 = new JobVertexID();
+			
+			ExecutionAttemptID eid1 = new ExecutionAttemptID();
+			ExecutionAttemptID eid2 = new ExecutionAttemptID();
+			
+			ChannelID senderId = new ChannelID();
+			ChannelID receiverId = new ChannelID();
+			
+			JobManager jobManager = getJobManagerMockBase();
+			when(jobManager.lookupConnectionInfo(Matchers.any(InstanceConnectionInfo.class), Matchers.eq(jid), Matchers.eq(senderId)))
+				.thenReturn(ConnectionInfoLookupResponse.createReceiverFoundAndReady(receiverId));
+			
+			TaskManager tm = createTaskManager(jobManager);
+			
+			ChannelDeploymentDescriptor cdd = new ChannelDeploymentDescriptor(senderId, receiverId);
+			
+			TaskDeploymentDescriptor tdd1 = new TaskDeploymentDescriptor(jid, vid1, eid1, "Sender", 0, 1,
+					new Configuration(), new Configuration(), Sender.class.getName(),
+					Collections.singletonList(new GateDeploymentDescriptor(Collections.singletonList(cdd))), 
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			TaskDeploymentDescriptor tdd2 = new TaskDeploymentDescriptor(jid, vid2, eid2, "Receiver", 2, 7,
+					new Configuration(), new Configuration(), Receiver.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					Collections.singletonList(new GateDeploymentDescriptor(Collections.singletonList(cdd))),
+					new String[0], 0);
+			
+			// register the job twice (for two tasks) at the lib cache
+			LibraryCacheManager.register(jid, new String[0]);
+			LibraryCacheManager.register(jid, new String[0]);
+			assertNotNull(LibraryCacheManager.getClassLoader(jid));
+			
+			// deploy sender before receiver, so the target is online when the sender requests the connection info
+			TaskOperationResult result2 = tm.submitTask(tdd2);
+			TaskOperationResult result1 = tm.submitTask(tdd1);
+			
+			assertTrue(result1.isSuccess());
+			assertTrue(result2.isSuccess());
+			assertEquals(eid1, result1.getExecutionId());
+			assertEquals(eid2, result2.getExecutionId());
+			assertEquals(vid1, result1.getVertexId());
+			assertEquals(vid2, result2.getVertexId());
+			
+			Map<ExecutionAttemptID, Task> tasks = tm.getAllRunningTasks();
+			
+			Task t1 = tasks.get(eid1);
+			Task t2 = tasks.get(eid2);
+			
+			// wait until the tasks are done
+			if (t1 != null) {
+				t1.getEnvironment().getExecutingThread().join();
+			}
+			if (t2 != null) {
+				t2.getEnvironment().getExecutingThread().join();
+			}
+			
+			assertEquals(ExecutionState2.FINISHED, t1.getExecutionState());
+			assertEquals(ExecutionState2.FINISHED, t2.getExecutionState());
+			
+			tasks = tm.getAllRunningTasks();
+			assertEquals(0, tasks.size());
+			
+			// the class loaders should be de-registered
+			assertNull(LibraryCacheManager.getClassLoader(jid));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static JobManager getJobManagerMockBase() {
+		JobManager jm = mock(JobManager.class);
+		
+		final InstanceID iid = new InstanceID();
+		
+		when(jm.registerTaskManager(Matchers.any(InstanceConnectionInfo.class), Matchers.any(HardwareDescription.class), Matchers.anyInt()))
+			.thenReturn(iid);
+		
+		when(jm.sendHeartbeat(iid)).thenReturn(true);
+		
+		return jm;
+	}
+	
+	public static TaskManager createTaskManager(JobManager jm) throws Exception {
+		InetAddress localhost = InetAddress.getLoopbackAddress();
+		InetSocketAddress jmMockAddress = new InetSocketAddress(localhost, 55443);
+		
+		Configuration cfg = new Configuration();
+		cfg.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 10);
+		GlobalConfiguration.includeConfiguration(cfg);
+		
+		return new TaskManager(ExecutionMode.LOCAL, jm, jm, jm, jm, jmMockAddress, localhost);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static final class TestInvokableCorrect extends AbstractInvokable {
+
+		@Override
+		public void registerInputOutput() {}
+
+		@Override
+		public void invoke() {}
+	}
+	
+	public static final class TestInvokableBlockingCancelable extends AbstractInvokable {
+
+		@Override
+		public void registerInputOutput() {}
+
+		@Override
+		public void invoke() throws Exception {
+			Object o = new Object();
+			synchronized (o) {
+				o.wait();
+			}
+		}
+	}
+	
+	public static final class Sender extends AbstractInvokable {
+
+		private RecordWriter<IntegerRecord> writer;
+		
+		@Override
+		public void registerInputOutput() {
+			writer = new RecordWriter<IntegerRecord>(this);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			writer.initializeSerializers();
+			writer.emit(new IntegerRecord(42));
+			writer.emit(new IntegerRecord(1337));
+			writer.flush();
+		}
+	}
+	
+	public static final class Receiver extends AbstractInvokable {
+
+		private RecordReader<IntegerRecord> reader;
+		
+		@Override
+		public void registerInputOutput() {
+			reader = new RecordReader<IntegerRecord>(this, IntegerRecord.class);
+		}
+
+		@Override
+		public void invoke() throws Exception {
+			IntegerRecord i1 = reader.next();
+			IntegerRecord i2 = reader.next();
+			IntegerRecord i3 = reader.next();
+			
+			if (i1.getValue() != 42 || i2.getValue() != 1337 || i3 != null) {
+				throw new Exception("Wrong Data Received");
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
new file mode 100644
index 0000000..d225483
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -0,0 +1,346 @@
+/**
+ * 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.taskmanager;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+import java.util.Collections;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.deployment.GateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.execution.RuntimeEnvironment;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
+import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.protocols.AccumulatorProtocol;
+import org.apache.flink.util.ExceptionUtils;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+public class TaskTest {
+
+	@Test
+	public void testTaskStates() {
+		try {
+			final JobID jid = new JobID();
+			final JobVertexID vid = new JobVertexID();
+			final ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			final TaskManager taskManager = mock(TaskManager.class);
+			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
+			
+			Task task = new Task(jid, vid, 2, 7, eid, "TestTask", taskManager);
+			task.setEnvironment(env);
+			
+			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			
+			// cancel
+			task.cancelExecution();
+			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
+			
+			// cannot go into running or finished state
+			
+			assertFalse(task.startExecution());
+			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
+			
+			assertFalse(task.markAsFinished());
+			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
+			
+			task.markFailed(new Exception("test"));
+			assertTrue(ExecutionState2.CANCELED == task.getExecutionState());
+			
+			verify(taskManager, times(1)).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELED, null);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTaskStartFinish() {
+		try {
+			final JobID jid = new JobID();
+			final JobVertexID vid = new JobVertexID();
+			final ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			final TaskManager taskManager = mock(TaskManager.class);
+			
+			
+			final Task task = new Task(jid, vid, 2, 7, eid, "TestTask", taskManager);
+			
+			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
+			
+			Thread operation = new Thread() {
+				public void run() {
+					try {
+						assertTrue(task.markAsFinished());
+					}
+					catch (Throwable t) {
+						error.set(t);
+					}
+				}
+			};
+			
+			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
+			when(env.getExecutingThread()).thenReturn(operation);
+			
+			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			
+			// start the execution
+			task.setEnvironment(env);
+			task.startExecution();
+			
+			// wait for the execution to be finished
+			operation.join();
+			
+			if (error.get() != null) {
+				ExceptionUtils.rethrow(error.get());
+			}
+			
+			assertEquals(ExecutionState2.FINISHED, task.getExecutionState());
+			
+			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState2.FINISHED, null);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTaskFailesInRunning() {
+		try {
+			final JobID jid = new JobID();
+			final JobVertexID vid = new JobVertexID();
+			final ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			final TaskManager taskManager = mock(TaskManager.class);
+			
+			final Task task = new Task(jid, vid, 2, 7, eid, "TestTask", taskManager);
+			
+			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
+			
+			Thread operation = new Thread() {
+				public void run() {
+					try {
+						task.markFailed(new Exception("test exception message"));
+					}
+					catch (Throwable t) {
+						error.set(t);
+					}
+				}
+			};
+			
+			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
+			when(env.getExecutingThread()).thenReturn(operation);
+			
+			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			
+			// start the execution
+			task.setEnvironment(env);
+			task.startExecution();
+			
+			// wait for the execution to be finished
+			operation.join();
+			
+			if (error.get() != null) {
+				ExceptionUtils.rethrow(error.get());
+			}
+			
+			// make sure the final state is correct and the task manager knows the changes
+			assertEquals(ExecutionState2.FAILED, task.getExecutionState());
+			verify(taskManager).notifyExecutionStateChange(Matchers.eq(jid), Matchers.eq(eid), Matchers.eq(ExecutionState2.FAILED), Matchers.anyString());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTaskCanceledInRunning() {
+		try {
+			final JobID jid = new JobID();
+			final JobVertexID vid = new JobVertexID();
+			final ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			final TaskManager taskManager = mock(TaskManager.class);
+			
+			final Task task = new Task(jid, vid, 2, 7, eid, "TestTask", taskManager);
+			
+			final AtomicReference<Throwable> error = new AtomicReference<Throwable>();
+			
+			// latches to create a deterministic order of events
+			final OneShotLatch toRunning = new OneShotLatch();
+			final OneShotLatch afterCanceling = new OneShotLatch();
+			
+			Thread operation = new Thread() {
+				public void run() {
+					try {
+						toRunning.trigger();
+						afterCanceling.await();
+						assertFalse(task.markAsFinished());
+						task.cancelingDone();
+					}
+					catch (Throwable t) {
+						error.set(t);
+					}
+				}
+			};
+			
+			final RuntimeEnvironment env = mock(RuntimeEnvironment.class);
+			when(env.getExecutingThread()).thenReturn(operation);
+			
+			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			
+			// start the execution
+			task.setEnvironment(env);
+			task.startExecution();
+			
+			toRunning.await();
+			task.cancelExecution();
+			afterCanceling.trigger();
+			
+			// wait for the execution to be finished
+			operation.join();
+			
+			if (error.get() != null) {
+				ExceptionUtils.rethrow(error.get());
+			}
+			
+			// make sure the final state is correct and the task manager knows the changes
+			assertEquals(ExecutionState2.CANCELED, task.getExecutionState());
+			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELED, null);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTaskWithEnvironment() {
+		try {
+			final JobID jid = new JobID();
+			final JobVertexID vid = new JobVertexID();
+			final ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			final TaskManager taskManager = mock(TaskManager.class);
+			
+			TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, "TestTask", 2, 7,
+					new Configuration(), new Configuration(), TestInvokableCorrect.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(), 
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			Task task = new Task(jid, vid, 2, 7, eid, "TestTask", taskManager);
+			
+			RuntimeEnvironment env = new RuntimeEnvironment(task, tdd, getClass().getClassLoader(),
+					mock(MemoryManager.class), mock(IOManager.class), mock(InputSplitProvider.class),
+					mock(AccumulatorProtocol.class));
+			
+			task.setEnvironment(env);
+			
+			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			
+			task.startExecution();
+			task.getEnvironment().getExecutingThread().join();
+			
+			assertEquals(ExecutionState2.FINISHED, task.getExecutionState());
+			
+			verify(taskManager).notifyExecutionStateChange(jid, eid, ExecutionState2.FINISHED, null);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testTaskWithEnvironmentAndException() {
+		try {
+			final JobID jid = new JobID();
+			final JobVertexID vid = new JobVertexID();
+			final ExecutionAttemptID eid = new ExecutionAttemptID();
+			
+			final TaskManager taskManager = mock(TaskManager.class);
+			
+			TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jid, vid, eid, "TestTask", 2, 7,
+					new Configuration(), new Configuration(), TestInvokableWithException.class.getName(),
+					Collections.<GateDeploymentDescriptor>emptyList(), 
+					Collections.<GateDeploymentDescriptor>emptyList(),
+					new String[0], 0);
+			
+			Task task = new Task(jid, vid, 2, 7, eid, "TestTask", taskManager);
+			
+			RuntimeEnvironment env = new RuntimeEnvironment(task, tdd, getClass().getClassLoader(),
+					mock(MemoryManager.class), mock(IOManager.class), mock(InputSplitProvider.class),
+					mock(AccumulatorProtocol.class));
+			
+			task.setEnvironment(env);
+			
+			assertEquals(ExecutionState2.DEPLOYING, task.getExecutionState());
+			
+			task.startExecution();
+			task.getEnvironment().getExecutingThread().join();
+			
+			assertEquals(ExecutionState2.FAILED, task.getExecutionState());
+			
+			verify(taskManager).notifyExecutionStateChange(Matchers.eq(jid), Matchers.eq(eid), Matchers.eq(ExecutionState2.FAILED), Matchers.anyString());
+			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELING, null);
+			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState2.CANCELED, null);
+			verify(taskManager, times(0)).notifyExecutionStateChange(jid, eid, ExecutionState2.FINISHED, null);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static final class TestInvokableCorrect extends AbstractInvokable {
+
+		@Override
+		public void registerInputOutput() {}
+
+		@Override
+		public void invoke() {}
+	}
+	
+	public static final class TestInvokableWithException extends AbstractInvokable {
+
+		@Override
+		public void registerInputOutput() {}
+
+		@Override
+		public void invoke() throws Exception {
+			throw new Exception("test exception");
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/DoubleSourceTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/DoubleSourceTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/DoubleSourceTask.java
deleted file mode 100644
index c98536e..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/DoubleSourceTask.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/**
- * 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.testutils.tasks;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.fs.LineReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-
-public class DoubleSourceTask extends AbstractInvokable {
-
-	private RecordWriter<StringRecord> output1 = null;
-
-	private RecordWriter<StringRecord> output2 = null;
-
-	@Override
-	public void invoke() throws Exception {
-		this.output1.initializeSerializers();
-		this.output2.initializeSerializers();
-
-		final Iterator<FileInputSplit> splitIterator = getInputSplits();
-
-		while (splitIterator.hasNext()) {
-
-			final FileInputSplit split = splitIterator.next();
-
-			final long start = split.getStart();
-			final long length = split.getLength();
-
-			final FileSystem fs = FileSystem.get(split.getPath().toUri());
-
-			final FSDataInputStream fdis = fs.open(split.getPath());
-
-			final LineReader lineReader = new LineReader(fdis, start, length, (1024 * 1024));
-
-			byte[] line = lineReader.readLine();
-
-			while (line != null) {
-
-				// Create a string object from the data read
-				StringRecord str = new StringRecord();
-				str.set(line);
-
-				// Send out string
-				output1.emit(str);
-				output2.emit(str);
-
-				line = lineReader.readLine();
-			}
-
-			// Close the stream;
-			lineReader.close();
-		}
-
-		this.output1.flush();
-		this.output2.flush();
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.output1 = new RecordWriter<StringRecord>(this);
-		this.output2 = new RecordWriter<StringRecord>(this);
-	}
-
-	private Iterator<FileInputSplit> getInputSplits() {
-
-		final InputSplitProvider provider = getEnvironment().getInputSplitProvider();
-
-		return new Iterator<FileInputSplit>() {
-
-			private FileInputSplit nextSplit;
-			
-			private boolean exhausted;
-
-			@Override
-			public boolean hasNext() {
-				if (exhausted) {
-					return false;
-				}
-				
-				if (nextSplit != null) {
-					return true;
-				}
-				
-				FileInputSplit split = (FileInputSplit) provider.getNextInputSplit();
-				
-				if (split != null) {
-					this.nextSplit = split;
-					return true;
-				}
-				else {
-					exhausted = true;
-					return false;
-				}
-			}
-
-			@Override
-			public FileInputSplit next() {
-				if (this.nextSplit == null && !hasNext()) {
-					throw new NoSuchElementException();
-				}
-
-				final FileInputSplit tmp = this.nextSplit;
-				this.nextSplit = null;
-				return tmp;
-			}
-
-			@Override
-			public void remove() {
-				throw new UnsupportedOperationException();
-			}
-		};
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineReader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineReader.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineReader.java
deleted file mode 100644
index 500f9a1..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineReader.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/**
- * 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.testutils.tasks;
-
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.fs.LineReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-
-/**
- * A file line reader reads the associated file input splits line by line and outputs the lines as string records.
- * 
- */
-public class FileLineReader extends AbstractInvokable {
-
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void invoke() throws Exception {
-
-		output.initializeSerializers();
-
-		final Iterator<FileInputSplit> splitIterator = getInputSplits();
-
-		while (splitIterator.hasNext()) {
-
-			final FileInputSplit split = splitIterator.next();
-
-			long start = split.getStart();
-			long length = split.getLength();
-
-			final FileSystem fs = FileSystem.get(split.getPath().toUri());
-
-			final FSDataInputStream fdis = fs.open(split.getPath());
-
-			final LineReader lineReader = new LineReader(fdis, start, length, (1024 * 1024));
-
-			byte[] line = lineReader.readLine();
-
-			while (line != null) {
-
-				// Create a string object from the data read
-				StringRecord str = new StringRecord();
-				str.set(line);
-
-				// Send out string
-				output.emit(str);
-
-				line = lineReader.readLine();
-			}
-
-			// Close the stream;
-			lineReader.close();
-		}
-
-		this.output.flush();
-	}
-
-	@Override
-	public void registerInputOutput() {
-		output = new RecordWriter<StringRecord>(this);
-	}
-	
-	private Iterator<FileInputSplit> getInputSplits() {
-
-		final InputSplitProvider provider = getEnvironment().getInputSplitProvider();
-
-		return new Iterator<FileInputSplit>() {
-
-			private FileInputSplit nextSplit;
-			
-			private boolean exhausted;
-
-			@Override
-			public boolean hasNext() {
-				if (exhausted) {
-					return false;
-				}
-				
-				if (nextSplit != null) {
-					return true;
-				}
-				
-				FileInputSplit split = (FileInputSplit) provider.getNextInputSplit();
-				
-				if (split != null) {
-					this.nextSplit = split;
-					return true;
-				}
-				else {
-					exhausted = true;
-					return false;
-				}
-			}
-
-			@Override
-			public FileInputSplit next() {
-				if (this.nextSplit == null && !hasNext()) {
-					throw new NoSuchElementException();
-				}
-
-				final FileInputSplit tmp = this.nextSplit;
-				this.nextSplit = null;
-				return tmp;
-			}
-
-			@Override
-			public void remove() {
-				throw new UnsupportedOperationException();
-			}
-		};
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineWriter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineWriter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineWriter.java
deleted file mode 100644
index 529ae06..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/FileLineWriter.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * 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.testutils.tasks;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * A file line writer reads string records its input gate and writes them to the associated output file.
- * 
- */
-public class FileLineWriter extends AbstractInvokable {
-	/**
-	 * The record reader through which incoming string records are received.
-	 */
-	private RecordReader<StringRecord> input = null;
-
-
-	@Override
-	public void invoke() throws Exception {
-
-		final Configuration conf = getEnvironment().getTaskConfiguration();
-		final String outputPathString = conf.getString(JobFileOutputVertex.PATH_PROPERTY, null);
-		
-		Path outputPath = new Path(outputPathString);
-
-		FileSystem fs = FileSystem.get(outputPath.toUri());
-		if (fs.exists(outputPath)) {
-			FileStatus status = fs.getFileStatus(outputPath);
-
-			if (status.isDir()) {
-				outputPath = new Path(outputPath.toUri().toString() + "/file_" + getIndexInSubtaskGroup() + ".txt");
-			}
-		}
-
-		final FSDataOutputStream outputStream = fs.create(outputPath, true);
-
-		while (this.input.hasNext()) {
-
-			StringRecord record = this.input.next();
-			byte[] recordByte = (record.toString() + "\r\n").getBytes();
-			outputStream.write(recordByte, 0, recordByte.length);
-		}
-
-		outputStream.close();
-
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileInputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileInputVertex.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileInputVertex.java
deleted file mode 100644
index 944e8f1..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileInputVertex.java
+++ /dev/null
@@ -1,259 +0,0 @@
-/**
- * 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.testutils.tasks;
-
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.jobgraph.AbstractJobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-
-
-public final class JobFileInputVertex extends AbstractJobInputVertex {
-
-	/**
-	 * The fraction that the last split may be larger than the others.
-	 */
-	private static final float MAX_SPLIT_SIZE_DISCREPANCY = 1.1f;
-	
-	/**
-	 * The path pointing to the input file/directory.
-	 */
-	private Path path;
-
-
-	public JobFileInputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-	
-	/**
-	 * Creates a new job file input vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file input vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileInputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileInputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-
-	/**
-	 * Sets the path of the file the job file input vertex's task should read from.
-	 * 
-	 * @param path
-	 *        the path of the file the job file input vertex's task should read from
-	 */
-	public void setFilePath(final Path path) {
-		this.path = path;
-	}
-
-	/**
-	 * Returns the path of the file the job file input vertex's task should read from.
-	 * 
-	 * @return the path of the file the job file input vertex's task should read from or <code>null</code> if no path
-	 *         has yet been set
-	 */
-	public Path getFilePath() {
-		return this.path;
-	}
-
-	@Override
-	public void read(final DataInputView in) throws IOException {
-		super.read(in);
-
-		// Read path of the input file
-		final boolean isNotNull = in.readBoolean();
-		if (isNotNull) {
-			this.path = new Path();
-			this.path.read(in);
-		}
-	}
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-		super.write(out);
-
-		// Write out the path of the input file
-		if (this.path == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			this.path.write(out);
-		}
-	}
-
-	// --------------------------------------------------------------------------------------------
-
-
-	@Override
-	public InputSplit[] getInputSplits(int minNumSplits) throws Exception {
-		final Path path = this.path;
-		final List<FileInputSplit> inputSplits = new ArrayList<FileInputSplit>();
-
-		// get all the files that are involved in the splits
-		final List<FileStatus> files = new ArrayList<FileStatus>();
-		long totalLength = 0;
-
-		final FileSystem fs = path.getFileSystem();
-		final FileStatus pathFile = fs.getFileStatus(path);
-
-		if (pathFile.isDir()) {
-			// input is directory. list all contained files
-			final FileStatus[] dir = fs.listStatus(path);
-			for (int i = 0; i < dir.length; i++) {
-				if (!dir[i].isDir()) {
-					files.add(dir[i]);
-					totalLength += dir[i].getLen();
-				}
-			}
-
-		} else {
-			files.add(pathFile);
-			totalLength += pathFile.getLen();
-		}
-
-		final long minSplitSize = 1;
-		final long maxSplitSize = (minNumSplits < 1) ? Long.MAX_VALUE : (totalLength / minNumSplits +
-					(totalLength % minNumSplits == 0 ? 0 : 1));
-
-		// now that we have the files, generate the splits
-		int splitNum = 0;
-		for (final FileStatus file : files) {
-
-			final long len = file.getLen();
-			final long blockSize = file.getBlockSize();
-
-			final long splitSize = Math.max(minSplitSize, Math.min(maxSplitSize, blockSize));
-			final long halfSplit = splitSize >>> 1;
-
-			final long maxBytesForLastSplit = (long) (splitSize * MAX_SPLIT_SIZE_DISCREPANCY);
-
-			if (len > 0) {
-
-				// get the block locations and make sure they are in order with respect to their offset
-				final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, len);
-				Arrays.sort(blocks);
-
-				long bytesUnassigned = len;
-				long position = 0;
-
-				int blockIndex = 0;
-
-				while (bytesUnassigned > maxBytesForLastSplit) {
-					// get the block containing the majority of the data
-					blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex);
-					// create a new split
-					final FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), position, splitSize,
-						blocks[blockIndex]
-							.getHosts());
-					inputSplits.add(fis);
-
-					// adjust the positions
-					position += splitSize;
-					bytesUnassigned -= splitSize;
-				}
-
-				// assign the last split
-				if (bytesUnassigned > 0) {
-					blockIndex = getBlockIndexForPosition(blocks, position, halfSplit, blockIndex);
-					final FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), position,
-						bytesUnassigned,
-						blocks[blockIndex].getHosts());
-					inputSplits.add(fis);
-				}
-			} else {
-				// special case with a file of zero bytes size
-				final BlockLocation[] blocks = fs.getFileBlockLocations(file, 0, 0);
-				String[] hosts;
-				if (blocks.length > 0) {
-					hosts = blocks[0].getHosts();
-				} else {
-					hosts = new String[0];
-				}
-				final FileInputSplit fis = new FileInputSplit(splitNum++, file.getPath(), 0, 0, hosts);
-				inputSplits.add(fis);
-			}
-		}
-
-		return inputSplits.toArray(new FileInputSplit[inputSplits.size()]);
-	}
-
-	/**
-	 * Retrieves the index of the <tt>BlockLocation</tt> that contains the part of the file described by the given
-	 * offset.
-	 * 
-	 * @param blocks
-	 *        The different blocks of the file. Must be ordered by their offset.
-	 * @param offset
-	 *        The offset of the position in the file.
-	 * @param startIndex
-	 *        The earliest index to look at.
-	 * @return The index of the block containing the given position.
-	 */
-	private final int getBlockIndexForPosition(final BlockLocation[] blocks, final long offset,
-			final long halfSplitSize, final int startIndex) {
-		
-		// go over all indexes after the startIndex
-		for (int i = startIndex; i < blocks.length; i++) {
-			long blockStart = blocks[i].getOffset();
-			long blockEnd = blockStart + blocks[i].getLength();
-
-			if (offset >= blockStart && offset < blockEnd) {
-				// got the block where the split starts
-				// check if the next block contains more than this one does
-				if (i < blocks.length - 1 && blockEnd - offset < halfSplitSize) {
-					return i + 1;
-				} else {
-					return i;
-				}
-			}
-		}
-		throw new IllegalArgumentException("The given offset is not contained in the any block.");
-	}
-
-
-	@Override
-	public Class<FileInputSplit> getInputSplitType() {
-		return FileInputSplit.class;
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileOutputVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileOutputVertex.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileOutputVertex.java
deleted file mode 100644
index 6f8ce85..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/tasks/JobFileOutputVertex.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * 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.testutils.tasks;
-
-import java.io.IOException;
-
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.jobgraph.AbstractJobOutputVertex;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
-
-
-public class JobFileOutputVertex extends AbstractJobOutputVertex {
-
-	public static final String PATH_PROPERTY = "outputPath";
-	
-	/**
-	 * The path pointing to the output file/directory.
-	 */
-	private Path path;
-
-
-	public JobFileOutputVertex(String name, JobVertexID id, JobGraph jobGraph) {
-		super(name, id, jobGraph);
-	}
-	
-	/**
-	 * Creates a new job file output vertex with the specified name.
-	 * 
-	 * @param name
-	 *        the name of the new job file output vertex
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileOutputVertex(String name, JobGraph jobGraph) {
-		this(name, null, jobGraph);
-	}
-
-	/**
-	 * Creates a new job file input vertex.
-	 * 
-	 * @param jobGraph
-	 *        the job graph this vertex belongs to
-	 */
-	public JobFileOutputVertex(JobGraph jobGraph) {
-		this(null, jobGraph);
-	}
-
-	/**
-	 * Sets the path of the file the job file input vertex's task should write to.
-	 * 
-	 * @param path
-	 *        the path of the file the job file input vertex's task should write to
-	 */
-	public void setFilePath(Path path) {
-		this.path = path;
-		getConfiguration().setString(PATH_PROPERTY, path.toString());
-	}
-
-	/**
-	 * Returns the path of the file the job file output vertex's task should write to.
-	 * 
-	 * @return the path of the file the job file output vertex's task should write to or <code>null</code> if no path
-	 *         has yet been set
-	 */
-	public Path getFilePath() {
-		return this.path;
-	}
-
-	@Override
-	public void read(final DataInputView in) throws IOException {
-		super.read(in);
-
-		// Read path of the input file
-		boolean isNotNull = in.readBoolean();
-		if (isNotNull) {
-			this.path = new Path();
-			this.path.read(in);
-		}
-	}
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-		super.write(out);
-
-		// Write out the path of the input file
-		if (this.path == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			this.path.write(out);
-		}
-	}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/util/DelegatingConfigurationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DelegatingConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DelegatingConfigurationTest.java
new file mode 100644
index 0000000..f55f1cf
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DelegatingConfigurationTest.java
@@ -0,0 +1,93 @@
+/**
+ * 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.util;
+
+import static org.junit.Assert.assertTrue;
+
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Comparator;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.operators.util.TaskConfig.DelegatingConfiguration;
+import org.junit.Test;
+
+
+public class DelegatingConfigurationTest {
+
+	/**
+	 * http://stackoverflow.com/questions/22225663/checking-in-a-unit-test-whether-all-methods-are-delegated
+	 */
+	@Test
+	public void testIfDelegatesImplementAllMethods() throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
+
+		Comparator<Method> methodComparator = new Comparator<Method>() {
+			@Override
+			public int compare(Method o1, Method o2) {
+				String o1Str = o1.getName() + typeParamToString(o1.getParameterTypes());
+				String o2Str = o2.getName() + typeParamToString(o2.getParameterTypes());
+				return o1Str.compareTo( o2Str ); 
+			}
+
+			private String typeParamToString(Class<?>[] classes) {
+				String str = "";
+				for(Object t : classes) {
+					str += t.toString();
+				}
+				return str;
+			}
+		};
+		
+		// For each method in the Configuration class...
+		Method[] confMethods = Configuration.class.getDeclaredMethods();
+		Method[] delegateMethods = DelegatingConfiguration.class.getDeclaredMethods();
+		Arrays.sort(confMethods, methodComparator);
+		Arrays.sort(delegateMethods, methodComparator);
+		match : for (Method configurationMethod : confMethods) {
+			boolean hasMethod = false;
+			if(!Modifier.isPublic(configurationMethod.getModifiers()) ) {
+				continue;
+			}
+			// Find matching method in wrapper class and call it
+			mismatch: for (Method wrapperMethod : delegateMethods) {
+				if (configurationMethod.getName().equals(wrapperMethod.getName())) {
+					
+					// Get parameters for method
+					Class<?>[] wrapperMethodParams = wrapperMethod.getParameterTypes();
+					Class<?>[] configMethodParams = configurationMethod.getParameterTypes();
+					if(wrapperMethodParams.length != configMethodParams.length) {
+						System.err.println("Length");
+						break mismatch;
+					}
+					for(int i = 0; i < wrapperMethodParams.length; i++) {
+						if(wrapperMethodParams[i] != configMethodParams[i]) {
+							break mismatch;
+						}
+					}
+					hasMethod = true;
+					break match;
+				}
+			}
+			assertTrue("Foo method '" + configurationMethod.getName() + "' has not been wrapped correctly in DelegatingConfiguration wrapper", hasMethod);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestDelegatingConfiguration.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestDelegatingConfiguration.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestDelegatingConfiguration.java
deleted file mode 100644
index 5fdf433..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestDelegatingConfiguration.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * 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.util;
-
-import static org.junit.Assert.assertTrue;
-
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
-import java.util.Arrays;
-import java.util.Comparator;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.operators.util.TaskConfig.DelegatingConfiguration;
-import org.junit.Test;
-
-
-public class TestDelegatingConfiguration {
-
-	/**
-	 * http://stackoverflow.com/questions/22225663/checking-in-a-unit-test-whether-all-methods-are-delegated
-	 */
-	@Test
-	public void testIfDelegatesImplementAllMethods() throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
-
-		Comparator<Method> methodComparator = new Comparator<Method>() {
-			@Override
-			public int compare(Method o1, Method o2) {
-				String o1Str = o1.getName() + typeParamToString(o1.getParameterTypes());
-				String o2Str = o2.getName() + typeParamToString(o2.getParameterTypes());
-				return o1Str.compareTo( o2Str ); 
-			}
-
-			private String typeParamToString(Class<?>[] classes) {
-				String str = "";
-				for(Object t : classes) {
-					str += t.toString();
-				}
-				return str;
-			}
-		};
-		
-		// For each method in the Configuration class...
-		Method[] confMethods = Configuration.class.getDeclaredMethods();
-		Method[] delegateMethods = DelegatingConfiguration.class.getDeclaredMethods();
-		Arrays.sort(confMethods, methodComparator);
-		Arrays.sort(delegateMethods, methodComparator);
-		match : for (Method configurationMethod : confMethods) {
-			boolean hasMethod = false;
-			if(!Modifier.isPublic(configurationMethod.getModifiers()) ) {
-				continue;
-			}
-			// Find matching method in wrapper class and call it
-			mismatch: for (Method wrapperMethod : delegateMethods) {
-				if (configurationMethod.getName().equals(wrapperMethod.getName())) {
-					
-					// Get parameters for method
-					Class<?>[] wrapperMethodParams = wrapperMethod.getParameterTypes();
-					Class<?>[] configMethodParams = configurationMethod.getParameterTypes();
-					if(wrapperMethodParams.length != configMethodParams.length) {
-						System.err.println("Length");
-						break mismatch;
-					}
-					for(int i = 0; i < wrapperMethodParams.length; i++) {
-						if(wrapperMethodParams[i] != configMethodParams[i]) {
-							break mismatch;
-						}
-					}
-					hasMethod = true;
-					break match;
-				}
-			}
-			assertTrue("Foo method '" + configurationMethod.getName() + "' has not been wrapped correctly in DelegatingConfiguration wrapper", hasMethod);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
index a409222..33112af 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarsNepheleITCase.java
@@ -227,9 +227,9 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	// -------------------------------------------------------------------------------------------------------------
 
 	@SuppressWarnings("unchecked")
-	private static InputFormatInputVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		CsvInputFormat pointsInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class, LongValue.class, LongValue.class);
-		InputFormatInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "Input[Points]", jobGraph, numSubTasks);
+		InputFormatVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "Input[Points]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
@@ -241,9 +241,9 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 	}
 
 	@SuppressWarnings("unchecked")
-	private static InputFormatInputVertex createModelsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatVertex createModelsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		CsvInputFormat modelsInFormat = new CsvInputFormat(' ', LongValue.class, LongValue.class, LongValue.class, LongValue.class);
-		InputFormatInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, pointsPath, "Input[Models]", jobGraph, numSubTasks);
+		InputFormatVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, pointsPath, "Input[Models]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(modelsInput.getConfiguration());
@@ -278,8 +278,8 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 		return pointsInput;
 	}
 
-	private static OutputFormatOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
-		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
+	private static OutputFormatVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(output.getConfiguration());
@@ -308,10 +308,10 @@ public class BroadcastVarsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Distance Builder");
 
 		// -- vertices ---------------------------------------------------------------------------------------------
-		InputFormatInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
-		InputFormatInputVertex models = createModelsInput(jobGraph, centersPath, numSubTasks, serializer);
+		InputFormatVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
+		InputFormatVertex models = createModelsInput(jobGraph, centersPath, numSubTasks, serializer);
 		JobTaskVertex mapper = createMapper(jobGraph, numSubTasks, serializer);
-		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
 
 		// -- edges ------------------------------------------------------------------------------------------------
 		JobGraphUtils.connect(points, mapper, ChannelType.NETWORK, DistributionPattern.POINTWISE);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
index 4d46b16..678a7e5 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/KMeansIterativeNepheleITCase.java
@@ -32,12 +32,9 @@ import org.apache.flink.runtime.iterative.task.IterationHeadPactTask;
 import org.apache.flink.runtime.iterative.task.IterationIntermediatePactTask;
 import org.apache.flink.runtime.iterative.task.IterationTailPactTask;
 import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.InputFormatInputVertex;
+import org.apache.flink.runtime.jobgraph.InputFormatVertex;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.jobgraph.OutputFormatOutputVertex;
-import org.apache.flink.runtime.jobgraph.SimpleOutputVertex;
 import org.apache.flink.runtime.operators.CollectorMapDriver;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.GroupReduceDriver;
@@ -96,10 +93,10 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 	// Job vertex builder methods
 	// -------------------------------------------------------------------------------------------------------------
 
-	private static InputFormatInputVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatVertex createPointsInput(JobGraph jobGraph, String pointsPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat pointsInFormat = new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class);
-		InputFormatInputVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "[Points]", jobGraph, numSubTasks);
+		InputFormatVertex pointsInput = JobGraphUtils.createInput(pointsInFormat, pointsPath, "[Points]", jobGraph, numSubTasks);
 		{
 			TaskConfig taskConfig = new TaskConfig(pointsInput.getConfiguration());
 			taskConfig.addOutputShipStrategy(ShipStrategyType.FORWARD);
@@ -117,10 +114,10 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return pointsInput;
 	}
 
-	private static InputFormatInputVertex createCentersInput(JobGraph jobGraph, String centersPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static InputFormatVertex createCentersInput(JobGraph jobGraph, String centersPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat modelsInFormat = new CsvInputFormat('|', IntValue.class, DoubleValue.class, DoubleValue.class, DoubleValue.class);
-		InputFormatInputVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, centersPath, "[Models]", jobGraph, numSubTasks);
+		InputFormatVertex modelsInput = JobGraphUtils.createInput(modelsInFormat, centersPath, "[Models]", jobGraph, numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(modelsInput.getConfiguration());
@@ -139,9 +136,9 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		return modelsInput;
 	}
 
-	private static OutputFormatOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
+	private static OutputFormatVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks, TypeSerializerFactory<?> serializer) {
 		
-		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "Output", numSubTasks);
 
 		{
 			TaskConfig taskConfig = new TaskConfig(output.getConfiguration());
@@ -250,13 +247,13 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		tailConfig.setOutputSerializer(outputSerializer);
 		
 		// the udf
-		tailConfig.setStubWrapper(new UserCodeObjectWrapper<WrappingReduceFunction>(new WrappingReduceFunction(new RecomputeClusterCenter())));
+		tailConfig.setStubWrapper(new UserCodeObjectWrapper<RecomputeClusterCenter>(new RecomputeClusterCenter()));
 		
 		return tail;
 	}
 	
-	private static SimpleOutputVertex createSync(JobGraph jobGraph, int numIterations, int dop) {
-		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, dop);
+	private static OutputFormatVertex createSync(JobGraph jobGraph, int numIterations, int dop) {
+		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, dop);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(numIterations);
 		syncConfig.setIterationId(ITERATION_ID);
@@ -277,19 +274,19 @@ public class KMeansIterativeNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("KMeans Iterative");
 
 		// -- vertices ---------------------------------------------------------------------------------------------
-		InputFormatInputVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
-		InputFormatInputVertex centers = createCentersInput(jobGraph, centersPath, numSubTasks, serializer);
+		InputFormatVertex points = createPointsInput(jobGraph, pointsPath, numSubTasks, serializer);
+		InputFormatVertex centers = createCentersInput(jobGraph, centersPath, numSubTasks, serializer);
 		
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer);
 		JobTaskVertex mapper = createMapper(jobGraph, numSubTasks, serializer, serializer, serializer, int0Comparator);
 		
 		JobTaskVertex reducer = createReducer(jobGraph, numSubTasks, serializer, int0Comparator, serializer);
 		
-		SimpleOutputVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
+		OutputFormatVertex fakeTailOutput = JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		
-		SimpleOutputVertex sync = createSync(jobGraph, numIterations, numSubTasks);
+		OutputFormatVertex sync = createSync(jobGraph, numIterations, numSubTasks);
 		
-		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
 
 		// -- edges ------------------------------------------------------------------------------------------------
 		JobGraphUtils.connect(points, mapper, ChannelType.NETWORK, DistributionPattern.POINTWISE);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
index 8da4e5c..dad2370 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/nephele/ConnectedComponentsNepheleITCase.java
@@ -174,12 +174,12 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 	// Invariant vertices across all variants
 	// -----------------------------------------------------------------------------------------------------------------
 
-	private static InputFormatInputVertex createVerticesInput(JobGraph jobGraph, String verticesPath, int numSubTasks,
+	private static InputFormatVertex createVerticesInput(JobGraph jobGraph, String verticesPath, int numSubTasks,
 			TypeSerializerFactory<?> serializer,
 			TypeComparatorFactory<?> comparator) {
 		@SuppressWarnings("unchecked")
 		CsvInputFormat verticesInFormat = new CsvInputFormat(' ', LongValue.class);
-		InputFormatInputVertex verticesInput = JobGraphUtils.createInput(verticesInFormat, verticesPath, "VerticesInput",
+		InputFormatVertex verticesInput = JobGraphUtils.createInput(verticesInFormat, verticesPath, "VerticesInput",
 			jobGraph, numSubTasks);
 		TaskConfig verticesInputConfig = new TaskConfig(verticesInput.getConfiguration());
 		{
@@ -327,9 +327,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return intermediate;
 	}
 
-	private static OutputFormatOutputVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks,
+	private static OutputFormatVertex createOutput(JobGraph jobGraph, String resultPath, int numSubTasks,
 			TypeSerializerFactory<?> serializer) {
-		OutputFormatOutputVertex output = JobGraphUtils.createFileOutput(jobGraph, "Final Output", numSubTasks);
+		OutputFormatVertex output = JobGraphUtils.createFileOutput(jobGraph, "Final Output", numSubTasks);
 		TaskConfig outputConfig = new TaskConfig(output.getConfiguration());
 		{
 
@@ -352,14 +352,14 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		return output;
 	}
 
-	private static SimpleOutputVertex createFakeTail(JobGraph jobGraph, int numSubTasks) {
-		SimpleOutputVertex fakeTailOutput =
+	private static OutputFormatVertex createFakeTail(JobGraph jobGraph, int numSubTasks) {
+		OutputFormatVertex fakeTailOutput =
 			JobGraphUtils.createFakeOutput(jobGraph, "FakeTailOutput", numSubTasks);
 		return fakeTailOutput;
 	}
 
-	private static SimpleOutputVertex createSync(JobGraph jobGraph, int numSubTasks, int maxIterations) {
-		SimpleOutputVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
+	private static OutputFormatVertex createSync(JobGraph jobGraph, int numSubTasks, int maxIterations) {
+		OutputFormatVertex sync = JobGraphUtils.createSync(jobGraph, numSubTasks);
 		TaskConfig syncConfig = new TaskConfig(sync.getConfiguration());
 		syncConfig.setNumberOfIterations(maxIterations);
 		syncConfig.setIterationId(ITERATION_ID);
@@ -389,16 +389,16 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Unified Tails)");
 
 		// -- invariant vertices -----------------------------------------------------------------------------------
-		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
 
 		JobTaskVertex intermediate = createIterationIntermediate(jobGraph, numSubTasks, serializer, comparator);
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
-		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		SimpleOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// --------------- the tail (solution set join) ---------------
 		JobTaskVertex tail = JobGraphUtils.createTask(IterationTailPactTask.class, "IterationTail", jobGraph,
@@ -473,8 +473,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Unified Tails)");
 
 		// input
-		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
@@ -486,10 +486,10 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		SimpleOutputVertex ssFakeTail = createFakeTail(jobGraph, numSubTasks);
-		SimpleOutputVertex wsFakeTail = createFakeTail(jobGraph, numSubTasks);
-		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatVertex ssFakeTail = createFakeTail(jobGraph, numSubTasks);
+		OutputFormatVertex wsFakeTail = createFakeTail(jobGraph, numSubTasks);
+		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ss join) ----------------------
 		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,
@@ -624,8 +624,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Intermediate Workset Update, Solution Set Tail)");
 
 		// input
-		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
@@ -637,9 +637,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		SimpleOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ws update) ----------------------
 		JobTaskVertex wsUpdateIntermediate =
@@ -750,8 +750,8 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		JobGraph jobGraph = new JobGraph("Connected Components (Intermediate Solution Set Update, Workset Tail)");
 
 		// input
-		InputFormatInputVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
-		InputFormatInputVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex vertices = createVerticesInput(jobGraph, verticesPath, numSubTasks, serializer, comparator);
+		InputFormatVertex edges = createEdgesInput(jobGraph, edgesPath, numSubTasks, serializer, comparator);
 
 		// head
 		JobTaskVertex head = createIterationHead(jobGraph, numSubTasks, serializer, comparator, pairComparator);
@@ -761,9 +761,9 @@ public class ConnectedComponentsNepheleITCase extends RecordAPITestBase {
 		TaskConfig intermediateConfig = new TaskConfig(intermediate.getConfiguration());
 
 		// output and auxiliaries
-		OutputFormatOutputVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
-		SimpleOutputVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
-		SimpleOutputVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
+		OutputFormatVertex output = createOutput(jobGraph, resultPath, numSubTasks, serializer);
+		OutputFormatVertex fakeTail = createFakeTail(jobGraph, numSubTasks);
+		OutputFormatVertex sync = createSync(jobGraph, numSubTasks, maxIterations);
 
 		// ------------------ the intermediate (ss update) ----------------------
 		JobTaskVertex ssJoinIntermediate = JobGraphUtils.createTask(IterationIntermediatePactTask.class,


[05/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java
index 0698b56..c8ded86 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/JobEventTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.job;
 
 import static org.junit.Assert.assertEquals;
@@ -25,18 +24,16 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 
-import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.jobgraph.JobStatus;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.junit.Test;
 
 /**
  * This class contains tests concerning the serialization/deserialization of job events which have been derived from
  * {@link org.apache.flink.runtime.event.job.AbstractEvent}.
- * 
  */
 public class JobEventTest {
+
 	/**
 	 * This test checks the correct serialization/deserialization of a {@link JobEvent}.
 	 */
@@ -57,31 +54,4 @@ public class JobEventTest {
 			fail(ioe.getMessage());
 		}
 	}
-
-	/**
-	 * This test checks the correct serialization/deserialization of a {@link VertexEvent}.
-	 */
-	@Test
-	public void testVertexEvent() {
-
-		try {
-
-			final VertexEvent orig = new VertexEvent(23423423L, new JobVertexID(), "Test Vertex", 2, 0,
-				ExecutionState.READY, "Test Description");
-			final VertexEvent copy = (VertexEvent) CommonTestUtils.createCopyWritable(orig);
-
-			assertEquals(orig.getTimestamp(), copy.getTimestamp());
-			assertEquals(orig.getJobVertexID(), copy.getJobVertexID());
-			assertEquals(orig.getJobVertexName(), copy.getJobVertexName());
-			assertEquals(orig.getTotalNumberOfSubtasks(), copy.getTotalNumberOfSubtasks());
-			assertEquals(orig.getIndexOfSubtask(), copy.getIndexOfSubtask());
-			assertEquals(orig.getCurrentExecutionState(), copy.getCurrentExecutionState());
-			assertEquals(orig.getDescription(), copy.getDescription());
-			assertEquals(orig.hashCode(), copy.hashCode());
-			assertTrue(orig.equals(copy));
-
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		}
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java
index 2a30ae4..99e750b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/event/job/ManagementEventTest.java
@@ -16,13 +16,14 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.event.job;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent;
+import org.apache.flink.runtime.event.job.RecentJobEvent;
+import org.apache.flink.runtime.execution.ExecutionState2;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobStatus;
 import org.apache.flink.runtime.managementgraph.ManagementVertexID;
@@ -31,9 +32,9 @@ import org.junit.Test;
 
 /**
  * This test checks the proper serialization and deserialization of job events.
- * 
  */
 public class ManagementEventTest {
+
 	/**
 	 * The time stamp used during the tests.
 	 */
@@ -51,7 +52,7 @@ public class ManagementEventTest {
 	public void testExecutionStateChangeEvent() {
 
 		final ExecutionStateChangeEvent orig = new ExecutionStateChangeEvent(TIMESTAMP, new ManagementVertexID(),
-			ExecutionState.READY);
+			ExecutionState2.DEPLOYING);
 
 		final ExecutionStateChangeEvent copy = (ExecutionStateChangeEvent) ManagementTestUtils.createCopy(orig);
 
@@ -68,8 +69,7 @@ public class ManagementEventTest {
 	@Test
 	public void testRecentJobEvent() {
 
-		final RecentJobEvent orig = new RecentJobEvent(new JobID(), JOBNAME, JobStatus.SCHEDULED, true, TIMESTAMP,
-			TIMESTAMP);
+		final RecentJobEvent orig = new RecentJobEvent(new JobID(), JOBNAME, JobStatus.RUNNING, true, TIMESTAMP, TIMESTAMP);
 
 		final RecentJobEvent copy = (RecentJobEvent) ManagementTestUtils.createCopy(orig);
 
@@ -82,20 +82,4 @@ public class ManagementEventTest {
 		assertEquals(orig.hashCode(), copy.hashCode());
 		assertTrue(orig.equals(copy));
 	}
-
-	/**
-	 * Tests serialization/deserialization for {@link VertexAssignmentEvent}.
-	 */
-	@Test
-	public void testVertexAssignmentEvent() {
-
-		final VertexAssignmentEvent orig = new VertexAssignmentEvent(TIMESTAMP, new ManagementVertexID(), "test");
-		final VertexAssignmentEvent copy = (VertexAssignmentEvent) ManagementTestUtils.createCopy(orig);
-
-		assertEquals(orig.getVertexID(), copy.getVertexID());
-		assertEquals(orig.getTimestamp(), copy.getTimestamp());
-		assertEquals(orig.getInstanceName(), copy.getInstanceName());
-		assertEquals(orig.hashCode(), copy.hashCode());
-		assertTrue(orig.equals(copy));
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java
new file mode 100644
index 0000000..498f773
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/AllVerticesIteratorTest.java
@@ -0,0 +1,67 @@
+/**
+ * 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 java.util.Arrays;
+
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class AllVerticesIteratorTest {
+
+	@Test
+	public void testAllVertices() {
+		try {
+			
+			AbstractJobVertex v1 = new AbstractJobVertex("v1");
+			AbstractJobVertex v2 = new AbstractJobVertex("v2");
+			AbstractJobVertex v3 = new AbstractJobVertex("v3");
+			AbstractJobVertex v4 = new AbstractJobVertex("v4");
+			
+			v1.setParallelism(1);
+			v2.setParallelism(7);
+			v3.setParallelism(3);
+			v4.setParallelism(2);
+			
+			ExecutionGraph eg = Mockito.mock(ExecutionGraph.class);
+					
+			ExecutionJobVertex ejv1 = new ExecutionJobVertex(eg, v1, 1);
+			ExecutionJobVertex ejv2 = new ExecutionJobVertex(eg, v2, 1);
+			ExecutionJobVertex ejv3 = new ExecutionJobVertex(eg, v3, 1);
+			ExecutionJobVertex ejv4 = new ExecutionJobVertex(eg, v4, 1);
+			
+			AllVerticesIterator iter = new AllVerticesIterator(Arrays.asList(ejv1, ejv2, ejv3, ejv4).iterator());
+			
+			int numReturned = 0;
+			while (iter.hasNext()) {
+				iter.hasNext();
+				Assert.assertNotNull(iter.next());
+				numReturned++;
+			}
+			
+			Assert.assertEquals(13, numReturned);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
new file mode 100644
index 0000000..b6f532e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphConstructionTest.java
@@ -0,0 +1,570 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.mockito.Matchers;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.core.io.InputSplitSource;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+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.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.util.LogUtils;
+
+/**
+ * This class contains test concerning the correct conversion from {@link JobGraph} to {@link ExecutionGraph} objects.
+ */
+public class ExecutionGraphConstructionTest {
+	
+	@BeforeClass
+	public static void setLogLevel() {
+		LogUtils.initializeDefaultTestConsoleLogger();
+	}
+
+	
+	/**
+	 * Creates a JobGraph of the following form:
+	 * 
+	 * <pre>
+	 *  v1--->v2-->\
+	 *              \
+	 *               v4 --->\
+	 *        ----->/        \
+	 *  v3-->/                v5
+	 *       \               /
+	 *        ------------->/
+	 * </pre>
+	 */
+	@Test
+	public void testCreateSimpleGraphBipartite() {
+		
+		final JobID jobId = new JobID();
+		final String jobName = "Test Job Sample Name";
+		final Configuration cfg = new Configuration();
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+		AbstractJobVertex v3 = new AbstractJobVertex("vertex3");
+		AbstractJobVertex v4 = new AbstractJobVertex("vertex4");
+		AbstractJobVertex v5 = new AbstractJobVertex("vertex5");
+		
+		v1.setParallelism(5);
+		v2.setParallelism(7);
+		v3.setParallelism(2);
+		v4.setParallelism(11);
+		v5.setParallelism(4);
+		
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.BIPARTITE);
+		v4.connectNewDataSetAsInput(v2, DistributionPattern.BIPARTITE);
+		v4.connectNewDataSetAsInput(v3, DistributionPattern.BIPARTITE);
+		v5.connectNewDataSetAsInput(v4, DistributionPattern.BIPARTITE);
+		v5.connectNewDataSetAsInput(v3, DistributionPattern.BIPARTITE);
+		
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3, v4, v5));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		verifyTestGraph(eg, jobId, v1, v2, v3, v4, v5);
+	}
+	
+	@Test
+	public void testAttachViaDataSets() {
+		final JobID jobId = new JobID();
+		final String jobName = "Test Job Sample Name";
+		final Configuration cfg = new Configuration();
+		
+		// construct part one of the execution graph
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+		AbstractJobVertex v3 = new AbstractJobVertex("vertex3");
+		
+		v1.setParallelism(5);
+		v2.setParallelism(7);
+		v3.setParallelism(2);
+		
+		// this creates an intermediate result for v1
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.BIPARTITE);
+		
+		// create results for v2 and v3
+		IntermediateDataSet v2result = v2.createAndAddResultDataSet();
+		IntermediateDataSet v3result_1 = v3.createAndAddResultDataSet();
+		IntermediateDataSet v3result_2 = v3.createAndAddResultDataSet();
+		
+		
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		// attach the second part of the graph
+		
+		AbstractJobVertex v4 = new AbstractJobVertex("vertex4");
+		AbstractJobVertex v5 = new AbstractJobVertex("vertex5");
+		v4.setParallelism(11);
+		v5.setParallelism(4);
+		
+		v4.connectDataSetAsInput(v2result, DistributionPattern.BIPARTITE);
+		v4.connectDataSetAsInput(v3result_1, DistributionPattern.BIPARTITE);
+		v5.connectNewDataSetAsInput(v4, DistributionPattern.BIPARTITE);
+		v5.connectDataSetAsInput(v3result_2, DistributionPattern.BIPARTITE);
+		
+		List<AbstractJobVertex> ordered2 = new ArrayList<AbstractJobVertex>(Arrays.asList(v4, v5));
+		
+		try {
+			eg.attachJobGraph(ordered2);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		// verify
+		verifyTestGraph(eg, jobId, v1, v2, v3, v4, v5);
+	}
+	
+	@Test
+	public void testAttachViaIds() {
+		final JobID jobId = new JobID();
+		final String jobName = "Test Job Sample Name";
+		final Configuration cfg = new Configuration();
+		
+		// construct part one of the execution graph
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+		AbstractJobVertex v3 = new AbstractJobVertex("vertex3");
+		
+		v1.setParallelism(5);
+		v2.setParallelism(7);
+		v3.setParallelism(2);
+		
+		// this creates an intermediate result for v1
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.BIPARTITE);
+		
+		// create results for v2 and v3
+		IntermediateDataSet v2result = v2.createAndAddResultDataSet();
+		IntermediateDataSet v3result_1 = v3.createAndAddResultDataSet();
+		IntermediateDataSet v3result_2 = v3.createAndAddResultDataSet();
+		
+		
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		// attach the second part of the graph
+		
+		AbstractJobVertex v4 = new AbstractJobVertex("vertex4");
+		AbstractJobVertex v5 = new AbstractJobVertex("vertex5");
+		v4.setParallelism(11);
+		v5.setParallelism(4);
+		
+		v4.connectIdInput(v2result.getId(), DistributionPattern.BIPARTITE);
+		v4.connectIdInput(v3result_1.getId(), DistributionPattern.BIPARTITE);
+		v5.connectNewDataSetAsInput(v4, DistributionPattern.BIPARTITE);
+		v5.connectIdInput(v3result_2.getId(), DistributionPattern.BIPARTITE);
+		
+		List<AbstractJobVertex> ordered2 = new ArrayList<AbstractJobVertex>(Arrays.asList(v4, v5));
+		
+		try {
+			eg.attachJobGraph(ordered2);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		// verify
+		verifyTestGraph(eg, jobId, v1, v2, v3, v4, v5);
+	}
+	
+	private void verifyTestGraph(ExecutionGraph eg, JobID jobId,
+				AbstractJobVertex v1, AbstractJobVertex v2, AbstractJobVertex v3,
+				AbstractJobVertex v4, AbstractJobVertex v5)
+	{
+		Map<JobVertexID, ExecutionJobVertex> vertices = eg.getAllVertices();
+		
+		// verify v1
+		{
+			ExecutionJobVertex e1 = vertices.get(v1.getID());
+			assertNotNull(e1);
+			
+			// basic properties
+			assertEquals(v1.getParallelism(), e1.getParallelism());
+			assertEquals(v1.getID(), e1.getJobVertexId());
+			assertEquals(jobId, e1.getJobId());
+			assertEquals(v1, e1.getJobVertex());
+			
+			// produced data sets
+			assertEquals(1, e1.getProducedDataSets().length);
+			assertEquals(v1.getProducedDataSets().get(0).getId(), e1.getProducedDataSets()[0].getId());
+			assertEquals(v1.getParallelism(), e1.getProducedDataSets()[0].getPartitions().length);
+			
+			// task vertices
+			assertEquals(v1.getParallelism(), e1.getTaskVertices().length);
+			
+			int num = 0;
+			for (ExecutionVertex2 ev : e1.getTaskVertices()) {
+				assertEquals(jobId, ev.getJobId());
+				assertEquals(v1.getID(), ev.getJobvertexId());
+				
+				assertEquals(v1.getParallelism(), ev.getTotalNumberOfParallelSubtasks());
+				assertEquals(num++, ev.getParallelSubtaskIndex());
+				
+				assertEquals(0, ev.getNumberOfInputs());
+			}
+		}
+		
+		// verify v2
+		{
+			ExecutionJobVertex e2 = vertices.get(v2.getID());
+			assertNotNull(e2);
+			
+			// produced data sets
+			assertEquals(1, e2.getProducedDataSets().length);
+			assertEquals(v2.getProducedDataSets().get(0).getId(), e2.getProducedDataSets()[0].getId());
+			assertEquals(v2.getParallelism(), e2.getProducedDataSets()[0].getPartitions().length);
+			
+			// task vertices
+			assertEquals(v2.getParallelism(), e2.getTaskVertices().length);
+			
+			int num = 0;
+			for (ExecutionVertex2 ev : e2.getTaskVertices()) {
+				assertEquals(jobId, ev.getJobId());
+				assertEquals(v2.getID(), ev.getJobvertexId());
+				
+				assertEquals(v2.getParallelism(), ev.getTotalNumberOfParallelSubtasks());
+				assertEquals(num++, ev.getParallelSubtaskIndex());
+				
+				assertEquals(1, ev.getNumberOfInputs());
+				ExecutionEdge2[] inputs = ev.getInputEdges(0);
+				assertEquals(v1.getParallelism(), inputs.length);
+				
+				int sumOfPartitions = 0;
+				for (ExecutionEdge2 inEdge : inputs) {
+					assertEquals(0,inEdge.getInputNum());
+					sumOfPartitions += inEdge.getSource().getPartition();
+				}
+				
+				assertEquals(10, sumOfPartitions);
+			}
+		}
+		
+		// verify v3
+		{
+			ExecutionJobVertex e3 = vertices.get(v3.getID());
+			assertNotNull(e3);
+			
+			// produced data sets
+			assertEquals(2, e3.getProducedDataSets().length);
+			assertEquals(v3.getProducedDataSets().get(0).getId(), e3.getProducedDataSets()[0].getId());
+			assertEquals(v3.getProducedDataSets().get(1).getId(), e3.getProducedDataSets()[1].getId());
+			assertEquals(v3.getParallelism(), e3.getProducedDataSets()[0].getPartitions().length);
+			assertEquals(v3.getParallelism(), e3.getProducedDataSets()[1].getPartitions().length);
+			
+			// task vertices
+			assertEquals(v3.getParallelism(), e3.getTaskVertices().length);
+			
+			int num = 0;
+			for (ExecutionVertex2 ev : e3.getTaskVertices()) {
+				assertEquals(jobId, ev.getJobId());
+				assertEquals(v3.getID(), ev.getJobvertexId());
+				
+				assertEquals(v3.getParallelism(), ev.getTotalNumberOfParallelSubtasks());
+				assertEquals(num++, ev.getParallelSubtaskIndex());
+				
+				assertEquals(0, ev.getNumberOfInputs());
+			}
+		}
+
+		// verify v4
+		{
+			ExecutionJobVertex e4 = vertices.get(v4.getID());
+			assertNotNull(e4);
+			
+			// produced data sets
+			assertEquals(1, e4.getProducedDataSets().length);
+			assertEquals(v4.getProducedDataSets().get(0).getId(), e4.getProducedDataSets()[0].getId());
+			
+			// task vertices
+			assertEquals(v4.getParallelism(), e4.getTaskVertices().length);
+			
+			int num = 0;
+			for (ExecutionVertex2 ev : e4.getTaskVertices()) {
+				assertEquals(jobId, ev.getJobId());
+				assertEquals(v4.getID(), ev.getJobvertexId());
+				
+				assertEquals(v4.getParallelism(), ev.getTotalNumberOfParallelSubtasks());
+				assertEquals(num++, ev.getParallelSubtaskIndex());
+				
+				assertEquals(2, ev.getNumberOfInputs());
+				// first input
+				{
+					ExecutionEdge2[] inputs = ev.getInputEdges(0);
+					assertEquals(v2.getParallelism(), inputs.length);
+					
+					int sumOfPartitions = 0;
+					for (ExecutionEdge2 inEdge : inputs) {
+						assertEquals(0, inEdge.getInputNum());
+						sumOfPartitions += inEdge.getSource().getPartition();
+					}
+					
+					assertEquals(21, sumOfPartitions);
+				}
+				// second input
+				{
+					ExecutionEdge2[] inputs = ev.getInputEdges(1);
+					assertEquals(v3.getParallelism(), inputs.length);
+					
+					int sumOfPartitions = 0;
+					for (ExecutionEdge2 inEdge : inputs) {
+						assertEquals(1, inEdge.getInputNum());
+						sumOfPartitions += inEdge.getSource().getPartition();
+					}
+					
+					assertEquals(1, sumOfPartitions);
+				}
+			}
+		}
+		
+		// verify v5
+		{
+			ExecutionJobVertex e5 = vertices.get(v5.getID());
+			assertNotNull(e5);
+			
+			// produced data sets
+			assertEquals(0, e5.getProducedDataSets().length);
+			
+			// task vertices
+			assertEquals(v5.getParallelism(), e5.getTaskVertices().length);
+			
+			int num = 0;
+			for (ExecutionVertex2 ev : e5.getTaskVertices()) {
+				assertEquals(jobId, ev.getJobId());
+				assertEquals(v5.getID(), ev.getJobvertexId());
+				
+				assertEquals(v5.getParallelism(), ev.getTotalNumberOfParallelSubtasks());
+				assertEquals(num++, ev.getParallelSubtaskIndex());
+				
+				assertEquals(2, ev.getNumberOfInputs());
+				// first input
+				{
+					ExecutionEdge2[] inputs = ev.getInputEdges(0);
+					assertEquals(v4.getParallelism(), inputs.length);
+					
+					int sumOfPartitions = 0;
+					for (ExecutionEdge2 inEdge : inputs) {
+						assertEquals(0, inEdge.getInputNum());
+						sumOfPartitions += inEdge.getSource().getPartition();
+					}
+					
+					assertEquals(55, sumOfPartitions);
+				}
+				// second input
+				{
+					ExecutionEdge2[] inputs = ev.getInputEdges(1);
+					assertEquals(v3.getParallelism(), inputs.length);
+					
+					int sumOfPartitions = 0;
+					for (ExecutionEdge2 inEdge : inputs) {
+						assertEquals(1, inEdge.getInputNum());
+						sumOfPartitions += inEdge.getSource().getPartition();
+					}
+					
+					assertEquals(1, sumOfPartitions);
+				}
+			}
+		}
+	}
+	
+	@Test
+	public void testCannotConnectMissingId() {
+		final JobID jobId = new JobID();
+		final String jobName = "Test Job Sample Name";
+		final Configuration cfg = new Configuration();
+		
+		// construct part one of the execution graph
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		v1.setParallelism(7);
+		
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		// attach the second part of the graph
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+		v2.connectIdInput(new IntermediateDataSetID(), DistributionPattern.BIPARTITE);
+		
+		List<AbstractJobVertex> ordered2 = new ArrayList<AbstractJobVertex>(Arrays.asList(v2));
+		
+		try {
+			eg.attachJobGraph(ordered2);
+			fail("Attached wrong jobgraph");
+		}
+		catch (JobException e) {
+			// expected
+		}
+	}
+
+	@Test
+	public void testCannotConnectWrongOrder() {
+		final JobID jobId = new JobID();
+		final String jobName = "Test Job Sample Name";
+		final Configuration cfg = new Configuration();
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+		AbstractJobVertex v3 = new AbstractJobVertex("vertex3");
+		AbstractJobVertex v4 = new AbstractJobVertex("vertex4");
+		AbstractJobVertex v5 = new AbstractJobVertex("vertex5");
+		
+		v1.setParallelism(5);
+		v2.setParallelism(7);
+		v3.setParallelism(2);
+		v4.setParallelism(11);
+		v5.setParallelism(4);
+		
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.BIPARTITE);
+		v4.connectNewDataSetAsInput(v2, DistributionPattern.BIPARTITE);
+		v4.connectNewDataSetAsInput(v3, DistributionPattern.BIPARTITE);
+		v5.connectNewDataSetAsInput(v4, DistributionPattern.BIPARTITE);
+		v5.connectNewDataSetAsInput(v3, DistributionPattern.BIPARTITE);
+		
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3, v5, v4));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+			fail("Attached wrong jobgraph");
+		}
+		catch (JobException e) {
+			// expected
+		}
+	}
+	
+	@Test
+	public void testSetupInputSplits() {
+		try {
+			final InputSplit[] emptySplits = new InputSplit[0];
+			
+			InputSplitAssigner assigner1 = mock(InputSplitAssigner.class);
+			InputSplitAssigner assigner2 = mock(InputSplitAssigner.class);
+			
+			@SuppressWarnings("unchecked")
+			InputSplitSource<InputSplit> source1 = mock(InputSplitSource.class);
+			@SuppressWarnings("unchecked")
+			InputSplitSource<InputSplit> source2 = mock(InputSplitSource.class);
+			
+			when(source1.createInputSplits(Matchers.anyInt())).thenReturn(emptySplits);
+			when(source2.createInputSplits(Matchers.anyInt())).thenReturn(emptySplits);
+			when(source1.getInputSplitAssigner(emptySplits)).thenReturn(assigner1);
+			when(source2.getInputSplitAssigner(emptySplits)).thenReturn(assigner2);
+			
+			final JobID jobId = new JobID();
+			final String jobName = "Test Job Sample Name";
+			final Configuration cfg = new Configuration();
+			
+			AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+			AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+			AbstractJobVertex v3 = new AbstractJobVertex("vertex3");
+			AbstractJobVertex v4 = new AbstractJobVertex("vertex4");
+			AbstractJobVertex v5 = new AbstractJobVertex("vertex5");
+			
+			v1.setParallelism(5);
+			v2.setParallelism(7);
+			v3.setParallelism(2);
+			v4.setParallelism(11);
+			v5.setParallelism(4);
+			
+			v2.connectNewDataSetAsInput(v1, DistributionPattern.BIPARTITE);
+			v4.connectNewDataSetAsInput(v2, DistributionPattern.BIPARTITE);
+			v4.connectNewDataSetAsInput(v3, DistributionPattern.BIPARTITE);
+			v5.connectNewDataSetAsInput(v4, DistributionPattern.BIPARTITE);
+			v5.connectNewDataSetAsInput(v3, DistributionPattern.BIPARTITE);
+			
+			v3.setInputSplitSource(source1);
+			v5.setInputSplitSource(source2);
+			
+			List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3, v4, v5));
+
+			ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+			try {
+				eg.attachJobGraph(ordered);
+			}
+			catch (JobException e) {
+				e.printStackTrace();
+				fail("Job failed with exception: " + e.getMessage());
+			}
+			
+			assertEquals(assigner1, eg.getAllVertices().get(v3.getID()).getSplitAssigner());
+			assertEquals(assigner2, eg.getAllVertices().get(v5.getID()).getSplitAssigner());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
new file mode 100644
index 0000000..9705dcd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphDeploymentTest.java
@@ -0,0 +1,145 @@
+/**
+ * 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 static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.doAnswer;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.operators.RegularPactTask;
+import org.apache.flink.runtime.protocols.TaskOperationProtocol;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
+import org.junit.Test;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class ExecutionGraphDeploymentTest {
+
+	@Test
+	public void testBuildDeploymentDescriptor() {
+		try {
+			final JobID jobId = new JobID();
+			
+			final JobVertexID jid1 = new JobVertexID();
+			final JobVertexID jid2 = new JobVertexID();
+			final JobVertexID jid3 = new JobVertexID();
+			final JobVertexID jid4 = new JobVertexID();
+			
+			AbstractJobVertex v1 = new AbstractJobVertex("v1", jid1);
+			AbstractJobVertex v2 = new AbstractJobVertex("v2", jid2);
+			AbstractJobVertex v3 = new AbstractJobVertex("v3", jid3);
+			AbstractJobVertex v4 = new AbstractJobVertex("v4", jid4);
+			
+			v1.setParallelism(10);
+			v2.setParallelism(10);
+			v3.setParallelism(10);
+			v4.setParallelism(10);
+			
+			v1.setInvokableClass(RegularPactTask.class);
+			v2.setInvokableClass(RegularPactTask.class);
+			v3.setInvokableClass(RegularPactTask.class);
+			v4.setInvokableClass(RegularPactTask.class);
+			
+			v2.connectNewDataSetAsInput(v1, DistributionPattern.BIPARTITE);
+			v3.connectNewDataSetAsInput(v2, DistributionPattern.BIPARTITE);
+			v4.connectNewDataSetAsInput(v2, DistributionPattern.BIPARTITE);
+			
+			ExecutionGraph eg = spy(new ExecutionGraph(jobId, "some job", new Configuration()));
+			doAnswer(new Answer<Void>() {
+				@Override
+				public Void answer(InvocationOnMock invocation) {
+					final Runnable parameter = (Runnable) invocation.getArguments()[0];
+					parameter.run();
+					return null;
+				}
+				
+			}).when(eg).execute(Matchers.any(Runnable.class));
+			
+			List<AbstractJobVertex> ordered = Arrays.asList(v1, v2, v3, v4);
+			
+			eg.attachJobGraph(ordered);
+			
+			ExecutionJobVertex ejv = eg.getAllVertices().get(jid2);
+			ExecutionVertex2 vertex = ejv.getTaskVertices()[3];
+			
+			// just some reference (needs not be atomic)
+			final AtomicReference<TaskDeploymentDescriptor> reference = new AtomicReference<TaskDeploymentDescriptor>();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenAnswer(new Answer<TaskOperationResult>() {
+				@Override
+				public TaskOperationResult answer(InvocationOnMock invocation) {
+					final TaskDeploymentDescriptor parameter = (TaskDeploymentDescriptor) invocation.getArguments()[0];
+					reference.set(parameter);
+					return new TaskOperationResult(jid2, 0, true);
+				}
+			});
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(jobId);
+			
+			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			vertex.deployToSlot(slot);
+			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			TaskDeploymentDescriptor descr = reference.get();
+			assertNotNull(descr);
+			
+			assertEquals(jobId, descr.getJobID());
+			assertEquals(jid2, descr.getVertexID());
+			assertEquals(3, descr.getIndexInSubtaskGroup());
+			assertEquals(10, descr.getCurrentNumberOfSubtasks());
+			assertEquals(RegularPactTask.class.getName(), descr.getInvokableClassName());
+			assertEquals("v2", descr.getTaskName());
+			
+			assertEquals(2, descr.getOutputGates().size());
+			assertEquals(1, descr.getInputGates().size());
+			
+			assertEquals(10, descr.getOutputGates().get(0).getChannels().size());
+			assertEquals(10, descr.getOutputGates().get(1).getChannels().size());
+			assertEquals(10, descr.getInputGates().get(0).getChannels().size());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTest.java
deleted file mode 100644
index 36e3640..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTest.java
+++ /dev/null
@@ -1,955 +0,0 @@
-/**
- * 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 static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-import org.apache.flink.runtime.jobgraph.DistributionPattern;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-import org.apache.flink.runtime.jobgraph.JobGraphDefinitionException;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobgraph.JobInputVertex;
-import org.apache.flink.runtime.jobgraph.JobOutputVertex;
-import org.apache.flink.runtime.jobgraph.JobTaskVertex;
-import org.apache.flink.runtime.operators.DataSinkTask;
-import org.apache.flink.runtime.operators.DataSourceTask;
-import org.apache.flink.runtime.testutils.ServerTestUtils;
-import org.junit.Test;
-import org.apache.flink.api.java.io.DiscardingOuputFormat;
-import org.apache.flink.api.java.io.TextInputFormat;
-
-/**
- * This class contains test concerning the correct conversion from {@link JobGraph} to {@link ExecutionGraph} objects.
- * 
- */
-public class ExecutionGraphTest {
-	/*
-	 * input1 -> task1 -> output1
-	 * output1 shares instance with input1
-	 * input1 shares instance with task1
-	 * no subtasks defined
-	 * input1 is default, task1 is m1.large, output1 is m1.xlarge
-	 * no channel types defined
-	 */
-	@Test
-	public void testConvertJobGraphToExecutionGraph1() {
-
-		File inputFile = null;
-		JobID jobID = null;
-
-		try {
-			inputFile = ServerTestUtils.createInputFile(0);
-
-			// create job graph
-			final JobGraph jg = new JobGraph("Job Graph 1");
-			jobID = jg.getJobID();
-
-			// input vertex
-			final JobInputVertex i1 = new JobInputVertex("Input 1", jg);
-			i1.setNumberOfSubtasks(1);
-			i1.setInvokableClass(DataSourceTask.class);
-			TextInputFormat inputFormat = new TextInputFormat(new Path(inputFile.toURI()));
-			i1.setInputFormat(inputFormat);
-
-			// task vertex
-			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
-			t1.setInvokableClass(ForwardTask1Input1Output.class);
-
-			// output vertex
-			final JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
-			o1.setNumberOfSubtasks(1);
-			o1.setInvokableClass(DataSinkTask.class);
-			o1.setOutputFormat(new DiscardingOuputFormat<Object>());
-
-			o1.setVertexToShareInstancesWith(i1);
-			i1.setVertexToShareInstancesWith(t1);
-
-			// connect vertices
-			i1.connectTo(t1);
-			t1.connectTo(o1);
-
-			LibraryCacheManager.register(jobID, new String[0]);
-
-			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
-
-			// test all methods of ExecutionGraph
-			final ExecutionStage executionStage = eg.getCurrentExecutionStage();
-			assertEquals(1, executionStage.getMaxNumberSubtasks());
-
-			assertEquals(jobID, eg.getJobID());
-			assertEquals(0, eg.getIndexOfCurrentExecutionStage());
-			assertEquals(1, eg.getNumberOfInputVertices());
-			assertEquals(1, eg.getNumberOfOutputVertices());
-			assertEquals(1, eg.getNumberOfStages());
-			assertNotNull(eg.getInputVertex(0));
-			assertNull(eg.getInputVertex(1));
-			assertNotNull(eg.getOutputVertex(0));
-			assertNull(eg.getOutputVertex(1));
-			assertNotNull(eg.getStage(0));
-			assertNull(eg.getStage(1));
-
-			// test all methods of ExecutionStage stage0
-			ExecutionStage es = eg.getStage(0);
-
-			assertEquals(3, es.getNumberOfStageMembers());
-			assertEquals(0, es.getStageNumber());
-			assertNotNull(es.getStageMember(0));
-			assertNotNull(es.getStageMember(1));
-			assertNotNull(es.getStageMember(2));
-			assertNull(es.getStageMember(3));
-
-			// test all methods of ExecutionGroupVertex
-			ExecutionGroupVertex egv0 = null; // input1
-			ExecutionGroupVertex egv1 = null; // output1
-			ExecutionGroupVertex egv2 = null; // task1
-
-			if (es.getStageMember(0).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(0);
-			} else if (es.getStageMember(0).getName().equals("Output 1")) {
-				egv1 = es.getStageMember(0);
-			} else {
-				egv2 = es.getStageMember(0);
-			}
-
-			if (es.getStageMember(1).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(1);
-			} else if (es.getStageMember(1).getName().equals("Output 1")) {
-				egv1 = es.getStageMember(1);
-			} else {
-				egv2 = es.getStageMember(1);
-			}
-
-			if (es.getStageMember(2).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(2);
-			} else if (es.getStageMember(2).getName().equals("Output 1")) {
-				egv1 = es.getStageMember(2);
-			} else {
-				egv2 = es.getStageMember(2);
-			}
-
-			// egv0 (input1)
-			assertNull(egv0.getBackwardEdge(0));
-			assertNotNull(egv0.getConfiguration());
-			assertEquals(1, egv0.getCurrentNumberOfGroupMembers());
-			assertNotNull(egv0.getExecutionSignature());
-			assertEquals(es, egv0.getExecutionStage());
-			assertNotNull(egv0.getForwardEdge(0));
-			assertNull(egv0.getForwardEdge(1));
-			assertNotNull(egv0.getForwardEdges(egv2));
-			assertNotNull(egv0.getGroupMember(0));
-			assertNull(egv0.getGroupMember(1));
-			assertEquals(1, egv0.getInputSplits().length);
-			assertEquals("Input 1", egv0.getName());
-			assertEquals(0, egv0.getNumberOfBackwardLinks());
-			assertEquals(1, egv0.getNumberOfForwardLinks());
-			assertEquals(0, egv0.getStageNumber());
-			assertEquals(1, egv0.getUserDefinedNumberOfMembers());
-			assertEquals("Task 1", egv0.getVertexToShareInstancesWith().getName());
-
-			// egv1 (output1)
-			assertNotNull(egv1.getBackwardEdge(0));
-			assertNull(egv1.getBackwardEdge(1));
-			assertNotNull(egv1.getBackwardEdges(egv2));
-			assertNotNull(egv1.getConfiguration());
-			assertEquals(1, egv1.getCurrentNumberOfGroupMembers());
-			assertNotNull(egv1.getExecutionSignature());
-			assertEquals(es, egv1.getExecutionStage());
-			assertNull(egv1.getForwardEdge(0));
-			assertNotNull(egv1.getGroupMember(0));
-			assertNull(egv1.getGroupMember(1));
-			assertEquals("Output 1", egv1.getName());
-			assertEquals(1, egv1.getNumberOfBackwardLinks());
-			assertEquals(0, egv1.getNumberOfForwardLinks());
-			assertEquals(0, egv1.getStageNumber());
-			assertEquals(1, egv1.getUserDefinedNumberOfMembers());
-			assertEquals("Input 1", egv1.getVertexToShareInstancesWith().getName());
-
-			// egv2 (task1)
-			assertNotNull(egv2.getBackwardEdge(0));
-			assertNull(egv2.getBackwardEdge(1));
-			assertNotNull(egv2.getBackwardEdges(egv0));
-			assertNotNull(egv2.getConfiguration());
-			assertEquals(1, egv2.getCurrentNumberOfGroupMembers());
-			assertNotNull(egv2.getExecutionSignature());
-			assertEquals(es, egv2.getExecutionStage());
-			assertNotNull(egv2.getForwardEdge(0));
-			assertNull(egv2.getForwardEdge(1));
-			assertNotNull(egv2.getForwardEdges(egv1));
-			assertNotNull(egv2.getGroupMember(0));
-			assertNull(egv2.getGroupMember(1));
-			assertEquals("Task 1", egv2.getName());
-			assertEquals(1, egv2.getNumberOfBackwardLinks());
-			assertEquals(1, egv2.getNumberOfForwardLinks());
-			assertEquals(0, egv2.getStageNumber());
-			assertEquals(1, egv2.getUserDefinedNumberOfMembers());
-			assertNull(egv2.getVertexToShareInstancesWith());
-
-			// test all methods of ExecutionVertex
-			ExecutionVertex ev0 = egv0.getGroupMember(0); // input1
-			ExecutionVertex ev1 = egv1.getGroupMember(0); // output1
-			ExecutionVertex ev2 = egv2.getGroupMember(0); // task1
-
-			// ev0 (input1)
-			assertEquals(egv0, ev0.getGroupVertex());
-			assertNotNull(ev0.getID());
-			assertEquals("Input 1", ev0.getName());
-
-			// ev1 (output1)
-			assertEquals(egv1, ev1.getGroupVertex());
-			assertNotNull(ev1.getID());
-			assertEquals("Output 1", ev1.getName());
-
-			// ev2 (task1)
-			assertEquals(egv2, ev2.getGroupVertex());
-			assertNotNull(ev2.getID());
-			assertEquals("Task 1", ev2.getName());
-
-			assertEquals(ev0.getAllocatedResource(), ev1.getAllocatedResource());
-			assertEquals(ev0.getAllocatedResource(), ev2.getAllocatedResource());
-
-			// test channels
-			assertEquals(ChannelType.NETWORK, eg.getChannelType(ev0, ev2));
-			assertEquals(ChannelType.NETWORK, eg.getChannelType(ev2, ev1));
-
-		} catch (GraphConversionException e) {
-			fail(e.getMessage());
-		} catch (JobGraphDefinitionException e) {
-			fail(e.getMessage());
-		} catch (IOException e) {
-			fail(e.getMessage());
-		} finally {
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (jobID != null) {
-				try {
-					LibraryCacheManager.unregister(jobID);
-				} catch (IOException e) {
-				}
-			}
-		}
-	}
-
-	/*
-	 * input1 -> task1 -> output1
-	 * no subtasks defined
-	 * input1 is default, task1 is m1.large, output1 is m1.xlarge
-	 * all channels are INMEMORY
-	 */
-	@Test
-	public void testConvertJobGraphToExecutionGraph2() {
-
-		File inputFile = null;
-		JobID jobID = null;
-
-		try {
-			inputFile = ServerTestUtils.createInputFile(0);
-
-			// create job graph
-			final JobGraph jg = new JobGraph("Job Graph 1");
-			jobID = jg.getJobID();
-
-			// input vertex
-			final JobInputVertex i1 = new JobInputVertex("Input 1", jg);
-			i1.setInvokableClass(DataSourceTask.class);
-			i1.setInputFormat(new TextInputFormat(new Path(inputFile.toURI())));
-			i1.setNumberOfSubtasks(1);
-
-			// task vertex
-			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
-			t1.setInvokableClass(ForwardTask1Input1Output.class);
-
-			// output vertex
-			final JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
-			o1.setNumberOfSubtasks(1);
-			o1.setInvokableClass(DataSinkTask.class);
-			o1.setOutputFormat(new DiscardingOuputFormat<Object>());
-
-			// connect vertices
-			i1.connectTo(t1, ChannelType.IN_MEMORY);
-			t1.connectTo(o1, ChannelType.IN_MEMORY);
-
-			LibraryCacheManager.register(jobID, new String[0]);
-
-			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
-
-			// test instance types in ExecutionGraph
-			final ExecutionStage executionStage = eg.getCurrentExecutionStage();
-			assertEquals(1, executionStage.getMaxNumberSubtasks());
-
-			// stage0
-			ExecutionStage es = eg.getStage(0);
-			ExecutionGroupVertex egv0 = null; // input1
-			ExecutionGroupVertex egv1 = null; // output1
-			ExecutionGroupVertex egv2 = null; // task1
-			if (es.getStageMember(0).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(0);
-			} else if (es.getStageMember(0).getName().equals("Output 1")) {
-				egv1 = es.getStageMember(0);
-			} else {
-				egv2 = es.getStageMember(0);
-			}
-			if (es.getStageMember(1).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(1);
-			} else if (es.getStageMember(1).getName().equals("Output 1")) {
-				egv1 = es.getStageMember(1);
-			} else {
-				egv2 = es.getStageMember(1);
-			}
-			if (es.getStageMember(2).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(2);
-			} else if (es.getStageMember(2).getName().equals("Output 1")) {
-				egv1 = es.getStageMember(2);
-			} else {
-				egv2 = es.getStageMember(2);
-			}
-			ExecutionVertex ev0 = egv0.getGroupMember(0); // input1
-			ExecutionVertex ev1 = egv1.getGroupMember(0); // output1
-			ExecutionVertex ev2 = egv2.getGroupMember(0); // task1
-			assertEquals(ev0.getAllocatedResource(), ev1.getAllocatedResource());
-			assertEquals(ev0.getAllocatedResource(), ev2.getAllocatedResource());
-		} catch (GraphConversionException e) {
-			fail(e.getMessage());
-		} catch (IOException e) {
-			fail(e.getMessage());
-		} catch (JobGraphDefinitionException e) {
-			fail(e.getMessage());
-		} finally {
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (jobID != null) {
-				try {
-					LibraryCacheManager.unregister(jobID);
-				} catch (IOException e) {
-				}
-			}
-		}
-	}
-
-	/*
-	 * input1 -> task1 ->
-	 * task3 -> output1
-	 * input2 -> task2 ->
-	 * each vertex has 2 subtasks
-	 * no instance types defined
-	 * no channel types defined
-	 */
-	@Test
-	public void testConvertJobGraphToExecutionGraph3() {
-
-		File inputFile1 = null;
-		File inputFile2 = null;
-		File outputFile = null;
-		JobID jobID = null;
-
-		try {
-
-			inputFile1 = ServerTestUtils.createInputFile(0);
-			inputFile2 = ServerTestUtils.createInputFile(0);
-			outputFile = new File(ServerTestUtils.getRandomFilename());
-
-			// create job graph
-			final JobGraph jg = new JobGraph("Job Graph 1");
-			jobID = jg.getJobID();
-
-			// input vertex
-			final JobInputVertex i1 = new JobInputVertex("Input 1", jg);
-			i1.setInvokableClass(DataSourceTask.class);
-			i1.setInputFormat(new TextInputFormat(new Path(inputFile1.toURI())));
-			i1.setNumberOfSubtasks(2);
-			
-			final JobInputVertex i2 = new JobInputVertex("Input 2", jg);
-			i2.setInvokableClass(DataSourceTask.class);
-			i2.setInputFormat(new TextInputFormat(new Path(inputFile2.toURI())));
-			i2.setNumberOfSubtasks(2);
-
-			// task vertex
-			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
-			t1.setInvokableClass(ForwardTask1Input1Output.class);
-			t1.setNumberOfSubtasks(2);
-			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setInvokableClass(ForwardTask1Input1Output.class);
-			t2.setNumberOfSubtasks(2);
-			final JobTaskVertex t3 = new JobTaskVertex("Task 3", jg);
-			t3.setInvokableClass(ForwardTask2Inputs1Output.class);
-			t3.setNumberOfSubtasks(2);
-
-			
-			// output vertex
-			final JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
-			o1.setInvokableClass(DataSinkTask.class);
-			o1.setOutputFormat(new DiscardingOuputFormat<Object>());
-			o1.setNumberOfSubtasks(2);
-			i1.setVertexToShareInstancesWith(t1);
-			t1.setVertexToShareInstancesWith(t3);
-			i2.setVertexToShareInstancesWith(t2);
-			t2.setVertexToShareInstancesWith(t3);
-			t3.setVertexToShareInstancesWith(o1);
-
-			// connect vertices
-			i1.connectTo(t1);
-			i2.connectTo(t2);
-			t1.connectTo(t3);
-			t2.connectTo(t3);
-			t3.connectTo(o1);
-
-			LibraryCacheManager.register(jobID, new String[0]);
-
-			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
-
-			// test instance types in ExecutionGraph
-			final ExecutionStage executionStage = eg.getCurrentExecutionStage();
-			assertEquals(2, executionStage.getMaxNumberSubtasks());
-
-			// stage0
-			final ExecutionStage es = eg.getStage(0);
-			ExecutionGroupVertex egv0 = null; // input1
-			ExecutionGroupVertex egv1 = null; // input2
-			ExecutionGroupVertex egv2 = null; // task1
-			ExecutionGroupVertex egv3 = null; // task2
-			ExecutionGroupVertex egv4 = null; // task3
-			ExecutionGroupVertex egv5 = null; // output1
-			if (es.getStageMember(0).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(0);
-			} else if (es.getStageMember(0).getName().equals("Input 2")) {
-				egv1 = es.getStageMember(0);
-			} else if (es.getStageMember(0).getName().equals("Task 1")) {
-				egv2 = es.getStageMember(0);
-			} else if (es.getStageMember(0).getName().equals("Task 2")) {
-				egv3 = es.getStageMember(0);
-			} else if (es.getStageMember(0).getName().equals("Task 3")) {
-				egv4 = es.getStageMember(0);
-			} else {
-				egv5 = es.getStageMember(0);
-			}
-
-			if (es.getStageMember(1).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(1);
-			} else if (es.getStageMember(1).getName().equals("Input 2")) {
-				egv1 = es.getStageMember(1);
-			} else if (es.getStageMember(1).getName().equals("Task 1")) {
-				egv2 = es.getStageMember(1);
-			} else if (es.getStageMember(1).getName().equals("Task 2")) {
-				egv3 = es.getStageMember(1);
-			} else if (es.getStageMember(1).getName().equals("Task 3")) {
-				egv4 = es.getStageMember(1);
-			} else {
-				egv5 = es.getStageMember(1);
-			}
-			if (es.getStageMember(2).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(2);
-			} else if (es.getStageMember(2).getName().equals("Input 2")) {
-				egv1 = es.getStageMember(2);
-			} else if (es.getStageMember(2).getName().equals("Task 1")) {
-				egv2 = es.getStageMember(2);
-			} else if (es.getStageMember(2).getName().equals("Task 2")) {
-				egv3 = es.getStageMember(2);
-			} else if (es.getStageMember(2).getName().equals("Task 3")) {
-				egv4 = es.getStageMember(2);
-			} else {
-				egv5 = es.getStageMember(2);
-			}
-			if (es.getStageMember(3).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(3);
-			} else if (es.getStageMember(3).getName().equals("Input 2")) {
-				egv1 = es.getStageMember(3);
-			} else if (es.getStageMember(3).getName().equals("Task 1")) {
-				egv2 = es.getStageMember(3);
-			} else if (es.getStageMember(3).getName().equals("Task 2")) {
-				egv3 = es.getStageMember(3);
-			} else if (es.getStageMember(3).getName().equals("Task 3")) {
-				egv4 = es.getStageMember(3);
-			} else {
-				egv5 = es.getStageMember(3);
-			}
-			if (es.getStageMember(4).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(4);
-			} else if (es.getStageMember(4).getName().equals("Input 2")) {
-				egv1 = es.getStageMember(4);
-			} else if (es.getStageMember(4).getName().equals("Task 1")) {
-				egv2 = es.getStageMember(4);
-			} else if (es.getStageMember(4).getName().equals("Task 2")) {
-				egv3 = es.getStageMember(4);
-			} else if (es.getStageMember(4).getName().equals("Task 3")) {
-				egv4 = es.getStageMember(4);
-			} else {
-				egv5 = es.getStageMember(4);
-			}
-			if (es.getStageMember(5).getName().equals("Input 1")) {
-				egv0 = es.getStageMember(5);
-			} else if (es.getStageMember(5).getName().equals("Input 2")) {
-				egv1 = es.getStageMember(5);
-			} else if (es.getStageMember(5).getName().equals("Task 1")) {
-				egv2 = es.getStageMember(5);
-			} else if (es.getStageMember(5).getName().equals("Task 2")) {
-				egv3 = es.getStageMember(5);
-			} else if (es.getStageMember(5).getName().equals("Task 3")) {
-				egv4 = es.getStageMember(5);
-			} else {
-				egv5 = es.getStageMember(5);
-			}
-			final ExecutionVertex i1_0 = egv0.getGroupMember(0); // input1
-			final ExecutionVertex i1_1 = egv0.getGroupMember(1); // input1
-			final ExecutionVertex i2_0 = egv1.getGroupMember(0); // input2
-			final ExecutionVertex i2_1 = egv1.getGroupMember(1); // input2
-			final ExecutionVertex t1_0 = egv2.getGroupMember(0); // task1
-			final ExecutionVertex t1_1 = egv2.getGroupMember(1); // task1
-			final ExecutionVertex t2_0 = egv3.getGroupMember(0); // task2
-			final ExecutionVertex t2_1 = egv3.getGroupMember(1); // task2
-			final ExecutionVertex t3_0 = egv4.getGroupMember(0); // task3
-			final ExecutionVertex t3_1 = egv4.getGroupMember(1); // task3
-			final ExecutionVertex o1_0 = egv5.getGroupMember(0); // output1
-			final ExecutionVertex o1_1 = egv5.getGroupMember(1); // otuput1
-
-			// instance 1
-			assertTrue((t1_0.getAllocatedResource().equals(i1_0.getAllocatedResource()) && !t1_0.getAllocatedResource()
-				.equals(i1_1.getAllocatedResource()))
-				|| (!t1_0.getAllocatedResource().equals(i1_0.getAllocatedResource()) && t1_0.getAllocatedResource()
-					.equals(i1_1.getAllocatedResource())));
-			assertTrue((t1_0.getAllocatedResource().equals(i2_0.getAllocatedResource()) && !t1_0.getAllocatedResource()
-				.equals(i2_1.getAllocatedResource()))
-				|| (!t1_0.getAllocatedResource().equals(i2_0.getAllocatedResource()) && t1_0.getAllocatedResource()
-					.equals(i2_1.getAllocatedResource())));
-			assertTrue((t1_0.getAllocatedResource().equals(t2_0.getAllocatedResource()) && !t1_0.getAllocatedResource()
-				.equals(t2_1.getAllocatedResource()))
-				|| (!t1_0.getAllocatedResource().equals(t2_0.getAllocatedResource()) && t1_0.getAllocatedResource()
-					.equals(t2_1.getAllocatedResource())));
-			assertTrue((t1_0.getAllocatedResource().equals(t3_0.getAllocatedResource()) && !t1_0.getAllocatedResource()
-				.equals(t3_1.getAllocatedResource()))
-				|| (!t1_0.getAllocatedResource().equals(t3_0.getAllocatedResource()) && t1_0.getAllocatedResource()
-					.equals(t3_1.getAllocatedResource())));
-			assertTrue((t1_0.getAllocatedResource().equals(o1_0.getAllocatedResource()) && !t1_0.getAllocatedResource()
-				.equals(o1_1.getAllocatedResource()))
-				|| (!t1_0.getAllocatedResource().equals(o1_0.getAllocatedResource()) && t1_0.getAllocatedResource()
-					.equals(o1_1.getAllocatedResource())));
-			// instance 2
-			assertTrue((t1_1.getAllocatedResource().equals(i1_0.getAllocatedResource()) && !t1_1.getAllocatedResource()
-				.equals(i1_1.getAllocatedResource()))
-				|| (!t1_1.getAllocatedResource().equals(i1_0.getAllocatedResource()) && t1_1.getAllocatedResource()
-					.equals(i1_1.getAllocatedResource())));
-			assertTrue((t1_1.getAllocatedResource().equals(i2_0.getAllocatedResource()) && !t1_1.getAllocatedResource()
-				.equals(i2_1.getAllocatedResource()))
-				|| (!t1_1.getAllocatedResource().equals(i2_0.getAllocatedResource()) && t1_1.getAllocatedResource()
-					.equals(i2_1.getAllocatedResource())));
-			assertTrue((t1_1.getAllocatedResource().equals(t2_0.getAllocatedResource()) && !t1_1.getAllocatedResource()
-				.equals(t2_1.getAllocatedResource()))
-				|| (!t1_1.getAllocatedResource().equals(t2_0.getAllocatedResource()) && t1_1.getAllocatedResource()
-					.equals(t2_1.getAllocatedResource())));
-			assertTrue((t1_1.getAllocatedResource().equals(t3_0.getAllocatedResource()) && !t1_1.getAllocatedResource()
-				.equals(t3_1.getAllocatedResource()))
-				|| (!t1_1.getAllocatedResource().equals(t3_0.getAllocatedResource()) && t1_1.getAllocatedResource()
-					.equals(t3_1.getAllocatedResource())));
-			assertTrue((t1_1.getAllocatedResource().equals(o1_0.getAllocatedResource()) && !t1_1.getAllocatedResource()
-				.equals(o1_1.getAllocatedResource()))
-				|| (!t1_1.getAllocatedResource().equals(o1_0.getAllocatedResource()) && t1_1.getAllocatedResource()
-					.equals(o1_1.getAllocatedResource())));
-		} catch (GraphConversionException e) {
-			fail(e.getMessage());
-		} catch (IOException e) {
-			fail(e.getMessage());
-		} catch (JobGraphDefinitionException e) {
-			fail(e.getMessage());
-		} finally {
-			if (inputFile1 != null) {
-				inputFile1.delete();
-			}
-			if (inputFile2 != null) {
-				inputFile2.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jobID != null) {
-				try {
-					LibraryCacheManager.unregister(jobID);
-				} catch (IOException ioe) {
-				}
-			}
-		}
-	}
-
-	/*
-	 * input1 -> task1 -> output1
-	 * -> task3 -> task4
-	 * input2 -> task2 -> output2
-	 * all subtasks defined
-	 * all instance types defined
-	 * all channel types defined
-	 */
-	@Test
-	public void testConvertJobGraphToExecutionGraph4() {
-
-		File inputFile1 = null;
-		File inputFile2 = null;
-		File outputFile1 = null;
-		File outputFile2 = null;
-		JobID jobID = null;
-
-		try {
-
-			inputFile1 = ServerTestUtils.createInputFile(0);
-			inputFile2 = ServerTestUtils.createInputFile(0);
-			outputFile1 = new File(ServerTestUtils.getRandomFilename());
-			outputFile2 = new File(ServerTestUtils.getRandomFilename());
-
-			// create job graph
-			final JobGraph jg = new JobGraph("Job Graph 1");
-			jobID = jg.getJobID();
-
-			// input vertex
-			final JobInputVertex i1 = new JobInputVertex("Input 1", jg);
-			i1.setInvokableClass(DataSourceTask.class);
-			i1.setInputFormat(new TextInputFormat(new Path(inputFile1.toURI())));
-			i1.setNumberOfSubtasks(4);
-			final JobInputVertex i2 = new JobInputVertex("Input 2", jg);
-			i2.setInvokableClass(DataSourceTask.class);
-			i2.setInputFormat(new TextInputFormat(new Path(inputFile2.toURI())));
-			i2.setNumberOfSubtasks(4);
-			// task vertex
-			final JobTaskVertex t1 = new JobTaskVertex("Task 1", jg);
-			t1.setInvokableClass(ForwardTask1Input1Output.class);
-			t1.setNumberOfSubtasks(4);
-			final JobTaskVertex t2 = new JobTaskVertex("Task 2", jg);
-			t2.setInvokableClass(ForwardTask1Input1Output.class);
-			t2.setNumberOfSubtasks(4);
-			final JobTaskVertex t3 = new JobTaskVertex("Task 3", jg);
-			t3.setInvokableClass(ForwardTask2Inputs1Output.class);
-			t3.setNumberOfSubtasks(8);
-			final JobTaskVertex t4 = new JobTaskVertex("Task 4", jg);
-			t4.setInvokableClass(ForwardTask1Input2Outputs.class);
-			t4.setNumberOfSubtasks(8);
-			// output vertex
-			final JobOutputVertex o1 = new JobOutputVertex("Output 1", jg);
-			o1.setInvokableClass(DataSinkTask.class);
-			o1.setOutputFormat(new DiscardingOuputFormat<Object>());
-			o1.setNumberOfSubtasks(4);
-			final JobOutputVertex o2 = new JobOutputVertex("Output 2", jg);
-			o2.setInvokableClass(DataSinkTask.class);
-			o2.setOutputFormat(new DiscardingOuputFormat<Object>());
-			o2.setNumberOfSubtasks(4);
-			o1.setVertexToShareInstancesWith(o2);
-
-			// connect vertices
-			i1.connectTo(t1, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-			i2.connectTo(t2, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-			t1.connectTo(t3, ChannelType.NETWORK);
-			t2.connectTo(t3, ChannelType.NETWORK);
-			t3.connectTo(t4, ChannelType.IN_MEMORY, DistributionPattern.POINTWISE);
-			t4.connectTo(o1, ChannelType.NETWORK);
-			t4.connectTo(o2, ChannelType.NETWORK);
-
-			LibraryCacheManager.register(jobID, new String[0]);
-
-			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
-
-			// test instance types in ExecutionGraph
-			ExecutionStage executionStage = eg.getCurrentExecutionStage();
-			assertNotNull(executionStage);
-			assertEquals(0, executionStage.getStageNumber());
-			
-			assertEquals(20, executionStage.getRequiredSlots());
-			// Fake transition to next stage by triggering execution state changes manually
-			final Iterator<ExecutionVertex> it = new ExecutionGraphIterator(eg, eg.getIndexOfCurrentExecutionStage(),
-				true, true);
-
-			while (it.hasNext()) {
-				final ExecutionVertex ev = it.next();
-				ev.updateExecutionState(ExecutionState.SCHEDULED);
-				ev.updateExecutionState(ExecutionState.ASSIGNED);
-				ev.updateExecutionState(ExecutionState.READY);
-				ev.updateExecutionState(ExecutionState.STARTING);
-				ev.updateExecutionState(ExecutionState.RUNNING);
-				ev.updateExecutionState(ExecutionState.FINISHING);
-				ev.updateExecutionState(ExecutionState.FINISHED);
-			}
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		} finally {
-			if (inputFile1 != null) {
-				inputFile1.delete();
-			}
-			if (inputFile2 != null) {
-				inputFile2.delete();
-			}
-			if (outputFile1 != null) {
-				outputFile1.delete();
-			}
-			if (outputFile2 != null) {
-				outputFile2.delete();
-			}
-			if (jobID != null) {
-				try {
-					LibraryCacheManager.unregister(jobID);
-				} catch (IOException e) {
-				}
-			}
-		}
-	}
-
-	/**
-	 * Tests the conversion of a job graph representing a self cross to an execution graph.
-	 */
-	@Test
-	public void testConvertSelfCross() {
-
-		final String inputTaskName = "Self Cross Input";
-		final String crossTaskName = "Self Cross Task";
-		final String outputTaskName = "Self Cross Output";
-		final int degreeOfParallelism = 4;
-		File inputFile = null;
-		File outputFile = null;
-		JobID jobID = null;
-
-		try {
-			inputFile = ServerTestUtils.createInputFile(0);
-			outputFile = new File(ServerTestUtils.getRandomFilename());
-
-			// create job graph
-			final JobGraph jg = new JobGraph("Self Cross Test Job");
-			jobID = jg.getJobID();
-
-			// input vertex
-			final JobInputVertex input = new JobInputVertex(inputTaskName, jg);
-			input.setInvokableClass(DataSourceTask.class);
-			input.setInputFormat(new TextInputFormat(new Path(inputFile.toURI())));
-			input.setNumberOfSubtasks(degreeOfParallelism);
-
-			// cross vertex
-			final JobTaskVertex cross = new JobTaskVertex(crossTaskName, jg);
-			cross.setInvokableClass(SelfCrossForwardTask.class);
-			cross.setNumberOfSubtasks(degreeOfParallelism);
-
-			// output vertex
-			final JobOutputVertex output = new JobOutputVertex(outputTaskName, jg);
-			output.setInvokableClass(DataSinkTask.class);
-			output.setOutputFormat(new DiscardingOuputFormat<Object>());
-			output.setNumberOfSubtasks(degreeOfParallelism);
-
-			// connect vertices
-			input.connectTo(cross, ChannelType.IN_MEMORY, 0, 0,
-				DistributionPattern.POINTWISE);
-			input.connectTo(cross, ChannelType.NETWORK, 1, 1,
-				DistributionPattern.BIPARTITE);
-			cross.connectTo(output, ChannelType.IN_MEMORY, 0, 0,
-				DistributionPattern.POINTWISE);
-
-			LibraryCacheManager.register(jobID, new String[0]);
-
-			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
-
-			assertEquals(1, eg.getNumberOfStages());
-
-			final ExecutionStage stage = eg.getStage(0);
-
-			assertEquals(3, stage.getNumberOfStageMembers());
-
-			ExecutionGroupVertex inputGroupVertex = null;
-			ExecutionGroupVertex crossGroupVertex = null;
-			ExecutionGroupVertex outputGroupVertex = null;
-			final ExecutionGroupVertexIterator groupIt = new ExecutionGroupVertexIterator(eg, true, -1);
-			while (groupIt.hasNext()) {
-
-				ExecutionGroupVertex gv = groupIt.next();
-				if (inputTaskName.equals(gv.getName())) {
-					inputGroupVertex = gv;
-				} else if (crossTaskName.equals(gv.getName())) {
-					crossGroupVertex = gv;
-				} else if (outputTaskName.equals(gv.getName())) {
-					outputGroupVertex = gv;
-				}
-			}
-
-			assertNotNull(inputGroupVertex);
-			assertNotNull(crossGroupVertex);
-			assertNotNull(outputGroupVertex);
-
-			assertEquals(degreeOfParallelism, inputGroupVertex.getCurrentNumberOfGroupMembers());
-			assertEquals(degreeOfParallelism, crossGroupVertex.getCurrentNumberOfGroupMembers());
-			assertEquals(degreeOfParallelism, outputGroupVertex.getCurrentNumberOfGroupMembers());
-
-			// Check that all subtasks on a pipeline share the same instance
-			assertEquals(inputGroupVertex.getGroupMember(0).getAllocatedResource(), crossGroupVertex.getGroupMember(0)
-				.getAllocatedResource());
-			assertEquals(inputGroupVertex.getGroupMember(1).getAllocatedResource(), crossGroupVertex.getGroupMember(1)
-				.getAllocatedResource());
-			assertEquals(inputGroupVertex.getGroupMember(2).getAllocatedResource(), crossGroupVertex.getGroupMember(2)
-				.getAllocatedResource());
-			assertEquals(inputGroupVertex.getGroupMember(3).getAllocatedResource(), crossGroupVertex.getGroupMember(3)
-				.getAllocatedResource());
-
-			assertEquals(crossGroupVertex.getGroupMember(0).getAllocatedResource(), outputGroupVertex.getGroupMember(0)
-				.getAllocatedResource());
-			assertEquals(crossGroupVertex.getGroupMember(1).getAllocatedResource(), outputGroupVertex.getGroupMember(1)
-				.getAllocatedResource());
-			assertEquals(crossGroupVertex.getGroupMember(2).getAllocatedResource(), outputGroupVertex.getGroupMember(2)
-				.getAllocatedResource());
-			assertEquals(crossGroupVertex.getGroupMember(3).getAllocatedResource(), outputGroupVertex.getGroupMember(3)
-				.getAllocatedResource());
-
-			// Check that all subtasks on different pipelines run on different instances
-			assertFalse(inputGroupVertex.getGroupMember(0).getAllocatedResource()
-				.equals(inputGroupVertex.getGroupMember(1).getAllocatedResource()));
-			assertFalse(inputGroupVertex.getGroupMember(1).getAllocatedResource()
-				.equals(inputGroupVertex.getGroupMember(2).getAllocatedResource()));
-			assertFalse(inputGroupVertex.getGroupMember(2).getAllocatedResource()
-				.equals(inputGroupVertex.getGroupMember(3).getAllocatedResource()));
-
-		} catch (GraphConversionException e) {
-			fail(e.getMessage());
-		} catch (JobGraphDefinitionException e) {
-			fail(e.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		} finally {
-			if (inputFile != null) {
-				inputFile.delete();
-			}
-			if (outputFile != null) {
-				outputFile.delete();
-			}
-			if (jobID != null) {
-				try {
-					LibraryCacheManager.unregister(jobID);
-				} catch (IOException e) {
-				}
-			}
-		}
-	}
-
-	/**
-	 * This test checks the correctness of the instance sharing API. In particular, the test checks the behavior of the
-	 * instance sharing as reported broken in ticket #198
-	 */
-	@Test
-	public void testInstanceSharing() {
-
-		final int degreeOfParallelism = 4;
-		File inputFile1 = null;
-		File outputFile1 = null;
-		JobID jobID = null;
-
-		try {
-
-			inputFile1 = ServerTestUtils.createInputFile(0);
-			outputFile1 = new File(ServerTestUtils.getRandomFilename());
-
-			// create job graph
-			final JobGraph jg = new JobGraph("Instance Sharing Test Job");
-			jobID = jg.getJobID();
-
-			// input vertex
-			final JobInputVertex input1 = new JobInputVertex("Input 1", jg);
-			input1.setInvokableClass(DataSourceTask.class);
-			input1.setInputFormat(new TextInputFormat(new Path(inputFile1.toURI())));
-			input1.setNumberOfSubtasks(degreeOfParallelism);
-			
-			
-
-			// forward vertex 1
-			final JobTaskVertex forward1 = new JobTaskVertex("Forward 1", jg);
-			forward1.setInvokableClass(ForwardTask1Input1Output.class);
-			forward1.setNumberOfSubtasks(degreeOfParallelism);
-
-			// forward vertex 2
-			final JobTaskVertex forward2 = new JobTaskVertex("Forward 2", jg);
-			forward2.setInvokableClass(ForwardTask1Input1Output.class);
-			forward2.setNumberOfSubtasks(degreeOfParallelism);
-
-			// forward vertex 3
-			final JobTaskVertex forward3 = new JobTaskVertex("Forward 3", jg);
-			forward3.setInvokableClass(ForwardTask1Input1Output.class);
-			forward3.setNumberOfSubtasks(degreeOfParallelism);
-
-			// output vertex
-			final JobOutputVertex output1 = new JobOutputVertex("Output 1", jg);
-			output1.setInvokableClass(DataSinkTask.class);
-			output1.setOutputFormat(new DiscardingOuputFormat<Object>());
-			output1.setNumberOfSubtasks(degreeOfParallelism);
-
-			// connect vertices
-			input1.connectTo(forward1, ChannelType.IN_MEMORY,
-				DistributionPattern.POINTWISE);
-			forward1.connectTo(forward2, ChannelType.IN_MEMORY,
-					DistributionPattern.POINTWISE);
-			forward2.connectTo(forward3, ChannelType.NETWORK,
-					DistributionPattern.POINTWISE);
-			forward3.connectTo(output1, ChannelType.IN_MEMORY);
-
-			// setup instance sharing
-			input1.setVertexToShareInstancesWith(forward1);
-			forward1.setVertexToShareInstancesWith(forward2);
-			forward2.setVertexToShareInstancesWith(forward3);
-			forward3.setVertexToShareInstancesWith(output1);
-
-			LibraryCacheManager.register(jobID, new String[0]);
-
-			// now convert job graph to execution graph
-			final ExecutionGraph eg = new ExecutionGraph(jg, 1);
-
-			// Check number of stages
-			assertEquals(1, eg.getNumberOfStages());
-
-			// Check number of vertices in stage
-			final ExecutionStage stage = eg.getStage(0);
-			assertEquals(5, stage.getNumberOfStageMembers());
-
-			final int numberOfRequiredSlots = stage.getMaxNumberSubtasks();
-			assertEquals(degreeOfParallelism, numberOfRequiredSlots);
-
-		} catch (GraphConversionException e) {
-			fail(e.getMessage());
-		} catch (JobGraphDefinitionException e) {
-			fail(e.getMessage());
-		} catch (IOException ioe) {
-			fail(ioe.getMessage());
-		} finally {
-			if (inputFile1 != null) {
-				inputFile1.delete();
-			}
-			if (outputFile1 != null) {
-				outputFile1.delete();
-			}
-			if (jobID != null) {
-				try {
-					LibraryCacheManager.unregister(jobID);
-				} catch (IOException e) {
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
new file mode 100644
index 0000000..2207475
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphTestUtils.java
@@ -0,0 +1,179 @@
+/**
+ * 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 static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+import java.lang.reflect.Field;
+import java.net.InetAddress;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.HardwareDescription;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
+import org.apache.flink.runtime.protocols.TaskOperationProtocol;
+import org.mockito.Matchers;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class ExecutionGraphTestUtils {
+
+	// --------------------------------------------------------------------------------------------
+	//  state modifications
+	// --------------------------------------------------------------------------------------------
+	
+	public static void setVertexState(ExecutionVertex2 vertex, ExecutionState2 state) {
+		try {
+			Field f = ExecutionVertex2.class.getDeclaredField("state");
+			f.setAccessible(true);
+			f.set(vertex, state);
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Modifying the state failed", e);
+		}
+	}
+	
+	public static void setVertexResource(ExecutionVertex2 vertex, AllocatedSlot slot) {
+		try {
+			Field f = ExecutionVertex2.class.getDeclaredField("assignedSlot");
+			f.setAccessible(true);
+			f.set(vertex, slot);
+		}
+		catch (Exception e) {
+			throw new RuntimeException("Modifying the slot failed", e);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  utility mocking methods
+	// --------------------------------------------------------------------------------------------
+	
+	public static Instance getInstance(final TaskOperationProtocol top) throws Exception {
+		HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024);
+		InetAddress address = InetAddress.getByName("127.0.0.1");
+		InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10000, 10001);
+		
+		return new Instance(connection, new InstanceID(), hardwareDescription, 1) {
+			@Override
+			public TaskOperationProtocol getTaskManagerProxy() {
+				return top;
+			}
+		};
+	}
+	
+	public static ExecutionJobVertex getJobVertexNotExecuting(JobVertexID id) throws JobException {
+		ExecutionJobVertex ejv = getJobVertexBase(id);
+		
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				return null;
+			}
+		}).when(ejv).execute(Matchers.any(Runnable.class));
+		
+		return ejv;
+	}
+	
+	public static ExecutionJobVertex getJobVertexExecutingSynchronously(JobVertexID id) throws JobException {
+		ExecutionJobVertex ejv = getJobVertexBase(id);
+		
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				Runnable r = (Runnable) invocation.getArguments()[0];
+				r.run();
+				return null;
+			}
+		}).when(ejv).execute(Matchers.any(Runnable.class));
+		
+		return ejv;
+	}
+	
+	public static ExecutionJobVertex getJobVertexExecutingAsynchronously(JobVertexID id) throws JobException {
+		ExecutionJobVertex ejv = getJobVertexBase(id);
+		
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				Runnable r = (Runnable) invocation.getArguments()[0];
+				new Thread(r).start();
+				return null;
+			}
+		}).when(ejv).execute(Matchers.any(Runnable.class));
+		
+		return ejv;
+	}
+	
+	public static ExecutionJobVertex getJobVertexExecutingTriggered(JobVertexID id, final ActionQueue queue) throws JobException {
+		ExecutionJobVertex ejv = getJobVertexBase(id);
+		
+		doAnswer(new Answer<Void>() {
+			@Override
+			public Void answer(InvocationOnMock invocation) {
+				
+				final Runnable action = (Runnable) invocation.getArguments()[0];
+				queue.queueAction(action);
+				return null;
+			}
+		}).when(ejv).execute(Matchers.any(Runnable.class));
+		
+		return ejv;
+	}
+	
+	private static ExecutionJobVertex getJobVertexBase(JobVertexID id) throws JobException {
+		AbstractJobVertex ajv = new AbstractJobVertex("TestVertex", id);
+		ajv.setInvokableClass(mock(AbstractInvokable.class).getClass());
+		
+		ExecutionGraph graph = new ExecutionGraph(new JobID(), "test job", new Configuration());
+		
+		return spy(new ExecutionJobVertex(graph, ajv, 1));
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static final class ActionQueue {
+		
+		private final LinkedBlockingQueue<Runnable> runnables = new LinkedBlockingQueue<Runnable>();
+		
+		public void triggerNextAction() {
+			Runnable r = runnables.remove();
+			r.run();
+		}
+		
+		public Runnable popNextAction() {
+			Runnable r = runnables.remove();
+			return r;
+		}
+
+		public void queueAction(Runnable r) {
+			this.runnables.add(r);
+		}
+	}
+}


[12/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphIterator.java
deleted file mode 100644
index f4efc88..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphIterator.java
+++ /dev/null
@@ -1,444 +0,0 @@
-/**
- * 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 java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.Stack;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * This class provides an implementation of the {@link Iterator} interface which allows to
- * traverse an execution graph and visit every reachable vertex exactly once. The order
- * in which the vertices are visited corresponds to the order of their discovery in a depth first
- * search.
- * <p>
- * This class is not thread-safe.
- * 
- */
-public class ExecutionGraphIterator implements Iterator<ExecutionVertex> {
-
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(ExecutionGraphIterator.class);
-
-	/**
-	 * The execution this iterator traverses.
-	 */
-	private final ExecutionGraph executionGraph;
-
-	/**
-	 * Stores whether the graph is traversed starting from the input or the output vertices.
-	 */
-	private final boolean forward;
-
-	/**
-	 * The stage that should be traversed by this iterator.
-	 */
-	private final int startStage;
-
-	/**
-	 * Stores whether the iterator is confined to the start stage or not.
-	 */
-	private final boolean confinedToStage;
-
-	/**
-	 * The number of visited vertices from the entry set (either input or output vertices).
-	 */
-	private int numVisitedEntryVertices = 0;
-
-	/**
-	 * Stack used for the depth first search.
-	 */
-	private final Stack<TraversalEntry> traversalStack = new Stack<TraversalEntry>();
-
-	/**
-	 * Set of already visited vertices during traversal.
-	 */
-	private final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-
-	/**
-	 * Auxiliary class which stores which vertices have already been visited.
-	 * 
-	 */
-	private static class TraversalEntry {
-
-		/**
-		 * Execution vertex this entry has been created for.
-		 */
-		private final ExecutionVertex executionVertex;
-
-		/**
-		 * Next gate to traverse.
-		 */
-		private int currentGate;
-
-		/**
-		 * Next channel to traverse.
-		 */
-		private int currentChannel;
-
-		/**
-		 * Constructs a new traversal entry.
-		 * 
-		 * @param executionVertex
-		 *        the execution vertex this entry belongs to
-		 * @param currentGate
-		 *        the gate index to use to visit the next vertex
-		 * @param currentChannel
-		 *        the channel index to use to visit the next vertex
-		 */
-		public TraversalEntry(final ExecutionVertex executionVertex, final int currentGate, final int currentChannel) {
-			this.executionVertex = executionVertex;
-			this.currentGate = currentGate;
-			this.currentChannel = currentChannel;
-		}
-
-		/**
-		 * Returns the execution vertex this entry belongs to.
-		 * 
-		 * @return the execution vertex this entry belongs to
-		 */
-		public ExecutionVertex getExecutionVertex() {
-			return this.executionVertex;
-		}
-
-		/**
-		 * Returns the gate index to use to visit the next vertex.
-		 * 
-		 * @return the gate index to use to visit the next vertex
-		 */
-		public int getCurrentGate() {
-			return this.currentGate;
-		}
-
-		/**
-		 * Returns the channel index to use to visit the next vertex.
-		 * 
-		 * @return the channel index to use to visit the next vertex
-		 */
-		public int getCurrentChannel() {
-			return this.currentChannel;
-		}
-
-		/**
-		 * Increases the channel index by one.
-		 */
-		public void increaseCurrentChannel() {
-			this.currentChannel++;
-		}
-
-		/**
-		 * Increases the gate index by one.
-		 */
-		public void increaseCurrentGate() {
-			this.currentGate++;
-		}
-
-		/**
-		 * Resets the channel index.
-		 */
-		public void resetCurrentChannel() {
-			this.currentChannel = 0;
-		}
-
-	}
-
-	/**
-	 * Creates a new execution graph iterator.
-	 * 
-	 * @param executionGraph
-	 *        the execution graph that should be traversed
-	 * @param forward
-	 *        <code>true</code> if the graph should be traversed in correct order, <code>false</code> to traverse it in
-	 *        reverse order
-	 */
-	public ExecutionGraphIterator(final ExecutionGraph executionGraph, final boolean forward) {
-		this(executionGraph, forward ? 0 : (executionGraph.getNumberOfStages() - 1), false, forward);
-	}
-
-	/**
-	 * Creates a new execution graph iterator.
-	 * 
-	 * @param executionGraph
-	 *        the execution graph that should be traversed
-	 * @param startStage
-	 *        the index of the stage of the graph where the traversal is supposed to begin
-	 * @param confinedToStage
-	 *        <code>false</code> if the graph iterator is allowed to traverse to upper (in case of reverse order
-	 *        traversal lower) stages, <code>true</code> otherwise.
-	 * @param forward
-	 *        <code>true</code> if the graph should be traversed in correct order, <code>false</code> to traverse it in
-	 *        reverse order
-	 */
-	public ExecutionGraphIterator(final ExecutionGraph executionGraph, final int startStage,
-			final boolean confinedToStage, final boolean forward) {
-
-		this.executionGraph = executionGraph;
-		this.forward = forward;
-		this.startStage = startStage;
-		this.confinedToStage = confinedToStage;
-
-		if (startStage >= this.executionGraph.getNumberOfStages()) {
-			return;
-		}
-
-		if (forward) {
-			if (executionGraph.getNumberOfInputVertices(startStage) > 0) {
-
-				final TraversalEntry te = new TraversalEntry(executionGraph.getInputVertex(startStage, 0), 0, 0);
-				this.traversalStack.push(te);
-				this.alreadyVisited.add(te.getExecutionVertex());
-
-			}
-		} else {
-			if (executionGraph.getNumberOfOutputVertices(startStage) > 0) {
-
-				final TraversalEntry te = new TraversalEntry(executionGraph.getOutputVertex(startStage, 0), 0, 0);
-				this.traversalStack.push(te);
-				this.alreadyVisited.add(te.getExecutionVertex());
-			}
-
-		}
-	}
-
-	/**
-	 * Creates a new execution graph iterator. This constructor can be used to
-	 * traverse only specific parts of the graph starting at <code>startVertex</code>.
-	 * The iterator will not switch to the next input/output vertex of an output/input vertex
-	 * has been reached.
-	 * 
-	 * @param executionGraph
-	 *        the execution graph that should be traversed
-	 * @param startVertex
-	 *        the vertex to start the traversal from
-	 * @param forward
-	 *        <code>true</code> if the graph should be traversed in correct order, <code>false</code> to reverse it in
-	 *        reverse order
-	 */
-	public ExecutionGraphIterator(final ExecutionGraph executionGraph, final ExecutionVertex startVertex,
-			final boolean forward) {
-
-		this.executionGraph = executionGraph;
-		this.forward = forward;
-		this.numVisitedEntryVertices = -1;
-		this.startStage = 0;
-		this.confinedToStage = false;
-
-		final TraversalEntry te = new TraversalEntry(startVertex, 0, 0);
-		this.traversalStack.push(te);
-		this.alreadyVisited.add(startVertex);
-	}
-
-
-	@Override
-	public boolean hasNext() {
-
-		if (this.traversalStack.isEmpty()) {
-
-			if (this.numVisitedEntryVertices < 0) {
-				// User chose a specific starting vertex
-				return false;
-			}
-
-			++this.numVisitedEntryVertices;
-
-			if (this.forward) {
-				if (this.executionGraph.getNumberOfInputVertices(this.startStage) <= this.numVisitedEntryVertices) {
-					return false;
-				}
-			} else {
-				if (this.executionGraph.getNumberOfOutputVertices(this.startStage) <= this.numVisitedEntryVertices) {
-					return false;
-				}
-			}
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public ExecutionVertex next() {
-
-		if (this.traversalStack.isEmpty()) {
-
-			if (this.numVisitedEntryVertices < 0) {
-				// User chose a specific entry vertex
-				return null;
-			}
-
-			TraversalEntry newentry;
-
-			if (this.forward) {
-				newentry = new TraversalEntry(this.executionGraph.getInputVertex(this.startStage,
-					this.numVisitedEntryVertices),
-					0, 0);
-			} else {
-				newentry = new TraversalEntry(this.executionGraph.getOutputVertex(this.startStage,
-					this.numVisitedEntryVertices),
-					0, 0);
-			}
-
-			this.traversalStack.push(newentry);
-			this.alreadyVisited.add(newentry.getExecutionVertex());
-		}
-
-		final ExecutionVertex returnVertex = this.traversalStack.peek().getExecutionVertex();
-
-		// Propose vertex to be visited next
-		do {
-
-			final TraversalEntry te = this.traversalStack.peek();
-
-			// Check if we can traverse deeper into the graph
-			final ExecutionVertex candidateVertex = getCandidateVertex(te, forward);
-			if (candidateVertex == null) {
-				// Pop it from the stack
-				this.traversalStack.pop();
-			} else {
-				// Create new entry and put it on the stack
-				final TraversalEntry newte = new TraversalEntry(candidateVertex, 0, 0);
-				this.traversalStack.push(newte);
-				this.alreadyVisited.add(candidateVertex);
-				break;
-			}
-
-		} while (!this.traversalStack.isEmpty());
-
-		return returnVertex;
-	}
-
-	/**
-	 * Returns a candidate vertex which could potentially be visited next because it is reachable from the
-	 * currently considered vertex.
-	 * 
-	 * @param te
-	 *        the traversal entry for the current source vertex
-	 * @param forward
-	 *        <code>true</code> if the graph should be traversed in correct order, <code>false</code> to traverse it in
-	 *        reverse order
-	 * @return a candidate vertex which could potentially be visited next
-	 */
-	private ExecutionVertex getCandidateVertex(final TraversalEntry te, final boolean forward) {
-
-		if (forward) {
-
-			while (true) {
-
-				if (this.confinedToStage && te.getCurrentChannel() == 0) {
-					while (currentGateLeadsToOtherStage(te, this.forward)) {
-						te.increaseCurrentGate();
-					}
-				}
-
-				// No more outgoing edges to consider
-				if (te.getCurrentGate() >= te.getExecutionVertex().getNumberOfOutputGates()) {
-					break;
-				}
-
-				if (te.getCurrentChannel() >= te.getExecutionVertex().getOutputGate(te.getCurrentGate())
-					.getNumberOfEdges()) {
-					te.increaseCurrentGate();
-					te.resetCurrentChannel();
-				} else {
-					final ExecutionEdge outputChannel = te.getExecutionVertex().getOutputGate(te.getCurrentGate())
-						.getEdge(te.getCurrentChannel());
-					final ExecutionVertex tmp = outputChannel.getInputGate().getVertex();
-					if (tmp == null) {
-						LOG.error("Inconsistency in vertex map found (forward)!");
-					}
-					te.increaseCurrentChannel();
-					if (!this.alreadyVisited.contains(tmp)) {
-						return tmp;
-					}
-				}
-			}
-		} else {
-
-			while (true) {
-
-				// No more outgoing edges to consider
-				if (te.getCurrentGate() >= te.getExecutionVertex().getNumberOfInputGates()) {
-					break;
-				}
-
-				if (te.getCurrentChannel() >= te.getExecutionVertex().getInputGate(te.getCurrentGate())
-					.getNumberOfEdges()) {
-					te.increaseCurrentGate();
-					te.resetCurrentChannel();
-				} else {
-					final ExecutionEdge inputChannel = te.getExecutionVertex().getInputGate(te.getCurrentGate()).getEdge(te.getCurrentChannel());
-					final ExecutionVertex tmp = inputChannel.getOutputGate().getVertex();
-					if (tmp == null) {
-						LOG.error("Inconsistency in vertex map found (backward)!");
-					}
-					te.increaseCurrentChannel();
-					if (!this.alreadyVisited.contains(tmp)) {
-						return tmp;
-					}
-				}
-			}
-		}
-
-		return null;
-	}
-
-	private boolean currentGateLeadsToOtherStage(final TraversalEntry te, final boolean forward) {
-
-		final ExecutionGroupVertex groupVertex = te.getExecutionVertex().getGroupVertex();
-
-		if (forward) {
-
-			if (te.getCurrentGate() >= groupVertex.getNumberOfForwardLinks()) {
-				return false;
-			}
-
-			final ExecutionGroupEdge edge = groupVertex.getForwardEdge(te.getCurrentGate());
-			if (edge.getTargetVertex().getStageNumber() == groupVertex.getStageNumber()) {
-				return false;
-			}
-
-		} else {
-			if (te.getCurrentGate() >= groupVertex.getNumberOfBackwardLinks()) {
-				return false;
-			}
-
-			final ExecutionGroupEdge edge = groupVertex.getBackwardEdge(te.getCurrentGate());
-			if (edge.getSourceVertex().getStageNumber() == groupVertex.getStageNumber()) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public void remove() {
-
-		throw new UnsupportedOperationException("The method remove is not implemented for this type of iterator");
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertexIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertexIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertexIterator.java
deleted file mode 100644
index 1095925..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGroupVertexIterator.java
+++ /dev/null
@@ -1,331 +0,0 @@
-/**
- * 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 java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.Stack;
-
-/**
- * This class provides an implementation of the {@link Iterator} interface which allows to
- * traverse an execution graph and visit every reachable group vertex exactly once. The order
- * in which the group vertices are visited corresponds to the order of their discovery in a depth first
- * search.
- * <p>
- * This class is not thread-safe.
- * 
- */
-public class ExecutionGroupVertexIterator implements Iterator<ExecutionGroupVertex> {
-
-	/**
-	 * Stores whether the group graph is traversed starting from the input or the output vertices.
-	 */
-	private final boolean forward;
-
-	/**
-	 * The stage to traverse, -1 to traverse all stages of the graph.
-	 */
-	private final int stage;
-
-	/**
-	 * List of entry vertices for the traversal (either input or output vertices).
-	 */
-	private final List<ExecutionGroupVertex> entryVertices = new ArrayList<ExecutionGroupVertex>();
-
-	/**
-	 * Number of already visited entry vertices.
-	 */
-	private int numVisitedEntryVertices = 0;
-
-	/**
-	 * Stack used for the traversal.
-	 */
-	private final Stack<TraversalEntry> traversalStack = new Stack<TraversalEntry>();
-
-	/**
-	 * Set storing the vertices already visited during traversal.
-	 */
-	private final Set<ExecutionGroupVertex> alreadyVisited = new HashSet<ExecutionGroupVertex>();
-
-	/**
-	 * Auxiliary class which stores which vertices have already been visited.
-	 * 
-	 */
-	private static class TraversalEntry {
-
-		/**
-		 * The group vertex this entry is created for.
-		 */
-		private final ExecutionGroupVertex groupVertex;
-
-		/**
-		 * The current outgoing link of the group vertex.
-		 */
-		private int currentLink = 0;
-
-		/**
-		 * Constructs a new traversal entry.
-		 * 
-		 * @param groupVertex
-		 *        the group vertex this traversal entry belongs to
-		 * @param currentLink
-		 *        the link index to use to visit the next group vertex
-		 */
-		public TraversalEntry(final ExecutionGroupVertex groupVertex, final int currentLink) {
-			this.groupVertex = groupVertex;
-			this.currentLink = currentLink;
-		}
-
-		/**
-		 * Returns the group vertex this traversal entry belongs to.
-		 * 
-		 * @return the group vertex this traversal entry belongs to
-		 */
-		public ExecutionGroupVertex getGroupVertex() {
-			return this.groupVertex;
-		}
-
-		/**
-		 * Returns the link index to use to visit the next group vertex.
-		 * 
-		 * @return the link index to use to visit the next group vertex
-		 */
-		public int getCurrentLink() {
-			return this.currentLink;
-		}
-
-		/**
-		 * Increases the link index by one.
-		 */
-		public void increaseCurrentLink() {
-			this.currentLink++;
-		}
-	}
-
-	/**
-	 * Creates a new execution group vertex iterator.
-	 * 
-	 * @param executionGraph
-	 *        the execution graph that should be traversed
-	 * @param forward
-	 *        <code>true</code> if the graph should be traversed in correct order, <code>false</code> to reverse it in
-	 *        reverse order
-	 * @param stage
-	 *        the number of the stage whose vertices should be traversed or -1 if all stages should be included in the
-	 *        traversal
-	 */
-	public ExecutionGroupVertexIterator(final ExecutionGraph executionGraph, final boolean forward, final int stage) {
-
-		this.forward = forward;
-		this.stage = stage;
-
-		// Collect start vertices
-		if (stage < 0) {
-
-			for (int i = 0; i < executionGraph.getNumberOfStages(); i++) {
-				collectStartVertices(executionGraph.getStage(i));
-			}
-		} else {
-			if (stage < executionGraph.getNumberOfStages()) {
-				collectStartVertices(executionGraph.getStage(stage));
-			}
-		}
-
-		if (this.entryVertices.size() > 0) {
-			final TraversalEntry te = new TraversalEntry(this.entryVertices.get(0), 0);
-			this.traversalStack.push(te);
-			this.alreadyVisited.add(te.getGroupVertex());
-		}
-	}
-
-	/**
-	 * Collects all input group vertices (i.e. vertices with no incoming link or incoming links from other stages) in
-	 * the given stage and adds them to an internal list.
-	 * 
-	 * @param stage
-	 *        the number of the stage whose input vertices should be collected
-	 */
-	private void collectStartVertices(final ExecutionStage stage) {
-
-		for (int i = 0; i < stage.getNumberOfStageMembers(); i++) {
-
-			final ExecutionGroupVertex groupVertex = stage.getStageMember(i);
-
-			if (forward) {
-				if ((groupVertex.getNumberOfBackwardLinks() == 0)
-					|| ((this.stage >= 0) && allConnectionsFromOtherStage(groupVertex, true))) {
-					this.entryVertices.add(groupVertex);
-				}
-			} else {
-				if ((groupVertex.getNumberOfForwardLinks() == 0)
-					|| ((this.stage >= 0) && allConnectionsFromOtherStage(groupVertex, false))) {
-					this.entryVertices.add(groupVertex);
-				}
-			}
-		}
-	}
-
-	/**
-	 * Checks if for the given group vertex all incoming (if forward is <code>true</code>) or outgoing links (if forward
-	 * is <code>false</code>) come from
-	 * other stages than the one the given vertex is in.
-	 * 
-	 * @param groupVertex
-	 *        the group vertex to check for
-	 * @param forward
-	 *        <code>true</code> if incoming links should be considered, <code>false</code> for outgoing links
-	 * @return <code>true</code> if all incoming or outgoing links (depends on the forward switch) come from other
-	 *         stages, <code>false</code> otherwise
-	 */
-	private boolean allConnectionsFromOtherStage(final ExecutionGroupVertex groupVertex, final boolean forward) {
-
-		if (forward) {
-			for (int i = 0; i < groupVertex.getNumberOfBackwardLinks(); i++) {
-				if (this.stage == groupVertex.getBackwardEdge(i).getSourceVertex().getStageNumber()) {
-					return false;
-				}
-			}
-		} else {
-			for (int i = 0; i < groupVertex.getNumberOfForwardLinks(); i++) {
-				if (this.stage == groupVertex.getForwardEdge(i).getTargetVertex().getStageNumber()) {
-					return false;
-				}
-			}
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public boolean hasNext() {
-
-		if (this.traversalStack.isEmpty()) {
-
-			++this.numVisitedEntryVertices;
-			if (this.entryVertices.size() <= this.numVisitedEntryVertices) {
-				return false;
-			}
-		}
-
-		return true;
-
-	}
-
-
-	@Override
-	public ExecutionGroupVertex next() {
-
-		if (this.traversalStack.isEmpty()) {
-
-			final TraversalEntry newentry = new TraversalEntry(this.entryVertices.get(this.numVisitedEntryVertices), 0);
-			this.traversalStack.push(newentry);
-			this.alreadyVisited.add(newentry.getGroupVertex());
-		}
-
-		final ExecutionGroupVertex returnVertex = this.traversalStack.peek().getGroupVertex();
-
-		// Propose vertex to be visited next
-		do {
-
-			final TraversalEntry te = this.traversalStack.peek();
-
-			// Check if we can traverse deeper into the graph
-			final ExecutionGroupVertex candidateVertex = getCandidateVertex(te, forward);
-			if (candidateVertex == null) {
-				// Pop it from the stack
-				traversalStack.pop();
-			} else {
-				// Create new entry and put it on the stack
-				final TraversalEntry newte = new TraversalEntry(candidateVertex, 0);
-				this.traversalStack.push(newte);
-				this.alreadyVisited.add(newte.getGroupVertex());
-				break;
-			}
-
-		} while (!this.traversalStack.isEmpty());
-
-		return returnVertex;
-
-	}
-
-	/**
-	 * Returns a candidate group vertex which could potentially be visited next because it is reachable from the
-	 * currently considered group vertex.
-	 * 
-	 * @param te
-	 *        the traversal entry for the current source group vertex
-	 * @param forward
-	 *        <code>true</code> if the graph should be traversed in correct order, <code>false</code> to traverse it in
-	 *        reverse order
-	 * @return a candidate group vertex which could potentially be visited next
-	 */
-	private ExecutionGroupVertex getCandidateVertex(final TraversalEntry te, final boolean forward) {
-
-		while (true) {
-
-			if (forward) {
-				// No more outgoing links to consider
-				if (te.getCurrentLink() >= te.getGroupVertex().getNumberOfForwardLinks()) {
-					break;
-				}
-			} else {
-				// No more outgoing links to consider
-				if (te.getCurrentLink() >= te.getGroupVertex().getNumberOfBackwardLinks()) {
-					break;
-				}
-			}
-
-			ExecutionGroupVertex tmp = null;
-			if (forward) {
-				tmp = te.getGroupVertex().getForwardEdge(te.getCurrentLink()).getTargetVertex();
-			} else {
-				tmp = te.getGroupVertex().getBackwardEdge(te.getCurrentLink()).getSourceVertex();
-			}
-
-			// Increase the current link index by one
-			te.increaseCurrentLink();
-
-			// If stage >= 0, tmp must be in the same stage as te.getGroupVertex()
-			if (this.stage >= 0) {
-				if (tmp.getStageNumber() != this.stage) {
-					continue;
-				}
-			}
-
-			if (!this.alreadyVisited.contains(tmp)) {
-				return tmp;
-			}
-		}
-
-		return null;
-	}
-
-
-	@Override
-	public void remove() {
-		// According to the documentation this method is optional and does not need to be implemented
-
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
new file mode 100644
index 0000000..72f03b4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java
@@ -0,0 +1,220 @@
+/**
+ * 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 java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.core.io.InputSplitSource;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSet;
+import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
+import org.apache.flink.runtime.jobgraph.JobEdge;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+
+
+public class ExecutionJobVertex {
+	
+	/** Use the same log for all ExecutionGraph classes */
+	private static final Log LOG = ExecutionGraph.LOG;
+	
+	
+	private final ExecutionGraph graph;
+	
+	private final AbstractJobVertex jobVertex;
+	
+	private final ExecutionVertex2[] taskVertices;
+
+	private final IntermediateResult[] producedDataSets;
+	
+	private final InputSplitAssigner splitAssigner;
+	
+	private final int parallelism;
+	
+	
+	private final AtomicInteger numRunningTasks = new AtomicInteger(0);
+	
+	private final AtomicInteger numFinishedTasks = new AtomicInteger(0);
+	
+	private final AtomicInteger numCancelledOrFailedTasks = new AtomicInteger(0);
+	
+	
+	private SlotSharingGroup slotSharingGroup;
+	
+	
+	public ExecutionJobVertex(ExecutionGraph graph, AbstractJobVertex jobVertex, int defaultParallelism) throws JobException {
+		if (graph == null || jobVertex == null) {
+			throw new NullPointerException();
+		}
+		
+		this.graph = graph;
+		this.jobVertex = jobVertex;
+		
+		int vertexParallelism = jobVertex.getParallelism();
+		int numTaskVertices = vertexParallelism > 0 ? vertexParallelism : defaultParallelism;
+		
+		this.parallelism = numTaskVertices;
+		this.taskVertices = new ExecutionVertex2[numTaskVertices];
+		
+		// create the intermediate results
+		this.producedDataSets = new IntermediateResult[jobVertex.getNumberOfProducedIntermediateDataSets()];
+		for (int i = 0; i < jobVertex.getProducedDataSets().size(); i++) {
+			IntermediateDataSet set = jobVertex.getProducedDataSets().get(i);
+			this.producedDataSets[i] = new IntermediateResult(set.getId(), this, numTaskVertices);
+		}
+		
+		// create all task vertices
+		for (int i = 0; i < numTaskVertices; i++) {
+			ExecutionVertex2 vertex = new ExecutionVertex2(this, i, this.producedDataSets);
+			this.taskVertices[i] = vertex;
+		}
+		
+		// take the sharing group
+		this.slotSharingGroup = jobVertex.getSlotSharingGroup();
+		
+		// set up the input splits, if the vertex has any
+		try {
+			@SuppressWarnings("unchecked")
+			InputSplitSource<InputSplit> splitSource = (InputSplitSource<InputSplit>) jobVertex.getInputSplitSource();
+			if (splitSource != null) {
+				InputSplit[] splits = splitSource.createInputSplits(numTaskVertices);
+				this.splitAssigner = splitSource.getInputSplitAssigner(splits);
+			} else {
+				this.splitAssigner = null;
+			}
+		}
+		catch (Throwable t) {
+			throw new JobException("Creating the input splits caused an error: " + t.getMessage(), t);
+		}
+	}
+
+	public ExecutionGraph getGraph() {
+		return this.graph;
+	}
+	
+	public AbstractJobVertex getJobVertex() {
+		return this.jobVertex;
+	}
+
+	public int getParallelism() {
+		return this.parallelism;
+	}
+	
+	public JobID getJobId() {
+		return this.graph.getJobID();
+	}
+	
+	public JobVertexID getJobVertexId() {
+		return this.jobVertex.getID();
+	}
+	
+	public ExecutionVertex2[] getTaskVertices() {
+		return taskVertices;
+	}
+	
+	public IntermediateResult[] getProducedDataSets() {
+		return producedDataSets;
+	}
+	
+	public InputSplitAssigner getSplitAssigner() {
+		return splitAssigner;
+	}
+	
+	public void setSlotSharingGroup(SlotSharingGroup slotSharingGroup) {
+		this.slotSharingGroup = slotSharingGroup;
+	}
+	
+	public SlotSharingGroup getSlotSharingGroup() {
+		return slotSharingGroup;
+	}
+	
+	//---------------------------------------------------------------------------------------------
+	
+	public void connectToPredecessors(Map<IntermediateDataSetID, IntermediateResult> intermediateDataSets) throws JobException {
+		
+		List<JobEdge> inputs = jobVertex.getInputs();
+		
+		if (LOG.isDebugEnabled()) {
+			LOG.debug(String.format("Connecting ExecutionJobVertex %s (%s) to %d predecessors.", jobVertex.getID(), jobVertex.getName(), inputs.size()));
+		}
+		
+		for (int num = 0; num < inputs.size(); num++) {
+			JobEdge edge = inputs.get(num);
+			
+			if (LOG.isDebugEnabled()) {
+				if (edge.getSource() == null) {
+					LOG.debug(String.format("Connecting input %d of vertex %s (%s) to intermediate result referenced via ID %s.", 
+							num, jobVertex.getID(), jobVertex.getName(), edge.getSourceId()));
+				} else {
+					LOG.debug(String.format("Connecting input %d of vertex %s (%s) to intermediate result referenced via predecessor %s (%s).",
+							num, jobVertex.getID(), jobVertex.getName(), edge.getSource().getProducer().getID(), edge.getSource().getProducer().getName()));
+				}
+			}
+			
+			// fetch the intermediate result via ID. if it does not exist, then it either has not been created, or the order
+			// in which this method is called for the job vertices is not a topological order
+			IntermediateResult ires = intermediateDataSets.get(edge.getSourceId());
+			if (ires == null) {
+				throw new JobException("Cannot connect this job graph to the previous graph. No previous intermediate result found for ID "
+						+ edge.getSourceId());
+			}
+			
+			int consumerIndex = ires.registerConsumer();
+			
+			for (int i = 0; i < parallelism; i++) {
+				ExecutionVertex2 ev = taskVertices[i];
+				ev.connectSource(num, ires, edge, consumerIndex);
+			}
+			
+			
+		}
+	}
+	
+	//---------------------------------------------------------------------------------------------
+	//  State, deployment, and recovery logic 
+	//---------------------------------------------------------------------------------------------
+	
+	void vertexSwitchedToRunning(int subtask) {
+		this.numRunningTasks.incrementAndGet();
+	}
+	
+	void vertexFailed(int subtask) {
+		
+	}
+	
+	void vertexCancelled(int subtask) {
+		
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Miscellaneous
+	// --------------------------------------------------------------------------------------------
+	
+	public void execute(Runnable action) {
+		this.graph.execute(action);
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionPipeline.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionPipeline.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionPipeline.java
deleted file mode 100644
index d998f60..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionPipeline.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * 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 java.util.Iterator;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.instance.AllocatedResource;
-import org.apache.flink.runtime.util.UnmodifiableIterator;
-
-/**
- * An execution pipeline is a scheduling abstraction which defines a set of {@link ExecutionVertex} objects which must
- * be deployed together. An {@link ExecutionVertex} always belongs to exactly one execution pipeline.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class ExecutionPipeline {
-
-	/**
-	 * The set of vertices belonging to this execution pipeline.
-	 */
-	private final CopyOnWriteArrayList<ExecutionVertex> vertices = new CopyOnWriteArrayList<ExecutionVertex>();
-
-	/**
-	 * Adds the given {@link ExecutionVertex} to this pipeline.
-	 * 
-	 * @param vertex
-	 *        the vertex to be added to this pipeline
-	 */
-	void addToPipeline(final ExecutionVertex vertex) {
-
-		if (!this.vertices.addIfAbsent(vertex)) {
-			throw new IllegalStateException("Vertex " + vertex + " has already been added to pipeline " + this);
-		}
-	}
-
-	/**
-	 * Removes the given {@link ExecutionVertex} from this pipeline.
-	 * 
-	 * @param vertex
-	 *        the vertex to be removed from this pipeline.
-	 */
-	void removeFromPipeline(final ExecutionVertex vertex) {
-
-		if (!this.vertices.remove(vertex)) {
-			throw new IllegalStateException("Vertex " + vertex + " was not part of the pipeline " + this);
-		}
-	}
-
-	/**
-	 * Returns an {@link Iterator} to the vertices which are part of this pipeline.
-	 * 
-	 * @return an iterator to the vertices of this pipeline
-	 */
-	public Iterator<ExecutionVertex> iterator() {
-
-		return new UnmodifiableIterator<ExecutionVertex>(this.vertices.iterator());
-	}
-
-	/**
-	 * Checks if the pipeline is currently finishing its execution, i.e. all vertices contained in the pipeline have
-	 * switched to the <code>FINISHING</code> or <code>FINISHED</code> state.
-	 * 
-	 * @return <code>true</code> if the pipeline is currently finishing, <code>false</code> otherwise
-	 */
-	public boolean isFinishing() {
-
-		final Iterator<ExecutionVertex> it = this.vertices.iterator();
-		while (it.hasNext()) {
-
-			final ExecutionState state = it.next().getExecutionState();
-			if (state != ExecutionState.FINISHING && state != ExecutionState.FINISHED) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-	/**
-	 * Sets the given allocated resource for all vertices included in this pipeline.
-	 * 
-	 * @param resource
-	 *        the allocated resource to set for all vertices included in this pipeline
-	 */
-	public void setAllocatedResource(final AllocatedResource resource) {
-
-		final Iterator<ExecutionVertex> it = this.vertices.iterator();
-		while (it.hasNext()) {
-			final ExecutionVertex vertex = it.next();
-			vertex.setAllocatedResource(resource);
-		}
-	}
-
-	/**
-	 * Updates the execution state for all vertices included in this pipeline.
-	 * 
-	 * @param executionState
-	 *        the execution state to set for all vertices included in this pipeline
-	 */
-	public void updateExecutionState(final ExecutionState executionState) {
-
-		final Iterator<ExecutionVertex> it = this.vertices.iterator();
-		while (it.hasNext()) {
-			final ExecutionVertex vertex = it.next();
-			vertex.updateExecutionState(executionState);
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionSignature.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionSignature.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionSignature.java
deleted file mode 100644
index 40f2468..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionSignature.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/**
- * 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 java.io.IOException;
-import java.security.MessageDigest;
-import java.security.NoSuchAlgorithmException;
-import java.util.Arrays;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * An execution signature allows to uniquely identify a job vertex. The signature involves the name of the
- * class to be invoked at runtime as well as a cryptographic hash of all the JAR files which are required to
- * instantiate the class and run it. The execution is the basis for feedback learning as it enables the profiler
- * to recognize particular parts of a job. Execution signature objects are immutable and, consequently, thread-safe.
- * <p>
- * This class is thread-safe.
- * 
- */
-public final class ExecutionSignature {
-
-	/**
-	 * The log object used for debugging.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(ExecutionSignature.class);
-
-	/**
-	 * The name of the hashing algorithm to be used.
-	 */
-	private static final String HASHINGALGORITHM = "SHA-1";
-
-	/**
-	 * The message digest object used to calculate the signature.
-	 */
-	private static MessageDigest messageDigest = null;
-
-	/**
-	 * The buffer storing the signature.
-	 */
-	private final byte[] signature;
-
-	/**
-	 * Constructs a new execution signature object and passes the signature buffer.
-	 * 
-	 * @param signature
-	 *        the byte buffer containing the signature.
-	 */
-	private ExecutionSignature(final byte[] signature) {
-		this.signature = signature;
-	}
-
-	/**
-	 * Calculates the execution signature from the given class name and job ID.
-	 * 
-	 * @param invokableClass
-	 *        the name of the class to contain the task program
-	 * @param jobID
-	 *        the ID of the job
-	 * @return the cryptographic signature of this vertex
-	 */
-	public static synchronized ExecutionSignature createSignature(
-			final Class<? extends AbstractInvokable> invokableClass, final JobID jobID) {
-
-		// First, try to load message digest algorithm, if necessary
-		if (messageDigest == null) {
-			try {
-				messageDigest = MessageDigest.getInstance(HASHINGALGORITHM);
-			} catch (NoSuchAlgorithmException e) {
-				LOG.error("Unable to load message digest algorithm " + HASHINGALGORITHM);
-				return null;
-			}
-		}
-
-		// Reset digest buffer and add the name of the invokable class to the message digest buffer
-		messageDigest.reset();
-		messageDigest.update(invokableClass.getName().getBytes());
-
-		String[] requiredJarFiles;
-		// Next, retrieve the JAR-files associated with this job
-		try {
-			requiredJarFiles = LibraryCacheManager.getRequiredJarFiles(jobID);
-		} catch (IOException ioe) {
-			// Output an error message and return
-			LOG.error("Cannot access library cache manager for job ID " + jobID);
-			return null;
-		}
-
-		// Now, sort the list of JAR-files in order to always calculate the signature in the same manner
-		Arrays.sort(requiredJarFiles);
-
-		// Finally, add the names of the JAR-files to the hash calculation
-		for (int i = 0; i < requiredJarFiles.length; i++) {
-			messageDigest.update(requiredJarFiles[i].getBytes());
-		}
-
-		return new ExecutionSignature(messageDigest.digest());
-	}
-
-
-	@Override
-	public boolean equals(final Object obj) {
-
-		if (obj instanceof ExecutionSignature) {
-
-			final ExecutionSignature executionSignature = (ExecutionSignature) obj;
-			return Arrays.equals(this.signature, executionSignature.signature);
-		}
-
-		return false;
-	}
-
-
-	@Override
-	public int hashCode() {
-
-		int hashCode = 0;
-
-		for (int i = 0; i < this.signature.length; i++) {
-			hashCode += this.signature[i];
-		}
-
-		return hashCode;
-	}
-
-
-	@Override
-	public String toString() {
-
-		final StringBuffer stringBuffer = new StringBuffer();
-		for (int i = 0; i < this.signature.length; i++) {
-			stringBuffer.append(Integer.toHexString(0xFF & this.signature[i]));
-		}
-
-		return stringBuffer.toString();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStage.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStage.java
deleted file mode 100644
index 54e7b12..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStage.java
+++ /dev/null
@@ -1,411 +0,0 @@
-/**
- * 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 java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-
-/**
- * An execution stage contains all execution group vertices (and as a result all execution vertices) which
- * must run at the same time. The execution of a job progresses in terms of stages, i.e. the next stage of a
- * job can only start to execute if the execution of its preceding stage is complete.
- * <p>
- * This class is thread-safe.
- */
-public final class ExecutionStage {
-
-	/**
-	 * The execution graph that this stage belongs to.
-	 */
-	private final ExecutionGraph executionGraph;
-
-	/**
-	 * List of group vertices which are assigned to this stage.
-	 */
-	private final CopyOnWriteArrayList<ExecutionGroupVertex> stageMembers = new CopyOnWriteArrayList<ExecutionGroupVertex>();
-
-	/**
-	 * Number of the stage.
-	 */
-	private volatile int stageNum = -1;
-
-	/**
-	 * Constructs a new execution stage and assigns the given stage number to it.
-	 * 
-	 * @param executionGraph
-	 *        the executionGraph that this stage belongs to
-	 * @param stageNum
-	 *        the number of this execution stage
-	 */
-	public ExecutionStage(final ExecutionGraph executionGraph, final int stageNum) {
-		this.executionGraph = executionGraph;
-		this.stageNum = stageNum;
-	}
-
-	/**
-	 * Sets the number of this execution stage.
-	 * 
-	 * @param stageNum
-	 *        the new number of this execution stage
-	 */
-	public void setStageNumber(final int stageNum) {
-		this.stageNum = stageNum;
-	}
-
-	/**
-	 * Returns the number of this execution stage.
-	 * 
-	 * @return the number of this execution stage
-	 */
-	public int getStageNumber() {
-
-		return this.stageNum;
-	}
-
-	/**
-	 * Adds a new execution group vertex to this stage if it is not already included.
-	 * 
-	 * @param groupVertex
-	 *        the new execution group vertex to include
-	 */
-	public void addStageMember(final ExecutionGroupVertex groupVertex) {
-
-		if (this.stageMembers.addIfAbsent(groupVertex)) {
-			groupVertex.setExecutionStage(this);
-		}
-	}
-
-	/**
-	 * Removes the specified group vertex from the execution stage.
-	 * 
-	 * @param groupVertex
-	 *        the group vertex to remove from the stage
-	 */
-	public void removeStageMember(ExecutionGroupVertex groupVertex) {
-
-		this.stageMembers.remove(groupVertex);
-	}
-
-	/**
-	 * Returns the number of group vertices this execution stage includes.
-	 * 
-	 * @return the number of group vertices this execution stage includes
-	 */
-	public int getNumberOfStageMembers() {
-
-		return this.stageMembers.size();
-	}
-
-	/**
-	 * Returns the stage member internally stored at index <code>index</code>.
-	 * 
-	 * @param index
-	 *        the index of the group vertex to return
-	 * @return the stage member internally stored at the specified index or <code>null</code> if no group vertex exists
-	 *         with such an index
-	 */
-	public ExecutionGroupVertex getStageMember(final int index) {
-
-		try {
-			return this.stageMembers.get(index);
-		} catch (ArrayIndexOutOfBoundsException e) {
-			return null;
-		}
-	}
-
-	/**
-	 * Returns the number of input execution vertices in this stage, i.e. the number
-	 * of execution vertices which are connected to vertices in a lower stage
-	 * or have no input channels.
-	 * 
-	 * @return the number of input vertices in this stage
-	 */
-	public int getNumberOfInputExecutionVertices() {
-
-		int retVal = 0;
-
-		final Iterator<ExecutionGroupVertex> it = this.stageMembers.iterator();
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			if (groupVertex.isInputVertex()) {
-				retVal += groupVertex.getCurrentNumberOfGroupMembers();
-			}
-		}
-
-		return retVal;
-	}
-
-	/**
-	 * Returns the number of output execution vertices in this stage, i.e. the number
-	 * of execution vertices which are connected to vertices in a higher stage
-	 * or have no output channels.
-	 * 
-	 * @return the number of output vertices in this stage
-	 */
-	public int getNumberOfOutputExecutionVertices() {
-
-		int retVal = 0;
-
-		final Iterator<ExecutionGroupVertex> it = this.stageMembers.iterator();
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			if (groupVertex.isOutputVertex()) {
-				retVal += groupVertex.getCurrentNumberOfGroupMembers();
-			}
-		}
-
-		return retVal;
-	}
-
-	/**
-	 * Returns the output execution vertex with the given index or <code>null</code> if no such vertex exists.
-	 * 
-	 * @param index
-	 *        the index of the vertex to be selected.
-	 * @return the output execution vertex with the given index or <code>null</code> if no such vertex exists
-	 */
-	public ExecutionVertex getInputExecutionVertex(int index) {
-
-		final Iterator<ExecutionGroupVertex> it = this.stageMembers.iterator();
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			if (groupVertex.isInputVertex()) {
-				final int numberOfMembers = groupVertex.getCurrentNumberOfGroupMembers();
-				if (index >= numberOfMembers) {
-					index -= numberOfMembers;
-				} else {
-					return groupVertex.getGroupMember(index);
-				}
-			}
-		}
-
-		return null;
-	}
-
-	/**
-	 * Returns the input execution vertex with the given index or <code>null</code> if no such vertex exists.
-	 * 
-	 * @param index
-	 *        the index of the vertex to be selected.
-	 * @return the input execution vertex with the given index or <code>null</code> if no such vertex exists
-	 */
-	public ExecutionVertex getOutputExecutionVertex(int index) {
-
-		final Iterator<ExecutionGroupVertex> it = this.stageMembers.iterator();
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-			if (groupVertex.isOutputVertex()) {
-				final int numberOfMembers = groupVertex.getCurrentNumberOfGroupMembers();
-				if (index >= numberOfMembers) {
-					index -= numberOfMembers;
-				} else {
-					return groupVertex.getGroupMember(index);
-				}
-			}
-		}
-
-		return null;
-	}
-
-	/**
-	 * Returns the execution graph that this stage belongs to.
-	 * 
-	 * @return the execution graph that this stage belongs to
-	 */
-	public ExecutionGraph getExecutionGraph() {
-
-		return this.executionGraph;
-	}
-
-	/**
-	 * Reconstructs the execution pipelines for this execution stage.
-	 */
-	void reconstructExecutionPipelines() {
-
-		Iterator<ExecutionGroupVertex> it = this.stageMembers.iterator();
-		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-
-			// We only look at input vertices first
-			if (!groupVertex.isInputVertex()) {
-				continue;
-			}
-
-			final Iterator<ExecutionVertex> vertexIt = groupVertex.iterator();
-			while (vertexIt.hasNext()) {
-
-				final ExecutionVertex vertex = vertexIt.next();
-				reconstructExecutionPipeline(vertex, true, alreadyVisited);
-			}
-		}
-
-		it = this.stageMembers.iterator();
-		alreadyVisited.clear();
-
-		while (it.hasNext()) {
-
-			final ExecutionGroupVertex groupVertex = it.next();
-
-			// We only look at input vertices first
-			if (!groupVertex.isOutputVertex()) {
-				continue;
-			}
-
-			final Iterator<ExecutionVertex> vertexIt = groupVertex.iterator();
-			while (vertexIt.hasNext()) {
-
-				final ExecutionVertex vertex = vertexIt.next();
-				reconstructExecutionPipeline(vertex, false, alreadyVisited);
-			}
-		}
-	}
-
-	/**
-	 * Reconstructs the execution pipeline starting at the given vertex by conducting a depth-first search.
-	 * 
-	 * @param vertex
-	 *        the vertex to start the depth-first search from
-	 * @param forward
-	 *        <code>true</code> to traverse the graph according to the original direction of the edges or
-	 *        <code>false</code> for the opposite direction
-	 * @param alreadyVisited
-	 *        a set of vertices that have already been visited in the depth-first search
-	 */
-	private void reconstructExecutionPipeline(final ExecutionVertex vertex, final boolean forward,
-			final Set<ExecutionVertex> alreadyVisited) {
-
-		ExecutionPipeline pipeline = vertex.getExecutionPipeline();
-		if (pipeline == null) {
-			pipeline = new ExecutionPipeline();
-			vertex.setExecutionPipeline(pipeline);
-		}
-
-		alreadyVisited.add(vertex);
-
-		if (forward) {
-
-			final int numberOfOutputGates = vertex.getNumberOfOutputGates();
-			for (int i = 0; i < numberOfOutputGates; ++i) {
-
-				final ExecutionGate outputGate = vertex.getOutputGate(i);
-				final ChannelType channelType = outputGate.getChannelType();
-				final int numberOfOutputChannels = outputGate.getNumberOfEdges();
-				for (int j = 0; j < numberOfOutputChannels; ++j) {
-
-					final ExecutionEdge outputChannel = outputGate.getEdge(j);
-					final ExecutionVertex connectedVertex = outputChannel.getInputGate().getVertex();
-
-					boolean recurse = false;
-
-					if (!alreadyVisited.contains(connectedVertex)) {
-						recurse = true;
-					}
-
-					if (channelType == ChannelType.IN_MEMORY
-						&& !pipeline.equals(connectedVertex.getExecutionPipeline())) {
-
-						connectedVertex.setExecutionPipeline(pipeline);
-						recurse = true;
-					}
-
-					if (recurse) {
-						reconstructExecutionPipeline(connectedVertex, true, alreadyVisited);
-					}
-				}
-			}
-		} else {
-
-			final int numberOfInputGates = vertex.getNumberOfInputGates();
-			for (int i = 0; i < numberOfInputGates; ++i) {
-
-				final ExecutionGate inputGate = vertex.getInputGate(i);
-				final ChannelType channelType = inputGate.getChannelType();
-				final int numberOfInputChannels = inputGate.getNumberOfEdges();
-				for (int j = 0; j < numberOfInputChannels; ++j) {
-
-					final ExecutionEdge inputChannel = inputGate.getEdge(j);
-					final ExecutionVertex connectedVertex = inputChannel.getOutputGate().getVertex();
-
-					boolean recurse = false;
-
-					if (!alreadyVisited.contains(connectedVertex)) {
-						recurse = true;
-					}
-
-					if (channelType == ChannelType.IN_MEMORY
-						&& !pipeline.equals(connectedVertex.getExecutionPipeline())) {
-
-						connectedVertex.setExecutionPipeline(pipeline);
-						recurse = true;
-					}
-
-					if (recurse) {
-						reconstructExecutionPipeline(connectedVertex, false, alreadyVisited);
-					}
-				}
-			}
-		}
-	}
-
-	public int getMaxNumberSubtasks(){
-		int maxDegree = 0;
-
-		for(int i =0; i < this.getNumberOfStageMembers(); i++){
-			final ExecutionGroupVertex groupVertex = this.getStageMember(i);
-
-			if(groupVertex.getCurrentNumberOfGroupMembers() > maxDegree){
-				maxDegree = groupVertex.getCurrentNumberOfGroupMembers();
-			}
-		}
-
-		return maxDegree;
-	}
-
-	public int getRequiredSlots(){
-		Set<Instance> instanceSet = new HashSet<Instance>();
-
-		for(int i=0; i< this.getNumberOfStageMembers(); i++){
-			final ExecutionGroupVertex groupVertex = this.getStageMember(i);
-
-			final Iterator<ExecutionVertex> vertexIterator = groupVertex.iterator();
-
-			while(vertexIterator.hasNext()){
-				final ExecutionVertex vertex = vertexIterator.next();
-
-				instanceSet.add(vertex.getAllocatedResource().getInstance());
-			}
-
-		}
-
-		return instanceSet.size();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStageListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStageListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStageListener.java
deleted file mode 100644
index 16c5478..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionStageListener.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * 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 org.apache.flink.runtime.jobgraph.JobID;
-
-/**
- * This interface must be implemented in order to receive a notification whenever the job has entered a new execution
- * stage.
- * 
- */
-public interface ExecutionStageListener {
-
-	/**
-	 * This method is called to indicate that the job with the given {@link JobID} has finished its previous
-	 * {@link ExecutionStage} and has entered the next {@link ExecutionStage}. Note that a notification is not sent when
-	 * the job has entered its initial execution stage.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the notification belongs to
-	 * @param executionStage
-	 *        the next execution stage that has just been entered
-	 */
-	void nextExecutionStageEntered(JobID jobID, ExecutionStage executionStage);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java
new file mode 100644
index 0000000..31108a4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionState.java
@@ -0,0 +1,34 @@
+/**
+ * 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;
+
+public enum ExecutionState {
+
+	/** The job is scheduled, but nothing is running yet */
+	CREATED,
+	
+	/** At least some tasks of the job are running */
+	RUNNING,
+	
+	/** The execution has finished, no further tasks may be added to the job */
+	FINISHED,
+	
+	/** The execution has failed */
+	FAILED
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
index d1ee262..adc0f09 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java
@@ -41,11 +41,11 @@ import org.apache.flink.runtime.execution.ExecutionStateTransition;
 import org.apache.flink.runtime.instance.AllocatedResource;
 import org.apache.flink.runtime.instance.AllocationID;
 import org.apache.flink.runtime.io.network.gates.GateID;
-import org.apache.flink.runtime.taskmanager.AbstractTaskResult;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
 import org.apache.flink.runtime.taskmanager.TaskCancelResult;
 import org.apache.flink.runtime.taskmanager.TaskKillResult;
 import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
-import org.apache.flink.runtime.taskmanager.AbstractTaskResult.ReturnCode;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult.ReturnCode;
 import org.apache.flink.runtime.util.AtomicEnum;
 import org.apache.flink.runtime.util.SerializableArrayList;
 import org.apache.flink.util.StringUtils;
@@ -432,8 +432,8 @@ public final class ExecutionVertex {
 
 		if (this.cancelRequested.compareAndSet(true, false)) {
 			final TaskCancelResult tsr = cancelTask();
-			if (tsr.getReturnCode() != AbstractTaskResult.ReturnCode.SUCCESS
-				&& tsr.getReturnCode() != AbstractTaskResult.ReturnCode.TASK_NOT_FOUND) {
+			if (tsr.getReturnCode() != AbstractTaskResult.TaskOperationResult.SUCCESS
+				&& tsr.getReturnCode() != AbstractTaskResult.TaskOperationResult.TASK_NOT_FOUND) {
 				LOG.error("Unable to cancel vertex " + this + ": " + tsr.getReturnCode().toString()
 					+ ((tsr.getDescription() != null) ? (" (" + tsr.getDescription() + ")") : ""));
 			}
@@ -675,7 +675,7 @@ public final class ExecutionVertex {
 
 		if (ar == null) {
 			final TaskSubmissionResult result = new TaskSubmissionResult(getID(),
-				AbstractTaskResult.ReturnCode.NO_INSTANCE);
+				AbstractTaskResult.TaskOperationResult.NO_INSTANCE);
 			result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
 			return result;
 		}
@@ -690,7 +690,7 @@ public final class ExecutionVertex {
 
 		} catch (IOException e) {
 			final TaskSubmissionResult result = new TaskSubmissionResult(getID(),
-				AbstractTaskResult.ReturnCode.IPC_ERROR);
+				AbstractTaskResult.TaskOperationResult.IPC_ERROR);
 			result.setDescription(StringUtils.stringifyException(e));
 			return result;
 		}
@@ -709,7 +709,7 @@ public final class ExecutionVertex {
 		final ExecutionState state = this.executionState.get();
 
 		if (state != ExecutionState.RUNNING) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.ILLEGAL_STATE);
+			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.TaskOperationResult.ILLEGAL_STATE);
 			result.setDescription("Vertex " + this.toString() + " is in state " + state);
 			return result;
 		}
@@ -717,7 +717,7 @@ public final class ExecutionVertex {
 		final AllocatedResource ar = this.allocatedResource.get();
 
 		if (ar == null) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.NO_INSTANCE);
+			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.TaskOperationResult.NO_INSTANCE);
 			result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
 			return result;
 		}
@@ -725,7 +725,7 @@ public final class ExecutionVertex {
 		try {
 			return ar.getInstance().killTask(this.vertexID);
 		} catch (IOException e) {
-			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.ReturnCode.IPC_ERROR);
+			final TaskKillResult result = new TaskKillResult(getID(), AbstractTaskResult.TaskOperationResult.IPC_ERROR);
 			result.setDescription(StringUtils.stringifyException(e));
 			return result;
 		}
@@ -746,15 +746,15 @@ public final class ExecutionVertex {
 			final ExecutionState previousState = this.executionState.get();
 
 			if (previousState == ExecutionState.CANCELED) {
-				return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS);
+				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
 			}
 
 			if (previousState == ExecutionState.FAILED) {
-				return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS);
+				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
 			}
 
 			if (previousState == ExecutionState.FINISHED) {
-				return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS);
+				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
 			}
 
 			// The vertex has already received a cancel request
@@ -774,7 +774,7 @@ public final class ExecutionVertex {
 					continue;
 				}
 
-				return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS);
+				return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
 			}
 
 			// Check if we had a race. If state change is accepted, send cancel request
@@ -783,20 +783,20 @@ public final class ExecutionVertex {
 				if (this.groupVertex.getStageNumber() != this.executionGraph.getIndexOfCurrentExecutionStage()) {
 					// Set to canceled directly
 					updateExecutionState(ExecutionState.CANCELED, null);
-					return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS);
+					return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
 				}
 
 				if (previousState != ExecutionState.RUNNING && previousState != ExecutionState.FINISHING) {
 					// Set to canceled directly
 					updateExecutionState(ExecutionState.CANCELED, null);
-					return new TaskCancelResult(getID(), AbstractTaskResult.ReturnCode.SUCCESS);
+					return new TaskCancelResult(getID(), AbstractTaskResult.TaskOperationResult.SUCCESS);
 				}
 
 				final AllocatedResource ar = this.allocatedResource.get();
 
 				if (ar == null) {
 					final TaskCancelResult result = new TaskCancelResult(getID(),
-						AbstractTaskResult.ReturnCode.NO_INSTANCE);
+						AbstractTaskResult.TaskOperationResult.NO_INSTANCE);
 					result.setDescription("Assigned instance of vertex " + this.toString() + " is null!");
 					return result;
 				}
@@ -806,7 +806,7 @@ public final class ExecutionVertex {
 
 				} catch (IOException e) {
 					final TaskCancelResult result = new TaskCancelResult(getID(),
-						AbstractTaskResult.ReturnCode.IPC_ERROR);
+						AbstractTaskResult.TaskOperationResult.IPC_ERROR);
 					result.setDescription(StringUtils.stringifyException(e));
 					return result;
 				}


[46/63] [abbrv] git commit: Adjust test logging for new execution graph tests to logback framework

Posted by se...@apache.org.
Adjust test logging for new execution graph tests to logback framework


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/22d8bf8e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/22d8bf8e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/22d8bf8e

Branch: refs/heads/master
Commit: 22d8bf8e42c2aff164c3b1ea84280c63652bb1b2
Parents: 09d1c33
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Sep 12 18:00:08 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 flink-runtime/src/test/resources/logback-test.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/22d8bf8e/flink-runtime/src/test/resources/logback-test.xml
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/resources/logback-test.xml b/flink-runtime/src/test/resources/logback-test.xml
index cb2f164..7fb3387 100644
--- a/flink-runtime/src/test/resources/logback-test.xml
+++ b/flink-runtime/src/test/resources/logback-test.xml
@@ -27,10 +27,14 @@
         <appender-ref ref="STDOUT"/>
     </root>
 
+    <!-- The following loggers are disabled during tests, because many tests deliberately
+         throw error to test failing scenarios. Logging those would overflow the log. -->
+         
     <logger name="org.apache.flink.runtime.operators.DataSinkTask" level="OFF"/>
     <logger name="org.apache.flink.runtime.operators.RegularPactTask" level="OFF"/>
     <logger name="org.apache.flink.runtime.client.JobClient" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.Task" level="OFF"/>
     <logger name="org.apache.flink.runtime.jobmanager.JobManager" level="OFF"/>
     <logger name="org.apache.flink.runtime.taskmanager.TaskManager" level="OFF"/>
+    <logger name="org.apache.flink.runtime.executiongraph.ExecutionGraph" level="OFF"/>
 </configuration>
\ No newline at end of file


[04/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
new file mode 100644
index 0000000..299f7ba
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
@@ -0,0 +1,466 @@
+/**
+ * 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 static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.times;
+
+import java.io.IOException;
+import java.util.NoSuchElementException;
+
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
+import org.apache.flink.runtime.protocols.TaskOperationProtocol;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
+import org.apache.flink.util.LogUtils;
+import org.apache.log4j.Level;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+public class ExecutionVertexCancelTest {
+
+	// --------------------------------------------------------------------------------------------
+	//  Canceling in different states
+	// --------------------------------------------------------------------------------------------
+	
+	@Test
+	public void testCancelFromCreated() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCancelFromScheduled() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			setVertexState(vertex, ExecutionState2.SCHEDULED);
+			assertEquals(ExecutionState2.SCHEDULED, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCancelConcurrentlyToDeploying_CallsNotOvertaking() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ActionQueue actions = new ActionQueue();
+			
+			final ExecutionJobVertex ejv = getJobVertexExecutingTriggered(jid, actions);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			setVertexState(vertex, ExecutionState2.SCHEDULED);
+			assertEquals(ExecutionState2.SCHEDULED, vertex.getExecutionState());
+			
+			// task manager mock
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
+			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, true), new TaskOperationResult(jid, 0, false));
+			
+			Instance instance = getInstance(taskManager);
+			AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			vertex.deployToSlot(slot);
+			
+			assertEquals(ExecutionState2.DEPLOYING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			 
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			
+			verify(taskManager, times(0)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+			verify(taskManager, times(0)).cancelTask(jid, 0);
+
+			// first action happens (deploy)
+			actions.triggerNextAction();
+			verify(taskManager, times(1)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+			
+			// the deploy call found itself in canceling after it returned and needs to send a cancel call
+			// the call did not yet execute, so it is still in canceling
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			
+			// second action happens (cancel call from cancel function)
+			actions.triggerNextAction();
+			
+			// should properly set state to cancelled
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			
+			// trigger the correction canceling call
+			actions.triggerNextAction();
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			
+			verify(taskManager, times(2)).cancelTask(jid, 0);
+			
+			assertTrue(slot.isReleased());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCancelConcurrentlyToDeploying_CallsOvertaking() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ActionQueue actions = new ActionQueue();
+			
+			final ExecutionJobVertex ejv = getJobVertexExecutingTriggered(jid, actions);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			setVertexState(vertex, ExecutionState2.SCHEDULED);
+			assertEquals(ExecutionState2.SCHEDULED, vertex.getExecutionState());
+			
+			// task manager mock
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
+			
+			// first return NOT SUCCESS (task not found, cancel call overtook deploy call), then success (cancel call after deploy call)
+			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, false), new TaskOperationResult(jid, 0, true));
+			
+			Instance instance = getInstance(taskManager);
+			AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			vertex.deployToSlot(slot);
+			
+			assertEquals(ExecutionState2.DEPLOYING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			 
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			
+			verify(taskManager, times(0)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+			verify(taskManager, times(0)).cancelTask(jid, 0);
+
+			// first action happens (deploy)
+			Runnable deployAction = actions.popNextAction();
+			Runnable cancelAction = actions.popNextAction();
+			
+			// cancel call first
+			cancelAction.run();
+			
+			// did not find the task, not properly cancelled, stay in canceling
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			
+			// deploy action next
+			deployAction.run();
+			
+			verify(taskManager, times(1)).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+			
+			// the deploy call found itself in canceling after it returned and needs to send a cancel call
+			// the call did not yet execute, so it is still in canceling
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			
+			// trigger the correcting cancel call, should properly set state to cancelled
+			actions.triggerNextAction();
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			
+			verify(taskManager, times(2)).cancelTask(jid, 0);
+			
+			assertTrue(slot.isReleased());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCancelFromRunning() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, true));
+			
+			Instance instance = getInstance(taskManager);
+			AllocatedSlot slot = instance.allocateSlot(new JobID());
+
+			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexResource(vertex, slot);
+			
+			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			
+			verify(taskManager).cancelTask(jid, 0);
+			
+			assertTrue(slot.isReleased());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testRepeatedCancelFromRunning() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ActionQueue actions = new ActionQueue();
+			final ExecutionJobVertex ejv = getJobVertexExecutingTriggered(jid, actions);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, true));
+			
+			Instance instance = getInstance(taskManager);
+			AllocatedSlot slot = instance.allocateSlot(new JobID());
+
+			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexResource(vertex, slot);
+			
+			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+			
+			actions.triggerNextAction();
+			
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			
+			try {
+				actions.triggerNextAction();
+				fail("Too many calls sent.");
+			} catch (NoSuchElementException e) {}
+			
+			assertTrue(slot.isReleased());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCancelFromRunningDidNotFindTask() {
+		// this may happen when the task finished or failed while the call was in progress
+		
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.cancelTask(jid, 0)).thenReturn(new TaskOperationResult(jid, 0, false));
+			
+			Instance instance = getInstance(taskManager);
+			AllocatedSlot slot = instance.allocateSlot(new JobID());
+
+			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexResource(vertex, slot);
+			
+			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState2.CANCELING, vertex.getExecutionState());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCancelCallFails() {
+		// this may happen when the task finished or failed while the call was in progress
+		LogUtils.initializeDefaultConsoleLogger(Level.OFF);
+		
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+
+			final TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.cancelTask(jid, 0)).thenThrow(new IOException("RPC call failed"));
+			
+			Instance instance = getInstance(taskManager);
+			AllocatedSlot slot = instance.allocateSlot(new JobID());
+
+			setVertexState(vertex, ExecutionState2.RUNNING);
+			setVertexResource(vertex, slot);
+			
+			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			vertex.cancel();
+			
+			assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+			
+			assertTrue(slot.isReleased());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Actions after a vertex has been canceled or while canceling
+	// --------------------------------------------------------------------------------------------
+	
+	@Test
+	public void testScheduleOrDeployAfterCancel() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			setVertexState(vertex, ExecutionState2.CANCELED);
+			
+			assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			
+			// 1)
+			// scheduling after being created should be tolerated (no exception) because
+			// it can occur as the result of races
+			{
+				DefaultScheduler scheduler = mock(DefaultScheduler.class);
+				vertex.scheduleForExecution(scheduler);
+				
+				assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			}
+			
+			// 2)
+			// deploying after canceling from CREATED needs to raise an exception, because
+			// the scheduler (or any caller) needs to know that the slot should be released
+			try {
+				TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+				Instance instance = getInstance(taskManager);
+				AllocatedSlot slot = instance.allocateSlot(new JobID());
+				
+				vertex.deployToSlot(slot);
+				fail("Method should throw an exception");
+			}
+			catch (IllegalStateException e) {
+				assertEquals(ExecutionState2.CANCELED, vertex.getExecutionState());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testActionsWhileCancelling() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
+			
+			// scheduling while canceling is an illegal state transition
+			try {
+				ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+				setVertexState(vertex, ExecutionState2.CANCELING);
+				
+				DefaultScheduler scheduler = mock(DefaultScheduler.class);
+				vertex.scheduleForExecution(scheduler);
+				fail("Method should throw an exception");
+			}
+			catch (IllegalStateException e) {}
+			
+			
+			// deploying while in canceling state is illegal (should immediately go to canceled)
+			try {
+				ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+				setVertexState(vertex, ExecutionState2.CANCELING);
+				
+				TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+				Instance instance = getInstance(taskManager);
+				AllocatedSlot slot = instance.allocateSlot(new JobID());
+				
+				vertex.deployToSlot(slot);
+				fail("Method should throw an exception");
+			}
+			catch (IllegalStateException e) {}
+			
+			
+			// fail while canceling
+			{
+				ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+				
+				TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+				Instance instance = getInstance(taskManager);
+				AllocatedSlot slot = instance.allocateSlot(new JobID());
+				
+				setVertexResource(vertex, slot);
+				setVertexState(vertex, ExecutionState2.CANCELING);
+				
+				Exception failureCause = new Exception("test exception");
+				
+				vertex.fail(failureCause);
+				assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+				assertEquals(failureCause, vertex.getFailureCause());
+				
+				assertTrue(slot.isReleased());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
new file mode 100644
index 0000000..5c61993
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexDeploymentTest.java
@@ -0,0 +1,219 @@
+/**
+ * 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 static org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.*;
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.protocols.TaskOperationProtocol;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+public class ExecutionVertexDeploymentTest {
+
+	@Test
+	public void testDeployCall() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexNotExecuting(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			vertex.deployToSlot(slot);
+			assertEquals(ExecutionState2.DEPLOYING, vertex.getExecutionState());
+			
+			// no repeated scheduling
+			try {
+				vertex.deployToSlot(slot);
+				fail("Scheduled from wrong state");
+			}
+			catch (IllegalStateException e) {}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testDeployWithSynchronousAnswer() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			vertex.deployToSlot(slot);
+			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			// no repeated scheduling
+			try {
+				vertex.deployToSlot(slot);
+				fail("Scheduled from wrong state");
+			}
+			catch (IllegalStateException e) {}
+			
+			verify(taskManager).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testDeployWithAsynchronousAnswer() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, true));
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexExecutingAsynchronously(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			vertex.deployToSlot(slot);
+			
+			// no repeated scheduling
+			try {
+				vertex.deployToSlot(slot);
+				fail("Scheduled from wrong state");
+			}
+			catch (IllegalStateException e) {}
+			
+			// wait until the state transition must be done
+			for (int i = 0; i < 100; i++) {
+				if (vertex.getExecutionState() != ExecutionState2.RUNNING) {
+					Thread.sleep(10);
+				}
+			}
+			
+			assertEquals(ExecutionState2.RUNNING, vertex.getExecutionState());
+			
+			// no repeated scheduling
+			try {
+				vertex.deployToSlot(slot);
+				fail("Scheduled from wrong state");
+			}
+			catch (IllegalStateException e) {}
+			
+			verify(taskManager).submitTask(Matchers.any(TaskDeploymentDescriptor.class));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testDeployFailedSynchronous() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, false, "failed"));
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexExecutingSynchronously(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			vertex.deployToSlot(slot);
+			
+			assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+			assertNotNull(vertex.getFailureCause());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testDeployFailedAsynchronously() {
+		try {
+			final JobVertexID jid = new JobVertexID();
+			
+			// mock taskmanager to simply accept the call
+			TaskOperationProtocol taskManager = mock(TaskOperationProtocol.class);
+			when(taskManager.submitTask(Matchers.any(TaskDeploymentDescriptor.class))).thenReturn(new TaskOperationResult(jid, 0, false, "failed"));
+			
+			final Instance instance = getInstance(taskManager);
+			final AllocatedSlot slot = instance.allocateSlot(new JobID());
+			
+			final ExecutionJobVertex ejv = getJobVertexExecutingAsynchronously(jid);
+			
+			final ExecutionVertex2 vertex = new ExecutionVertex2(ejv, 0, new IntermediateResult[0]);
+			
+			assertEquals(ExecutionState2.CREATED, vertex.getExecutionState());
+			vertex.deployToSlot(slot);
+			
+			// wait until the state transition must be done
+			for (int i = 0; i < 100; i++) {
+				if (vertex.getExecutionState() != ExecutionState2.FAILED) {
+					Thread.sleep(10);
+				}
+			}
+			
+			assertEquals(ExecutionState2.FAILED, vertex.getExecutionState());
+			assertNotNull(vertex.getFailureCause());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input1Output.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input1Output.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input1Output.java
deleted file mode 100644
index 3e500fe..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input1Output.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * 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 org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-public class ForwardTask1Input1Output extends AbstractInvokable {
-
-	private RecordReader<StringRecord> input = null;
-
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void invoke() throws Exception {
-		this.output.initializeSerializers();
-
-		while (this.input.hasNext()) {
-
-			StringRecord s = input.next();
-			this.output.emit(s);
-		}
-
-		this.output.flush();
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input2Outputs.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input2Outputs.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input2Outputs.java
deleted file mode 100644
index 162da14..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask1Input2Outputs.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * 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 org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-public class ForwardTask1Input2Outputs extends AbstractInvokable {
-
-	private RecordReader<StringRecord> input = null;
-
-	private RecordWriter<StringRecord> output1 = null;
-
-	private RecordWriter<StringRecord> output2 = null;
-
-	@Override
-	public void invoke() throws Exception {
-
-		this.output1.initializeSerializers();
-		this.output2.initializeSerializers();
-
-		while (this.input.hasNext()) {
-
-			StringRecord s = input.next();
-			this.output1.emit(s);
-			this.output2.emit(s);
-		}
-
-		this.output1.flush();
-		this.output2.flush();
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.input = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output1 = new RecordWriter<StringRecord>(this);
-		this.output2 = new RecordWriter<StringRecord>(this);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask2Inputs1Output.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask2Inputs1Output.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask2Inputs1Output.java
deleted file mode 100644
index 8035d92..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ForwardTask2Inputs1Output.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * 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 org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-public class ForwardTask2Inputs1Output extends AbstractInvokable {
-
-	private RecordReader<StringRecord> input1 = null;
-
-	private RecordReader<StringRecord> input2 = null;
-
-	private RecordWriter<StringRecord> output = null;
-
-	@Override
-	public void invoke() throws Exception {
-		this.output.initializeSerializers();
-
-		while (this.input1.hasNext()) {
-
-			StringRecord s = input1.next();
-			this.output.emit(s);
-		}
-
-		while (this.input2.hasNext()) {
-
-			try {
-				StringRecord s = input2.next();
-				this.output.emit(s);
-			} catch (InterruptedException e) {
-				e.printStackTrace();
-			}
-		}
-
-		this.output.flush();
-	}
-
-	@Override
-	public void registerInputOutput() {
-		this.input1 = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.input2 = new RecordReader<StringRecord>(this, StringRecord.class);
-		this.output = new RecordWriter<StringRecord>(this);
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
new file mode 100644
index 0000000..0ea00d3
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/PointwisePatternTest.java
@@ -0,0 +1,330 @@
+/**
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.JobException;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobID;
+
+
+public class PointwisePatternTest {
+
+	private final JobID jobId = new JobID();
+	private final String jobName = "Test Job Sample Name";
+	private final Configuration cfg = new Configuration();
+	
+	@Test
+	public void testNToN() {
+		final int N = 23;
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+	
+		v1.setParallelism(N);
+		v2.setParallelism(N);
+	
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+	
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
+		
+		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+			assertEquals(1, ev.getNumberOfInputs());
+			
+			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			assertEquals(1, inEdges.length);
+			
+			assertEquals(ev.getParallelSubtaskIndex(), inEdges[0].getSource().getPartition());
+		}
+	}
+	
+	@Test
+	public void test2NToN() {
+		final int N = 17;
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+	
+		v1.setParallelism(2 * N);
+		v2.setParallelism(N);
+	
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+	
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
+		
+		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+			assertEquals(1, ev.getNumberOfInputs());
+			
+			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			assertEquals(2, inEdges.length);
+			
+			assertEquals(ev.getParallelSubtaskIndex() * 2, inEdges[0].getSource().getPartition());
+			assertEquals(ev.getParallelSubtaskIndex() * 2 + 1, inEdges[1].getSource().getPartition());
+		}
+	}
+	
+	@Test
+	public void test3NToN() {
+		final int N = 17;
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+	
+		v1.setParallelism(3 * N);
+		v2.setParallelism(N);
+	
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+	
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
+		
+		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+			assertEquals(1, ev.getNumberOfInputs());
+			
+			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			assertEquals(3, inEdges.length);
+			
+			assertEquals(ev.getParallelSubtaskIndex() * 3, inEdges[0].getSource().getPartition());
+			assertEquals(ev.getParallelSubtaskIndex() * 3 + 1, inEdges[1].getSource().getPartition());
+			assertEquals(ev.getParallelSubtaskIndex() * 3 + 2, inEdges[2].getSource().getPartition());
+		}
+	}
+	
+	@Test
+	public void testNTo2N() {
+		final int N = 41;
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+	
+		v1.setParallelism(N);
+		v2.setParallelism(2 * N);
+	
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+	
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
+		
+		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+			assertEquals(1, ev.getNumberOfInputs());
+			
+			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			assertEquals(1, inEdges.length);
+			
+			assertEquals(ev.getParallelSubtaskIndex() / 2, inEdges[0].getSource().getPartition());
+		}
+	}
+	
+	@Test
+	public void testNTo7N() {
+		final int N = 11;
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+	
+		v1.setParallelism(N);
+		v2.setParallelism(7 * N);
+	
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+	
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
+		
+		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+			assertEquals(1, ev.getNumberOfInputs());
+			
+			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			assertEquals(1, inEdges.length);
+			
+			assertEquals(ev.getParallelSubtaskIndex() / 7, inEdges[0].getSource().getPartition());
+		}
+	}
+	
+	@Test
+	public void testLowHighIrregular() {
+		testLowToHigh(3, 16);
+		testLowToHigh(19, 21);
+		testLowToHigh(15, 20);
+		testLowToHigh(11, 31);
+	}
+	
+	@Test
+	public void testHighLowIrregular() {
+		testHighToLow(16, 3);
+		testHighToLow(21, 19);
+		testHighToLow(20, 15);
+		testHighToLow(31, 11);
+	}
+	
+	private void testLowToHigh(int lowDop, int highDop) {
+		if (highDop < lowDop) {
+			throw new IllegalArgumentException();
+		}
+		
+		final int factor = highDop / lowDop;
+		final int delta = highDop % lowDop == 0 ? 0 : 1;
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+	
+		v1.setParallelism(lowDop);
+		v2.setParallelism(highDop);
+	
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+	
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
+		
+		int[] timesUsed = new int[lowDop];
+		
+		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+			assertEquals(1, ev.getNumberOfInputs());
+			
+			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			assertEquals(1, inEdges.length);
+			
+			
+			timesUsed[inEdges[0].getSource().getPartition()]++;
+		}
+		
+		for (int i = 0; i < timesUsed.length; i++) {
+			assertTrue(timesUsed[i] >= factor && timesUsed[i] <= factor + delta);
+		}
+	}
+	
+	private void testHighToLow(int highDop, int lowDop) {
+		if (highDop < lowDop) {
+			throw new IllegalArgumentException();
+		}
+		
+		final int factor = highDop / lowDop;
+		final int delta = highDop % lowDop == 0 ? 0 : 1;
+		
+		AbstractJobVertex v1 = new AbstractJobVertex("vertex1");
+		AbstractJobVertex v2 = new AbstractJobVertex("vertex2");
+	
+		v1.setParallelism(highDop);
+		v2.setParallelism(lowDop);
+	
+		v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+	
+		List<AbstractJobVertex> ordered = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2));
+
+		ExecutionGraph eg = new ExecutionGraph(jobId, jobName, cfg);
+		try {
+			eg.attachJobGraph(ordered);
+		}
+		catch (JobException e) {
+			e.printStackTrace();
+			fail("Job failed with exception: " + e.getMessage());
+		}
+		
+		ExecutionJobVertex target = eg.getAllVertices().get(v2.getID());
+		
+		int[] timesUsed = new int[highDop];
+		
+		for (ExecutionVertex2 ev : target.getTaskVertices()) {
+			assertEquals(1, ev.getNumberOfInputs());
+			
+			ExecutionEdge2[] inEdges = ev.getInputEdges(0);
+			assertTrue(inEdges.length >= factor && inEdges.length <= factor + delta);
+			
+			for (ExecutionEdge2 ee : inEdges) {
+				timesUsed[ee.getSource().getPartition()]++;
+			}
+		}
+		
+		for (int i = 0; i < timesUsed.length; i++) {
+			assertEquals(1, timesUsed[i]);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java
deleted file mode 100644
index 7fafeda..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/SelfCrossForwardTask.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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 org.apache.flink.core.io.StringRecord;
-import org.apache.flink.runtime.io.network.api.RecordReader;
-import org.apache.flink.runtime.io.network.api.RecordWriter;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * This class represents the cross task in the self cross unit test.
- */
-public class SelfCrossForwardTask extends AbstractInvokable {
-
-	@Override
-	public void registerInputOutput() {
-		new RecordReader<StringRecord>(this, StringRecord.class);
-		new RecordReader<StringRecord>(this, StringRecord.class);
-		new RecordWriter<StringRecord>(this);
-	}
-
-	@Override
-	public void invoke() {}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java
new file mode 100644
index 0000000..540d9fd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/VertexSlotSharingTest.java
@@ -0,0 +1,103 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
+import org.apache.flink.runtime.jobgraph.DistributionPattern;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
+import org.junit.Test;
+
+public class VertexSlotSharingTest {
+
+	/*
+	 * Test setup:
+	 * - v1 is isolated, no slot sharing
+	 * - v2 and v3 (not connected) share slots
+	 * - v4 and v5 (connected) share slots
+	 */
+	@Test
+	public void testAssignSlotSharingGroup() {
+		try {
+			AbstractJobVertex v1 = new AbstractJobVertex("v1");
+			AbstractJobVertex v2 = new AbstractJobVertex("v2");
+			AbstractJobVertex v3 = new AbstractJobVertex("v3");
+			AbstractJobVertex v4 = new AbstractJobVertex("v4");
+			AbstractJobVertex v5 = new AbstractJobVertex("v5");
+			
+			v1.setParallelism(4);
+			v2.setParallelism(5);
+			v3.setParallelism(7);
+			v4.setParallelism(1);
+			v5.setParallelism(11);
+			
+			v2.connectNewDataSetAsInput(v1, DistributionPattern.POINTWISE);
+			v5.connectNewDataSetAsInput(v4, DistributionPattern.POINTWISE);
+			
+			SlotSharingGroup jg1 = new SlotSharingGroup();
+			v2.setSlotSharingGroup(jg1);
+			v3.setSlotSharingGroup(jg1);
+			
+			SlotSharingGroup jg2 = new SlotSharingGroup();
+			v4.setSlotSharingGroup(jg2);
+			v5.setSlotSharingGroup(jg2);
+			
+			List<AbstractJobVertex> vertices = new ArrayList<AbstractJobVertex>(Arrays.asList(v1, v2, v3, v4, v5));
+			
+			ExecutionGraph eg = new ExecutionGraph(new JobID(), "test job", new Configuration());
+			eg.attachJobGraph(vertices);
+			
+			// verify that the vertices are all in the same slot sharing group
+			SlotSharingGroup group1 = null;
+			SlotSharingGroup group2 = null;
+			
+			// verify that v1 tasks have no slot sharing group
+			assertNull(eg.getJobVertex(v1.getID()).getSlotSharingGroup());
+			
+			// v2 and v3 are shared
+			group1 = eg.getJobVertex(v2.getID()).getSlotSharingGroup();
+			assertNotNull(group1);
+			assertEquals(group1, eg.getJobVertex(v3.getID()).getSlotSharingGroup());
+			
+			assertEquals(2, group1.getJobVertexIds().size());
+			assertTrue(group1.getJobVertexIds().contains(v2.getID()));
+			assertTrue(group1.getJobVertexIds().contains(v3.getID()));
+			
+			// v4 and v5 are shared
+			group2 = eg.getJobVertex(v4.getID()).getSlotSharingGroup();
+			assertNotNull(group2);
+			assertEquals(group2, eg.getJobVertex(v5.getID()).getSlotSharingGroup());
+			
+			assertEquals(2, group1.getJobVertexIds().size());
+			assertTrue(group2.getJobVertexIds().contains(v4.getID()));
+			assertTrue(group2.getJobVertexIds().contains(v5.getID()));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java
new file mode 100644
index 0000000..c10a5ef
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/AllocatedSlotTest.java
@@ -0,0 +1,140 @@
+/**
+ * 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.instance;
+
+import static org.mockito.Mockito.*;
+import static org.junit.Assert.*;
+
+import java.net.InetAddress;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.junit.Test;
+import org.mockito.Matchers;
+
+public class AllocatedSlotTest {
+
+	@Test
+	public void testStateTransitions() {
+		try {
+			// cancel, then release
+			{
+				AllocatedSlot slot = getSlot();
+				assertTrue(slot.isAlive());
+				
+				slot.cancel();
+				assertFalse(slot.isAlive());
+				assertTrue(slot.isCanceled());
+				assertFalse(slot.isReleased());
+				
+				slot.releaseSlot();
+				assertFalse(slot.isAlive());
+				assertTrue(slot.isCanceled());
+				assertTrue(slot.isReleased());
+			}
+			
+			// release immediately
+			{
+				AllocatedSlot slot = getSlot();
+				assertTrue(slot.isAlive());
+				
+				slot.releaseSlot();
+				assertFalse(slot.isAlive());
+				assertTrue(slot.isCanceled());
+				assertTrue(slot.isReleased());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSetExecutionVertex() {
+		try {
+			ExecutionVertex2 ev = mock(ExecutionVertex2.class);
+			ExecutionVertex2 ev_2 = mock(ExecutionVertex2.class);
+			
+			// assign to alive slot
+			{
+				AllocatedSlot slot = getSlot();
+				
+				assertTrue(slot.setExecutedVertex(ev));
+				assertEquals(ev, slot.getExecutedVertex());
+				
+				// try to add another one
+				assertFalse(slot.setExecutedVertex(ev_2));
+				assertEquals(ev, slot.getExecutedVertex());
+			}
+			
+			// assign to canceled slot
+			{
+				AllocatedSlot slot = getSlot();
+				slot.cancel();
+				
+				assertFalse(slot.setExecutedVertex(ev));
+				assertNull(slot.getExecutedVertex());
+			}
+			
+			// assign to released
+			{
+				AllocatedSlot slot = getSlot();
+				slot.releaseSlot();
+				
+				assertFalse(slot.setExecutedVertex(ev));
+				assertNull(slot.getExecutedVertex());
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testReleaseCancelsVertex() {
+		try {
+			ExecutionVertex2 ev = mock(ExecutionVertex2.class);
+			
+			AllocatedSlot slot = getSlot();
+			assertTrue(slot.setExecutedVertex(ev));
+			assertEquals(ev, slot.getExecutedVertex());
+			
+			slot.cancel();
+			slot.releaseSlot();
+			slot.cancel();
+			
+			verify(ev, times(1)).fail(Matchers.any(Throwable.class));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	public static AllocatedSlot getSlot() throws Exception {
+		HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024);
+		InetAddress address = InetAddress.getByName("127.0.0.1");
+		InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10000, 10001);
+		
+		Instance instance = new Instance(connection, new InstanceID(), hardwareDescription, 1);
+		return instance.allocateSlot(new JobID());
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java
deleted file mode 100644
index 0bec1f3..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/DefaultInstanceManagerTest.java
+++ /dev/null
@@ -1,275 +0,0 @@
-/**
- * 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.instance;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.net.InetAddress;
-
-import org.apache.flink.runtime.testutils.CommonTestUtils;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-/**
- * Tests for {@link org.apache.flink.runtime.instance.DefaultInstanceManager}.
- */
-public class DefaultInstanceManagerTest {
-	
-	@Test
-	public void testInstanceRegistering() {
-		try {
-			DefaultInstanceManager cm = new DefaultInstanceManager();
-			
-			final int ipcPort = 10000;
-			final int dataPort = 20000;
-
-			HardwareDescription hardwareDescription = HardwareDescription.extractFromSystem(4096);
-
-			InetAddress address = InetAddress.getByName("127.0.0.1");
-			
-			// register three instances
-			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, ipcPort + 0, dataPort + 0);
-			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, ipcPort + 15, dataPort + 15);
-			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, ipcPort + 30, dataPort + 30);
-			
-			InstanceID i1 = cm.registerTaskManager(ici1, hardwareDescription, 1);
-			InstanceID i2 = cm.registerTaskManager(ici2, hardwareDescription, 2);
-			InstanceID i3 = cm.registerTaskManager(ici3, hardwareDescription, 5);
-			
-			assertEquals(3, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(8, cm.getTotalNumberOfSlots());
-			
-			assertEquals(ici1, cm.getAllRegisteredInstances().get(i1).getInstanceConnectionInfo());
-			assertEquals(ici2, cm.getAllRegisteredInstances().get(i2).getInstanceConnectionInfo());
-			assertEquals(ici3, cm.getAllRegisteredInstances().get(i3).getInstanceConnectionInfo());
-
-			cm.shutdown();
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail("Test erroneous: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testRegisteringAlreadyRegistered() {
-		try {
-			DefaultInstanceManager cm = new DefaultInstanceManager();
-			
-			final int ipcPort = 10000;
-			final int dataPort = 20000;
-
-			HardwareDescription resources = HardwareDescription.extractFromSystem(4096);
-			InetAddress address = InetAddress.getByName("127.0.0.1");
-			InstanceConnectionInfo ici = new InstanceConnectionInfo(address, ipcPort + 0, dataPort + 0);
-			
-			InstanceID i = cm.registerTaskManager(ici, resources, 1);
-
-			assertNotNull(i);
-			assertEquals(1, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(1, cm.getTotalNumberOfSlots());
-			
-			InstanceID next = cm.registerTaskManager(ici, resources, 1);
-			assertNull(next);
-			
-			assertEquals(1, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(1, cm.getTotalNumberOfSlots());
-
-			cm.shutdown();
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail("Test erroneous: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testReportHeartbeat() {
-		try {
-			DefaultInstanceManager cm = new DefaultInstanceManager();
-			
-			final int ipcPort = 10000;
-			final int dataPort = 20000;
-
-			HardwareDescription hardwareDescription = HardwareDescription.extractFromSystem(4096);
-
-			InetAddress address = InetAddress.getByName("127.0.0.1");
-			
-			// register three instances
-			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, ipcPort + 0, dataPort + 0);
-			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, ipcPort + 1, dataPort + 1);
-			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, ipcPort + 2, dataPort + 2);
-			
-			InstanceID i1 = cm.registerTaskManager(ici1, hardwareDescription, 1);
-			InstanceID i2 = cm.registerTaskManager(ici2, hardwareDescription, 1);
-			InstanceID i3 = cm.registerTaskManager(ici3, hardwareDescription, 1);
-
-			// report some immediate heart beats
-			assertTrue(cm.reportHeartBeat(i1));
-			assertTrue(cm.reportHeartBeat(i2));
-			assertTrue(cm.reportHeartBeat(i3));
-			
-			// report heart beat for non-existing instance
-			assertFalse(cm.reportHeartBeat(new InstanceID()));
-			
-			final long WAIT = 200;
-			CommonTestUtils.sleepUninterruptibly(WAIT);
-			
-			long h1 = cm.getAllRegisteredInstances().get(i1).getLastHeartBeat();
-			long h2 = cm.getAllRegisteredInstances().get(i2).getLastHeartBeat();
-			long h3 = cm.getAllRegisteredInstances().get(i3).getLastHeartBeat();
-
-			// send one heart beat again and verify that the
-			assertTrue(cm.reportHeartBeat(i1));
-			long newH1 = cm.getAllRegisteredInstances().get(i1).getLastHeartBeat();
-			
-			long now = System.currentTimeMillis();
-			
-			assertTrue(now - h1 >= WAIT);
-			assertTrue(now - h2 >= WAIT);
-			assertTrue(now - h3 >= WAIT);
-			assertTrue(now - newH1 <= WAIT);
-			
-			cm.shutdown();
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail("Test erroneous: " + e.getMessage());
-		}
-	}
-	
-	@Test
-	public void testShutdown() {
-		try {
-			DefaultInstanceManager cm = new DefaultInstanceManager();
-			cm.shutdown();
-			
-			try {
-				HardwareDescription resources = HardwareDescription.extractFromSystem(4096);
-				InetAddress address = InetAddress.getByName("127.0.0.1");
-				InstanceConnectionInfo ici = new InstanceConnectionInfo(address, 10000, 20000);
-		
-				cm.registerTaskManager(ici, resources, 1);
-				fail("Should raise exception in shutdown state");
-			}
-			catch (IllegalStateException e) {
-				// expected
-			}
-			
-			try {
-				cm.reportHeartBeat(new InstanceID());
-				fail("Should raise exception in shutdown state");
-			}
-			catch (IllegalStateException e) {
-				// expected
-			}
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail("Test erroneous: " + e.getMessage());
-		}
-	}
-
-	/**
-	 * This test checks the clean-up routines of the cluster manager.
-	 */
-	@Test
-	public void testCleanUp() {
-		try {
-			DefaultInstanceManager cm = new DefaultInstanceManager(200, 100);
-
-			HardwareDescription resources = HardwareDescription.extractFromSystem(4096);
-			InetAddress address = InetAddress.getByName("127.0.0.1");
-			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, 10000, 20000);
-			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, 10001, 20001);
-
-			// register three instances
-			InstanceID i1 = cm.registerTaskManager(ici1, resources, 1);
-			InstanceID i2 = cm.registerTaskManager(ici2, resources, 1);
-
-			assertNotNull(i1);
-			assertNotNull(i2);
-			
-			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(2, cm.getTotalNumberOfSlots());
-
-			// report a few heatbeats for both of the machines (each 50 msecs)...
-			for (int i = 0; i < 8; i++) {
-				CommonTestUtils.sleepUninterruptibly(50);
-				
-				assertTrue(cm.reportHeartBeat(i1));
-				assertTrue(cm.reportHeartBeat(i2));
-			}
-			
-			// all should be alive
-			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(2, cm.getTotalNumberOfSlots());
-
-			// report a few heatbeats for both only one machine
-			for (int i = 0; i < 8; i++) {
-				CommonTestUtils.sleepUninterruptibly(50);
-				
-				assertTrue(cm.reportHeartBeat(i1));
-			}
-			
-			// we should have lost one TM by now
-			assertEquals(1, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(1, cm.getTotalNumberOfSlots());
-			
-			// if the lost TM reports, it should not be accepted
-			assertFalse(cm.reportHeartBeat(i2));
-			
-			// allow the lost TM to re-register itself
-			i2 = cm.registerTaskManager(ici2, resources, 1);
-			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(2, cm.getTotalNumberOfSlots());
-			
-			// report a few heatbeats for both of the machines (each 50 msecs)...
-			for (int i = 0; i < 8; i++) {
-				CommonTestUtils.sleepUninterruptibly(50);
-				
-				assertTrue(cm.reportHeartBeat(i1));
-				assertTrue(cm.reportHeartBeat(i2));
-			}
-			
-			// all should be alive
-			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
-			assertEquals(2, cm.getTotalNumberOfSlots());
-
-			
-			cm.shutdown();
-		}
-		catch (Exception e) {
-			System.err.println(e.getMessage());
-			e.printStackTrace();
-			Assert.fail("Test erroneous: " + e.getMessage());
-		}
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
new file mode 100644
index 0000000..e15b461
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceManagerTest.java
@@ -0,0 +1,276 @@
+/**
+ * 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.instance;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.net.InetAddress;
+
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Tests for {@link org.apache.flink.runtime.instance.InstanceManager}.
+ */
+public class InstanceManagerTest {
+	
+	
+	@Test
+	public void testInstanceRegistering() {
+		try {
+			InstanceManager cm = new InstanceManager();
+			
+			final int ipcPort = 10000;
+			final int dataPort = 20000;
+
+			HardwareDescription hardwareDescription = HardwareDescription.extractFromSystem(4096);
+
+			InetAddress address = InetAddress.getByName("127.0.0.1");
+			
+			// register three instances
+			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, ipcPort + 0, dataPort + 0);
+			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, ipcPort + 15, dataPort + 15);
+			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, ipcPort + 30, dataPort + 30);
+			
+			InstanceID i1 = cm.registerTaskManager(ici1, hardwareDescription, 1);
+			InstanceID i2 = cm.registerTaskManager(ici2, hardwareDescription, 2);
+			InstanceID i3 = cm.registerTaskManager(ici3, hardwareDescription, 5);
+			
+			assertEquals(3, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(8, cm.getTotalNumberOfSlots());
+			
+			assertEquals(ici1, cm.getAllRegisteredInstances().get(i1).getInstanceConnectionInfo());
+			assertEquals(ici2, cm.getAllRegisteredInstances().get(i2).getInstanceConnectionInfo());
+			assertEquals(ici3, cm.getAllRegisteredInstances().get(i3).getInstanceConnectionInfo());
+
+			cm.shutdown();
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail("Test erroneous: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testRegisteringAlreadyRegistered() {
+		try {
+			InstanceManager cm = new InstanceManager();
+			
+			final int ipcPort = 10000;
+			final int dataPort = 20000;
+
+			HardwareDescription resources = HardwareDescription.extractFromSystem(4096);
+			InetAddress address = InetAddress.getByName("127.0.0.1");
+			InstanceConnectionInfo ici = new InstanceConnectionInfo(address, ipcPort + 0, dataPort + 0);
+			
+			InstanceID i = cm.registerTaskManager(ici, resources, 1);
+
+			assertNotNull(i);
+			assertEquals(1, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(1, cm.getTotalNumberOfSlots());
+			
+			InstanceID next = cm.registerTaskManager(ici, resources, 1);
+			assertNull(next);
+			
+			assertEquals(1, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(1, cm.getTotalNumberOfSlots());
+
+			cm.shutdown();
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail("Test erroneous: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testReportHeartbeat() {
+		try {
+			InstanceManager cm = new InstanceManager();
+			
+			final int ipcPort = 10000;
+			final int dataPort = 20000;
+
+			HardwareDescription hardwareDescription = HardwareDescription.extractFromSystem(4096);
+
+			InetAddress address = InetAddress.getByName("127.0.0.1");
+			
+			// register three instances
+			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, ipcPort + 0, dataPort + 0);
+			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, ipcPort + 1, dataPort + 1);
+			InstanceConnectionInfo ici3 = new InstanceConnectionInfo(address, ipcPort + 2, dataPort + 2);
+			
+			InstanceID i1 = cm.registerTaskManager(ici1, hardwareDescription, 1);
+			InstanceID i2 = cm.registerTaskManager(ici2, hardwareDescription, 1);
+			InstanceID i3 = cm.registerTaskManager(ici3, hardwareDescription, 1);
+
+			// report some immediate heart beats
+			assertTrue(cm.reportHeartBeat(i1));
+			assertTrue(cm.reportHeartBeat(i2));
+			assertTrue(cm.reportHeartBeat(i3));
+			
+			// report heart beat for non-existing instance
+			assertFalse(cm.reportHeartBeat(new InstanceID()));
+			
+			final long WAIT = 200;
+			CommonTestUtils.sleepUninterruptibly(WAIT);
+			
+			long h1 = cm.getAllRegisteredInstances().get(i1).getLastHeartBeat();
+			long h2 = cm.getAllRegisteredInstances().get(i2).getLastHeartBeat();
+			long h3 = cm.getAllRegisteredInstances().get(i3).getLastHeartBeat();
+
+			// send one heart beat again and verify that the
+			assertTrue(cm.reportHeartBeat(i1));
+			long newH1 = cm.getAllRegisteredInstances().get(i1).getLastHeartBeat();
+			
+			long now = System.currentTimeMillis();
+			
+			assertTrue(now - h1 >= WAIT);
+			assertTrue(now - h2 >= WAIT);
+			assertTrue(now - h3 >= WAIT);
+			assertTrue(now - newH1 <= WAIT);
+			
+			cm.shutdown();
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail("Test erroneous: " + e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testShutdown() {
+		try {
+			InstanceManager cm = new InstanceManager();
+			cm.shutdown();
+			
+			try {
+				HardwareDescription resources = HardwareDescription.extractFromSystem(4096);
+				InetAddress address = InetAddress.getByName("127.0.0.1");
+				InstanceConnectionInfo ici = new InstanceConnectionInfo(address, 10000, 20000);
+		
+				cm.registerTaskManager(ici, resources, 1);
+				fail("Should raise exception in shutdown state");
+			}
+			catch (IllegalStateException e) {
+				// expected
+			}
+			
+			try {
+				cm.reportHeartBeat(new InstanceID());
+				fail("Should raise exception in shutdown state");
+			}
+			catch (IllegalStateException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail("Test erroneous: " + e.getMessage());
+		}
+	}
+
+	/**
+	 * This test checks the clean-up routines of the cluster manager.
+	 */
+	@Test
+	public void testCleanUp() {
+		try {
+			InstanceManager cm = new InstanceManager(200, 100);
+
+			HardwareDescription resources = HardwareDescription.extractFromSystem(4096);
+			InetAddress address = InetAddress.getByName("127.0.0.1");
+			InstanceConnectionInfo ici1 = new InstanceConnectionInfo(address, 10000, 20000);
+			InstanceConnectionInfo ici2 = new InstanceConnectionInfo(address, 10001, 20001);
+
+			// register three instances
+			InstanceID i1 = cm.registerTaskManager(ici1, resources, 1);
+			InstanceID i2 = cm.registerTaskManager(ici2, resources, 1);
+
+			assertNotNull(i1);
+			assertNotNull(i2);
+			
+			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(2, cm.getTotalNumberOfSlots());
+
+			// report a few heatbeats for both of the machines (each 50 msecs)...
+			for (int i = 0; i < 8; i++) {
+				CommonTestUtils.sleepUninterruptibly(50);
+				
+				assertTrue(cm.reportHeartBeat(i1));
+				assertTrue(cm.reportHeartBeat(i2));
+			}
+			
+			// all should be alive
+			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(2, cm.getTotalNumberOfSlots());
+
+			// report a few heatbeats for both only one machine
+			for (int i = 0; i < 8; i++) {
+				CommonTestUtils.sleepUninterruptibly(50);
+				
+				assertTrue(cm.reportHeartBeat(i1));
+			}
+			
+			// we should have lost one TM by now
+			assertEquals(1, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(1, cm.getTotalNumberOfSlots());
+			
+			// if the lost TM reports, it should not be accepted
+			assertFalse(cm.reportHeartBeat(i2));
+			
+			// allow the lost TM to re-register itself
+			i2 = cm.registerTaskManager(ici2, resources, 1);
+			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(2, cm.getTotalNumberOfSlots());
+			
+			// report a few heatbeats for both of the machines (each 50 msecs)...
+			for (int i = 0; i < 8; i++) {
+				CommonTestUtils.sleepUninterruptibly(50);
+				
+				assertTrue(cm.reportHeartBeat(i1));
+				assertTrue(cm.reportHeartBeat(i2));
+			}
+			
+			// all should be alive
+			assertEquals(2, cm.getNumberOfRegisteredTaskManagers());
+			assertEquals(2, cm.getTotalNumberOfSlots());
+
+			
+			cm.shutdown();
+		}
+		catch (Exception e) {
+			System.err.println(e.getMessage());
+			e.printStackTrace();
+			Assert.fail("Test erroneous: " + e.getMessage());
+		}
+	}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
index bce692d..09da1a5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
+import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.accumulators.AccumulatorEvent;
 import org.apache.flink.runtime.client.JobCancelResult;
@@ -34,8 +35,6 @@ import org.apache.flink.runtime.event.job.AbstractEvent;
 import org.apache.flink.runtime.event.job.RecentJobEvent;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.executiongraph.InternalJobStatus;
 import org.apache.flink.runtime.executiongraph.JobStatusListener;
 import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
@@ -43,10 +42,10 @@ import org.apache.flink.runtime.ipc.RPC;
 import org.apache.flink.runtime.ipc.RPC.Server;
 import org.apache.flink.runtime.jobgraph.JobGraph;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobStatus;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobmanager.DeploymentManager;
-import org.apache.flink.runtime.jobmanager.splitassigner.InputSplitWrapper;
 import org.apache.flink.runtime.managementgraph.ManagementGraph;
-import org.apache.flink.runtime.managementgraph.ManagementVertexID;
 import org.apache.flink.runtime.protocols.AccumulatorProtocol;
 import org.apache.flink.runtime.protocols.ChannelLookupProtocol;
 import org.apache.flink.runtime.protocols.ExtendedManagementProtocol;
@@ -204,7 +203,7 @@ public class LocalInstanceManagerTest {
 		}
 
 		@Override
-		public void jobStatusHasChanged(ExecutionGraph executionGraph, InternalJobStatus newJobStatus, String optionalMessage) {}
+		public void jobStatusHasChanged(ExecutionGraph executionGraph, JobStatus newJobStatus, String optionalMessage) {}
 
 		@Override
 		public ConnectionInfoLookupResponse lookupConnectionInfo(InstanceConnectionInfo caller, JobID jobID, ChannelID sourceChannelID) {
@@ -215,11 +214,6 @@ public class LocalInstanceManagerTest {
 		public void updateTaskExecutionState(TaskExecutionState taskExecutionState) {}
 
 		@Override
-		public InputSplitWrapper requestNextInputSplit(JobID jobID, ExecutionVertexID vertexID, IntegerRecord sequenceNumber) {
-			return null;
-		}
-
-		@Override
 		public ManagementGraph getManagementGraph(JobID jobID) {
 			return null;
 		}
@@ -235,12 +229,6 @@ public class LocalInstanceManagerTest {
 		}
 
 		@Override
-		public void killTask(JobID jobID, ManagementVertexID id) {}
-
-		@Override
-		public void logBufferUtilization(JobID jobID) {}
-
-		@Override
 		public int getAvailableSlots() {
 			return 0;
 		}
@@ -257,5 +245,10 @@ public class LocalInstanceManagerTest {
 		public InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) {
 			return new InstanceID();
 		}
+
+		@Override
+		public InputSplit requestNextInputSplit(JobID jobID, JobVertexID vertex) throws IOException {
+			return null;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotTestUtils.java
new file mode 100644
index 0000000..ca2ef70
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/SlotTestUtils.java
@@ -0,0 +1,23 @@
+/**
+ * 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.instance;
+
+public class SlotTestUtils {
+
+}


[06/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
index f074f3c..67e4700 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
@@ -31,15 +31,12 @@ import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketAddress;
 import java.net.UnknownHostException;
-import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
-import java.util.Timer;
-import java.util.TimerTask;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -62,29 +59,30 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.protocols.VersionedProtocol;
 import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.ExecutionState2;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileRequest;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileResponse;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheUpdate;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.filecache.FileCache;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.ChannelManager;
-import org.apache.flink.runtime.io.network.InsufficientResourcesException;
 import org.apache.flink.runtime.io.network.LocalConnectionManager;
 import org.apache.flink.runtime.io.network.NetworkConnectionManager;
-import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.io.network.netty.NettyConnectionManager;
 import org.apache.flink.runtime.ipc.RPC;
 import org.apache.flink.runtime.ipc.Server;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
 import org.apache.flink.runtime.memorymanager.MemoryManager;
 import org.apache.flink.runtime.net.NetUtils;
@@ -95,10 +93,9 @@ import org.apache.flink.runtime.protocols.ChannelLookupProtocol;
 import org.apache.flink.runtime.protocols.InputSplitProviderProtocol;
 import org.apache.flink.runtime.protocols.JobManagerProtocol;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
-import org.apache.flink.runtime.types.IntegerRecord;
 import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
-import org.apache.flink.runtime.util.SerializableArrayList;
+import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -106,7 +103,6 @@ import org.apache.flink.util.StringUtils;
  * (or in case of an execution error) it reports the execution result back to the job manager.
  * Task managers are able to automatically discover the job manager and receive its configuration from it
  * as long as the job manager is running on the same local network
- * 
  */
 public class TaskManager implements TaskOperationProtocol {
 
@@ -114,10 +110,6 @@ public class TaskManager implements TaskOperationProtocol {
 
 	private static final int STARTUP_FAILURE_RETURN_CODE = 1;
 	
-	private static final int CRITICAL_ERROR_RETURN_CODE = 2;
-	
-	private static final int IPC_HANDLER_COUNT = 1;
-	
 	private static final int MAX_LOST_HEART_BEATS = 3;
 	
 	private static final int DELAY_AFTER_LOST_CONNECTION = 10000;
@@ -127,42 +119,36 @@ public class TaskManager implements TaskOperationProtocol {
 	
 	// --------------------------------------------------------------------------------------------
 	
+	private final ExecutorService executorService = Executors.newCachedThreadPool(ExecutorThreadFactory.INSTANCE);
+	
+	
 	private final InstanceConnectionInfo localInstanceConnectionInfo;
 	
 	private final HardwareDescription hardwareDescription;
 	
 	private final ExecutionMode executionMode;
 	
+	
 	private final JobManagerProtocol jobManager;
 
 	private final InputSplitProviderProtocol globalInputSplitProvider;
 
 	private final ChannelLookupProtocol lookupService;
-
-	private final ExecutorService executorService = Executors.newCachedThreadPool(ExecutorThreadFactory.INSTANCE);
 	
 	private final AccumulatorProtocol accumulatorProtocolProxy;
 
+	
 	private final Server taskManagerServer;
 
 	private final FileCache fileCache = new FileCache();
 	
-	/**
-	 * This map contains all the tasks whose threads are in a state other than TERMINATED. If any task
-	 * is stored inside this map and its thread status is TERMINATED, this indicates a virtual machine error.
-	 * As a result, task status will switch to FAILED and reported to the {@link org.apache.flink.runtime.jobmanager.JobManager}.
-	 */
-	private final Map<ExecutionVertexID, Task> runningTasks = new ConcurrentHashMap<ExecutionVertexID, Task>();
+	/** All currently running tasks */
+	private final ConcurrentHashMap<ExecutionAttemptID, Task> runningTasks = new ConcurrentHashMap<ExecutionAttemptID, Task>();
 
-	/**
-	 * The instance of the {@link ChannelManager} which is responsible for
-	 * setting up and cleaning up the byte buffered channels of the tasks.
-	 */
+	/** The {@link ChannelManager} sets up and cleans up the data exchange channels of the tasks. */
 	private final ChannelManager channelManager;
 
-	/**
-	 * Instance of the task manager profile if profiling is enabled.
-	 */
+	/** Instance of the task manager profile if profiling is enabled. */
 	private final TaskManagerProfiler profiler;
 
 	private final MemoryManager memoryManager;
@@ -181,50 +167,39 @@ public class TaskManager implements TaskOperationProtocol {
 	private volatile boolean shutdownComplete;
 	
 	
-	/**
-	 * All parameters are obtained from the 
-	 * {@link GlobalConfiguration}, which must be loaded prior to instantiating the task manager.
-	 */
-	public TaskManager(ExecutionMode executionMode) throws Exception {
-		if (executionMode == null) {
-			throw new NullPointerException("Execution mode must not be null.");
+	// --------------------------------------------------------------------------------------------
+	//  Constructor & Shutdown
+	// --------------------------------------------------------------------------------------------
+	
+	public TaskManager(ExecutionMode executionMode, JobManagerProtocol jobManager, InputSplitProviderProtocol splitProvider, 
+			ChannelLookupProtocol channelLookup, AccumulatorProtocol accumulators,
+			InetSocketAddress jobManagerAddress, InetAddress taskManagerBindAddress)
+		throws Exception
+	{
+		if (executionMode == null || jobManager == null || splitProvider == null || channelLookup == null || accumulators == null) {
+			throw new NullPointerException();
 		}
 		
-		LOG.info("Execution mode: " + executionMode);
+		LOG.info("TaskManager execution mode: " + executionMode);
+		
 		this.executionMode = executionMode;
-
-		// IMPORTANT! At this point, the GlobalConfiguration must have been read!
+		this.jobManager = jobManager;
+		this.lookupService = channelLookup;
+		this.globalInputSplitProvider = splitProvider;
+		this.accumulatorProtocolProxy = accumulators;
 		
-		final InetSocketAddress jobManagerAddress;
+		// initialize the number of slots
 		{
-			LOG.info("Reading location of job manager from configuration");
-			
-			final String address = GlobalConfiguration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
-			final int port = GlobalConfiguration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
-			
-			if (address == null) {
-				throw new Exception("Job manager address not configured in the GlobalConfiguration.");
-			}
-	
-			// Try to convert configured address to {@link InetAddress}
-			try {
-				final InetAddress tmpAddress = InetAddress.getByName(address);
-				jobManagerAddress = new InetSocketAddress(tmpAddress, port);
-			}
-			catch (UnknownHostException e) {
-				LOG.fatal("Could not resolve JobManager host name.");
-				throw new Exception("Could not resolve JobManager host name: " + e.getMessage(), e);
+			int slots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, -1);
+			if (slots == -1) {
+				slots = 1;
+				LOG.info("Number of task slots not configured. Creating one task slot.");
+			} else if (slots <= 0) {
+				throw new Exception("Illegal value for the number of task slots: " + slots);
+			} else {
+				LOG.info("Creating " + slots + " task slot(s).");
 			}
-			
-			LOG.info("Connecting to JobManager at: " + jobManagerAddress);
-		}
-		
-		// Create RPC connection to the JobManager
-		try {
-			this.jobManager = RPC.getProxy(JobManagerProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
-		} catch (IOException e) {
-			LOG.fatal("Could not connect to the JobManager: " + e.getMessage(), e);
-			throw new Exception("Failed to initialize connection to JobManager: " + e.getMessage(), e);
+			this.numberOfSlots = slots;
 		}
 		
 		int ipcPort = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_IPC_PORT_KEY, -1);
@@ -236,52 +211,18 @@ public class TaskManager implements TaskOperationProtocol {
 			dataPort = getAvailablePort();
 		}
 		
-		// Determine our own public facing address and start the server
-		{
-			final InetAddress taskManagerAddress;
-			try {
-				taskManagerAddress = getTaskManagerAddress(jobManagerAddress);
-			}
-			catch (Exception e) {
-				throw new RuntimeException("The TaskManager failed to connect to the JobManager.", e);
-			}
-			
-			this.localInstanceConnectionInfo = new InstanceConnectionInfo(taskManagerAddress, ipcPort, dataPort);
-			LOG.info("TaskManager connection information:" + this.localInstanceConnectionInfo);
-
-			// Start local RPC server
-			try {
-				this.taskManagerServer = RPC.getServer(this, taskManagerAddress.getHostAddress(), ipcPort, IPC_HANDLER_COUNT);
-				this.taskManagerServer.start();
-			} catch (IOException e) {
-				LOG.fatal("Failed to start TaskManager server. " + e.getMessage(), e);
-				throw new Exception("Failed to start taskmanager server. " + e.getMessage(), e);
-			}
-		}
-		
-		// Try to create local stub of the global input split provider
-		try {
-			this.globalInputSplitProvider = RPC.getProxy(InputSplitProviderProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
-		} catch (IOException e) {
-			LOG.fatal(e.getMessage(), e);
-			throw new Exception("Failed to initialize connection to global input split provider: " + e.getMessage(), e);
-		}
-
-		// Try to create local stub for the lookup service
-		try {
-			this.lookupService = RPC.getProxy(ChannelLookupProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
-		} catch (IOException e) {
-			LOG.fatal(e.getMessage(), e);
-			throw new Exception("Failed to initialize channel lookup protocol. " + e.getMessage(), e);
-		}
+		this.localInstanceConnectionInfo = new InstanceConnectionInfo(taskManagerBindAddress, ipcPort, dataPort);
+		LOG.info("TaskManager connection information:" + this.localInstanceConnectionInfo);
 
-		// Try to create local stub for the accumulators
+		// Start local RPC server, give it the number of threads as we have slots
 		try {
-			this.accumulatorProtocolProxy = RPC.getProxy(AccumulatorProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
+			this.taskManagerServer = RPC.getServer(this, taskManagerBindAddress.getHostAddress(), ipcPort, numberOfSlots);
+			this.taskManagerServer.start();
 		} catch (IOException e) {
-			LOG.fatal("Failed to initialize accumulator protocol: " + e.getMessage(), e);
-			throw new Exception("Failed to initialize accumulator protocol: " + e.getMessage(), e);
+			LOG.fatal("Failed to start TaskManager server. " + e.getMessage(), e);
+			throw new Exception("Failed to start taskmanager server. " + e.getMessage(), e);
 		}
+		
 
 		// Load profiler if it should be used
 		if (GlobalConfiguration.getBoolean(ProfilingUtils.ENABLE_PROFILING_KEY, false)) {
@@ -348,20 +289,6 @@ public class TaskManager implements TaskOperationProtocol {
 			LOG.error(StringUtils.stringifyException(ioe));
 			throw new Exception("Failed to instantiate ChannelManager.", ioe);
 		}
-
-		// initialize the number of slots
-		{
-			int slots = GlobalConfiguration.getInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, -1);
-			if (slots == -1) {
-				slots = 1;
-				LOG.info("Number of task slots not configured. Creating one task slot.");
-			} else if (slots <= 0) {
-				throw new Exception("Illegal value for the number of task slots: " + slots);
-			} else {
-				LOG.info("Creating " + slots + " task slot(s).");
-			}
-			this.numberOfSlots = slots;
-		}
 		
 		// initialize the memory manager
 		{
@@ -457,626 +384,328 @@ public class TaskManager implements TaskOperationProtocol {
 			}).start();
 		}
 	}
-
-	private int getAvailablePort() {
-		ServerSocket serverSocket = null;
-		int port = 0;
-		for (int i = 0; i < 50; i++){
-			try {
-				serverSocket = new ServerSocket(0);
-				port = serverSocket.getLocalPort();
-				if (port != 0) {
-					serverSocket.close();
-					break;
-				}
-			} catch (IOException e) {
-				LOG.debug("Unable to allocate port " + e.getMessage(), e);
-			}
-		}
-		if (!serverSocket.isClosed()) {
-			try {
-				serverSocket.close();
-			} catch (IOException e) {
-				LOG.debug("error closing port",e);
-			}
-		}
-		return port;
-	}
-
+	
 	/**
-	 * Entry point for the program.
-	 * 
-	 * @param args
-	 *        arguments from the command line
-	 * @throws IOException 
+	 * Shuts the task manager down.
 	 */
-	@SuppressWarnings("static-access")
-	public static void main(String[] args) throws IOException {		
-		Option configDirOpt = OptionBuilder.withArgName("config directory").hasArg().withDescription(
-			"Specify configuration directory.").create("configDir");
-		// tempDir option is used by the YARN client.
-		Option tempDir = OptionBuilder.withArgName("temporary directory (overwrites configured option)")
-				.hasArg().withDescription(
-				"Specify temporary directory.").create(ARG_CONF_DIR);
-		configDirOpt.setRequired(true);
-		tempDir.setRequired(false);
-		Options options = new Options();
-		options.addOption(configDirOpt);
-		options.addOption(tempDir);
+	public void shutdown() {
+		if (!this.shutdownStarted.compareAndSet(false, true)) {
+			return;
+		}
+
+		LOG.info("Shutting down TaskManager");
 		
+		// first, stop the heartbeat thread and wait for it to terminate
+		this.heartbeatThread.interrupt();
+		try {
+			this.heartbeatThread.join(1000);
+		} catch (InterruptedException e) {}
 
-		CommandLineParser parser = new GnuParser();
-		CommandLine line = null;
+		// Stop RPC proxy for the task manager
+		stopProxy(this.jobManager);
+
+		// Stop RPC proxy for the global input split assigner
+		stopProxy(this.globalInputSplitProvider);
+
+		// Stop RPC proxy for the lookup service
+		stopProxy(this.lookupService);
+
+		// Stop RPC proxy for accumulator reports
+		stopProxy(this.accumulatorProtocolProxy);
+
+		// Shut down the own RPC server
 		try {
-			line = parser.parse(options, args);
-		} catch (ParseException e) {
-			System.err.println("CLI Parsing failed. Reason: " + e.getMessage());
-			System.exit(STARTUP_FAILURE_RETURN_CODE);
+			this.taskManagerServer.stop();
+		} catch (Throwable t) {
+			LOG.warn("TaskManager RPC server did not shut down properly.", t);
 		}
 
-		String configDir = line.getOptionValue(configDirOpt.getOpt(), null);
-		String tempDirVal = line.getOptionValue(tempDir.getOpt(), null);
-
-		// First, try to load global configuration
-		GlobalConfiguration.loadConfiguration(configDir);
-		if(tempDirVal != null // the YARN TM runner has set a value for the temp dir
-				// the configuration does not contain a temp direcory
-				&& GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, null) == null) {
-			Configuration c = GlobalConfiguration.getConfiguration();
-			c.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, tempDirVal);
-			LOG.info("Setting temporary directory to "+tempDirVal);
-			GlobalConfiguration.includeConfiguration(c);
+		// Stop profiling if enabled
+		if (this.profiler != null) {
+			this.profiler.shutdown();
 		}
-		
-		// print some startup environment info, like user, code revision, etc
-		EnvironmentInformation.logEnvironmentInfo(LOG, "TaskManager");
-		
-		// Create a new task manager object
+
+		// Shut down the channel manager
 		try {
-			new TaskManager(ExecutionMode.CLUSTER);
-		} catch (Exception e) {
-			LOG.fatal("Taskmanager startup failed: " + e.getMessage(), e);
-			System.exit(STARTUP_FAILURE_RETURN_CODE);
+			this.channelManager.shutdown();
+		} catch (Throwable t) {
+			LOG.warn("ChannelManager did not shutdown properly: " + t.getMessage(), t);
 		}
-		
-		// park the main thread to keep the JVM alive (all other threads may be daemon threads)
-		Object mon = new Object();
-		synchronized (mon) {
-			try {
-				mon.wait();
-			} catch (InterruptedException ex) {}
+
+		// Shut down the memory manager
+		if (this.ioManager != null) {
+			this.ioManager.shutdown();
+		}
+
+		if (this.memoryManager != null) {
+			this.memoryManager.shutdown();
 		}
-	}
 
+		this.fileCache.shutdown();
 
+		// Shut down the executor service
+		if (this.executorService != null) {
+			this.executorService.shutdown();
+			try {
+				this.executorService.awaitTermination(5000L, TimeUnit.MILLISECONDS);
+			} catch (InterruptedException e) {
+				LOG.debug(e);
+			}
+		}
 
+		this.shutdownComplete = true;
+	}
 	
 	/**
-	 * The states of address detection mechanism.
-	 * There is only a state transition if the current state failed to determine the address.
+	 * Checks whether the task manager has already been shut down.
+	 * 
+	 * @return <code>true</code> if the task manager has already been shut down, <code>false</code> otherwise
 	 */
-	private enum AddressDetectionState {
-		ADDRESS(50), 		//detect own IP based on the JobManagers IP address. Look for common prefix
-		FAST_CONNECT(50),	//try to connect to the JobManager on all Interfaces and all their addresses.
-							//this state uses a low timeout (say 50 ms) for fast detection.
-		SLOW_CONNECT(1000);	//same as FAST_CONNECT, but with a timeout of 1000 ms (1s).
-		
-		
-		private int timeout;
-		AddressDetectionState(int timeout) {
-			this.timeout = timeout;
-		}
-		public int getTimeout() {
-			return timeout;
-		}
+	public boolean isShutDown() {
+		return this.shutdownComplete;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Properties
+	// --------------------------------------------------------------------------------------------
+	
+	public InstanceConnectionInfo getConnectionInfo() {
+		return this.localInstanceConnectionInfo;
+	}
+	
+	public ExecutionMode getExecutionMode() {
+		return this.executionMode;
+	}
+	
+	/**
+	 * Gets the ID under which the TaskManager is currently registered at its JobManager.
+	 * If the TaskManager has not been registered, yet, or if it lost contact, this is is null.
+	 * 
+	 * @return The ID under which the TaskManager is currently registered.
+	 */
+	public InstanceID getRegisteredId() {
+		return this.registeredId;
 	}
 	
 	/**
-	 * Find out the TaskManager's own IP address.
+	 * Checks if the TaskManager is properly registered and ready to receive work.
+	 * 
+	 * @return True, if the TaskManager is registered, false otherwise.
 	 */
-	private InetAddress getTaskManagerAddress(InetSocketAddress jobManagerAddress) throws IOException {
-		AddressDetectionState strategy = AddressDetectionState.ADDRESS;
-
-		while (true) {
-			Enumeration<NetworkInterface> e = NetworkInterface.getNetworkInterfaces();
-			while (e.hasMoreElements()) {
-				NetworkInterface n = e.nextElement();
-				Enumeration<InetAddress> ee = n.getInetAddresses();
-				while (ee.hasMoreElements()) {
-					InetAddress i = ee.nextElement();
-					switch (strategy) {
-					case ADDRESS:
-						if (hasCommonPrefix(jobManagerAddress.getAddress().getAddress(), i.getAddress())) {
-							if (tryToConnect(i, jobManagerAddress, strategy.getTimeout())) {
-								LOG.info("Determined " + i + " as the TaskTracker's own IP address");
-								return i;
-							}
-						}
-						break;
-					case FAST_CONNECT:
-					case SLOW_CONNECT:
-						boolean correct = tryToConnect(i, jobManagerAddress, strategy.getTimeout());
-						if (correct) {
-							LOG.info("Determined " + i + " as the TaskTracker's own IP address");
-							return i;
-						}
-						break;
-					default:
-						throw new RuntimeException("Unkown address detection strategy: " + strategy);
-					}
-				}
-			}
-			// state control
-			switch (strategy) {
-			case ADDRESS:
-				strategy = AddressDetectionState.FAST_CONNECT;
-				break;
-			case FAST_CONNECT:
-				strategy = AddressDetectionState.SLOW_CONNECT;
-				break;
-			case SLOW_CONNECT:
-				throw new RuntimeException("The TaskManager is unable to connect to the JobManager (Address: '"+jobManagerAddress+"').");
-			}
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Defaulting to detection strategy " + strategy);
-			}
-		}
-	}
-	
-	/**
-	 * Checks if two addresses have a common prefix (first 2 bytes).
-	 * Example: 192.168.???.???
-	 * Works also with ipv6, but accepts probably too many addresses
-	 */
-	private static boolean hasCommonPrefix(byte[] address, byte[] address2) {
-		return address[0] == address2[0] && address[1] == address2[1];
-	}
-
-	public static boolean tryToConnect(InetAddress fromAddress, SocketAddress toSocket, int timeout) throws IOException {
-		if (LOG.isDebugEnabled()) {
-			LOG.debug("Trying to connect to JobManager (" + toSocket + ") from local address " + fromAddress
-				+ " with timeout " + timeout);
-		}
-		boolean connectable = true;
-		Socket socket = null;
-		try {
-			socket = new Socket();
-			SocketAddress bindP = new InetSocketAddress(fromAddress, 0); // 0 = let the OS choose the port on this
-																			// machine
-			socket.bind(bindP);
-			socket.connect(toSocket, timeout);
-		} catch (Exception ex) {
-			LOG.info("Failed to connect to JobManager from address '" + fromAddress + "': " + ex.getMessage());
-			if (LOG.isDebugEnabled()) {
-				LOG.debug("Failed with exception", ex);
-			}
-			connectable = false;
-		} finally {
-			if (socket != null) {
-				socket.close();
-			}
-		}
-		return connectable;
-	}
-	
+	public boolean isRegistered() {
+		return this.registeredId != null;
+	}
+	
+	public Map<ExecutionAttemptID, Task> getAllRunningTasks() {
+		return Collections.unmodifiableMap(this.runningTasks);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Task Operation
+	// --------------------------------------------------------------------------------------------
 
 	@Override
-	public TaskCancelResult cancelTask(final ExecutionVertexID id) throws IOException {
+	public TaskOperationResult cancelTask(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId) throws IOException {
 
-		final Task task = this.runningTasks.get(id);
+		final Task task = this.runningTasks.get(executionId);
 
 		if (task == null) {
-			final TaskCancelResult taskCancelResult = new TaskCancelResult(id,
-				AbstractTaskResult.ReturnCode.TASK_NOT_FOUND);
-			taskCancelResult.setDescription("No task with ID " + id + " is currently running");
-			return taskCancelResult;
+			return new TaskOperationResult(vertexId, subtaskIndex, executionId, false, "No task with that execution ID was found.");
 		}
 
 		// Pass call to executor service so IPC thread can return immediately
 		final Runnable r = new Runnable() {
-
 			@Override
 			public void run() {
-
-				// Finally, request user code to cancel
 				task.cancelExecution();
 			}
 		};
-
 		this.executorService.execute(r);
 
-		return new TaskCancelResult(id, AbstractTaskResult.ReturnCode.SUCCESS);
-	}
-
-	@Override
-	public TaskKillResult killTask(final ExecutionVertexID id) throws IOException {
-
-		final Task task = this.runningTasks.get(id);
-
-		if (task == null) {
-			final TaskKillResult taskKillResult = new TaskKillResult(id,
-					AbstractTaskResult.ReturnCode.TASK_NOT_FOUND);
-			taskKillResult.setDescription("No task with ID + " + id + " is currently running");
-			return taskKillResult;
-		}
-
-		// Pass call to executor service so IPC thread can return immediately
-		final Runnable r = new Runnable() {
-
-			@Override
-			public void run() {
-
-				// Finally, request user code to cancel
-				task.killExecution();
-			}
-		};
-
-		this.executorService.execute(r);
-
-		return new TaskKillResult(id, AbstractTaskResult.ReturnCode.SUCCESS);
+		// return success
+		return new TaskOperationResult(vertexId, subtaskIndex, executionId, true);
 	}
 
 
 	@Override
-	public List<TaskSubmissionResult> submitTasks(final List<TaskDeploymentDescriptor> tasks) throws IOException {
-
-		final List<TaskSubmissionResult> submissionResultList = new SerializableArrayList<TaskSubmissionResult>();
-		final List<Task> tasksToStart = new ArrayList<Task>();
-
-		// Make sure all tasks are fully registered before they are started
-		for (final TaskDeploymentDescriptor tdd : tasks) {
-
-			final JobID jobID = tdd.getJobID();
-			final ExecutionVertexID vertexID = tdd.getVertexID();
-			RuntimeEnvironment re;
-
-			// retrieve the registered cache files from job configuration and create the local tmp file.
-			Map<String, FutureTask<Path>> cpTasks = new HashMap<String, FutureTask<Path>>();
-			for (Entry<String, DistributedCacheEntry> e : DistributedCache.readFileInfoFromConfig(tdd.getJobConfiguration())) {
-				FutureTask<Path> cp = this.fileCache.createTmpFile(e.getKey(), e.getValue(), jobID);
-				cpTasks.put(e.getKey(), cp);
+	public TaskOperationResult submitTask(TaskDeploymentDescriptor tdd) {
+		final JobID jobID = tdd.getJobID();
+		final JobVertexID vertexId = tdd.getVertexID();
+		final ExecutionAttemptID executionId = tdd.getExecutionId();
+		final int taskIndex = tdd.getIndexInSubtaskGroup();
+		final int numSubtasks = tdd.getCurrentNumberOfSubtasks();
+		
+		try {
+			final ClassLoader userCodeClassLoader = LibraryCacheManager.getClassLoader(jobID);
+			if (userCodeClassLoader == null) {
+				throw new Exception("No user code ClassLoader available.");
 			}
-
-			try {
-				re = new RuntimeEnvironment(tdd, this.memoryManager, this.ioManager, new TaskInputSplitProvider(jobID,
-					vertexID, this.globalInputSplitProvider), this.accumulatorProtocolProxy, cpTasks);
-			} catch (Throwable t) {
-				final TaskSubmissionResult result = new TaskSubmissionResult(vertexID,
-					AbstractTaskResult.ReturnCode.DEPLOYMENT_ERROR);
-				result.setDescription(StringUtils.stringifyException(t));
-				LOG.error(result.getDescription(), t);
-				submissionResultList.add(result);
-				continue;
+			
+			final Task task = new Task(jobID, vertexId, taskIndex, numSubtasks, executionId, tdd.getTaskName(), this);
+			if (this.runningTasks.putIfAbsent(executionId, task) != null) {
+				throw new Exception("TaskManager contains already a task with executionId " + executionId);
 			}
-
-			final Configuration jobConfiguration = tdd.getJobConfiguration();
-
-			// Register the task
-			Task task;
+			
+			// another try/finally-success block to ensure that the tasks are removed properly in case of an exception
+			boolean success = false;
 			try {
-				task = createAndRegisterTask(vertexID, jobConfiguration, re);
-			} catch (InsufficientResourcesException e) {
-				final TaskSubmissionResult result = new TaskSubmissionResult(vertexID,
-					AbstractTaskResult.ReturnCode.INSUFFICIENT_RESOURCES);
-				result.setDescription(e.getMessage());
-				LOG.error(result.getDescription(), e);
-				submissionResultList.add(result);
-				continue;
-			}
-
-			if (task == null) {
-				final TaskSubmissionResult result = new TaskSubmissionResult(vertexID,
-					AbstractTaskResult.ReturnCode.TASK_NOT_FOUND);
-				result.setDescription("Task " + re.getTaskNameWithIndex() + " (" + vertexID + ") was already running");
-				LOG.error(result.getDescription());
-				submissionResultList.add(result);
-				continue;
-			}
-
-			submissionResultList.add(new TaskSubmissionResult(vertexID, AbstractTaskResult.ReturnCode.SUCCESS));
-			tasksToStart.add(task);
-		}
-
-		// Now start the tasks
-		for (final Task task : tasksToStart) {
-			task.startExecution();
-		}
-
-		return submissionResultList;
-	}
-
-	/**
-	 * Registers an newly incoming runtime task with the task manager.
-	 * 
-	 * @param id
-	 *        the ID of the task to register
-	 * @param jobConfiguration
-	 *        the job configuration that has been attached to the original job graph
-	 * @param environment
-	 *        the environment of the task to be registered
-	 * @return the task to be started or <code>null</code> if a task with the same ID was already running
-	 */
-	private Task createAndRegisterTask(final ExecutionVertexID id, final Configuration jobConfiguration,
-			final RuntimeEnvironment environment)
-					throws InsufficientResourcesException, IOException {
-
-		if (id == null) {
-			throw new IllegalArgumentException("Argument id is null");
-		}
-
-		if (environment == null) {
-			throw new IllegalArgumentException("Argument environment is null");
-		}
-
-		// Task creation and registration must be atomic
-		Task task;
-
-		synchronized (this) {
-			final Task runningTask = this.runningTasks.get(id);
-			boolean registerTask = true;
-			if (runningTask == null) {
-				task = new Task(id, environment, this);
-			} else {
-
-				if (runningTask instanceof Task) {
-					// Task is already running
-					return null;
-				} else {
-					// There is already a replay task running, we will simply restart it
-					task = runningTask;
-					registerTask = false;
-				}
-
-			}
-
-			if (registerTask) {
-				// Register the task with the byte buffered channel manager
+				final InputSplitProvider splitProvider = new TaskInputSplitProvider(this.globalInputSplitProvider, jobID, vertexId);
+				final RuntimeEnvironment env = new RuntimeEnvironment(task, tdd, userCodeClassLoader, this.memoryManager, this.ioManager, splitProvider, this.accumulatorProtocolProxy);
+				task.setEnvironment(env);
+				
+				// register the task with the network stack and profilers
 				this.channelManager.register(task);
-
-				boolean enableProfiling = false;
-				if (this.profiler != null && jobConfiguration.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) {
-					enableProfiling = true;
-				}
-
+				
+				final Configuration jobConfig = tdd.getJobConfiguration();
+	
+				boolean enableProfiling = this.profiler != null && jobConfig.getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true);
+	
 				// Register environment, input, and output gates for profiling
 				if (enableProfiling) {
-					task.registerProfiler(this.profiler, jobConfiguration);
+					task.registerProfiler(this.profiler, jobConfig);
 				}
-
-				this.runningTasks.put(id, task);
-			}
-		}
-		return task;
-	}
-
-	/**
-	 * Unregisters a finished or aborted task.
-	 * 
-	 * @param id
-	 *        the ID of the task to be unregistered
-	 */
-	private void unregisterTask(final ExecutionVertexID id) {
-
-		// Task de-registration must be atomic
-		synchronized (this) {
-
-			final Task task = this.runningTasks.remove(id);
-			if (task == null) {
-				LOG.error("Cannot find task with ID " + id + " to unregister");
-				return;
-			}
-
-			// remove the local tmp file for unregistered tasks.
-			for (Entry<String, DistributedCacheEntry> e: DistributedCache.readFileInfoFromConfig(task.getEnvironment().getJobConfiguration())) {
-				this.fileCache.deleteTmpFile(e.getKey(), e.getValue(), task.getJobID());
-			}
-			// Unregister task from the byte buffered channel manager
-			this.channelManager.unregister(id, task);
-
-			// Unregister task from profiling
-			task.unregisterProfiler(this.profiler);
-
-			// Unregister task from memory manager
-			task.unregisterMemoryManager(this.memoryManager);
-
-			// Unregister task from library cache manager
-			try {
-				LibraryCacheManager.unregister(task.getJobID());
-			} catch (IOException e) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Unregistering the job vertex ID " + id + " caused an IOException");
+				
+				// now that the task is successfully created and registered, we can start copying the
+				// distributed cache temp files
+				Map<String, FutureTask<Path>> cpTasks = new HashMap<String, FutureTask<Path>>();
+				for (Entry<String, DistributedCacheEntry> e : DistributedCache.readFileInfoFromConfig(tdd.getJobConfiguration())) {
+					FutureTask<Path> cp = this.fileCache.createTmpFile(e.getKey(), e.getValue(), jobID);
+					cpTasks.put(e.getKey(), cp);
+				}
+				env.addCopyTasksForCacheFile(cpTasks);
+			
+				if (!task.startExecution()) {
+					throw new Exception("Cannot start task. Task was canceled or failed.");
 				}
+			
+				success = true;
+				return new TaskOperationResult(vertexId, taskIndex, executionId, true);
 			}
-		}
-	}
-
-
-	@Override
-	public LibraryCacheProfileResponse getLibraryCacheProfile(LibraryCacheProfileRequest request) throws IOException {
-
-		LibraryCacheProfileResponse response = new LibraryCacheProfileResponse(request);
-		String[] requiredLibraries = request.getRequiredLibraries();
-
-		for (int i = 0; i < requiredLibraries.length; i++) {
-			if (LibraryCacheManager.contains(requiredLibraries[i]) == null) {
-				response.setCached(i, false);
-			} else {
-				response.setCached(i, true);
+			finally {
+				if (!success) {
+					// remove task 
+					this.runningTasks.remove(executionId);
+					// delete distributed cache files
+					for (Entry<String, DistributedCacheEntry> e : DistributedCache.readFileInfoFromConfig(tdd.getJobConfiguration())) {
+						this.fileCache.deleteTmpFile(e.getKey(), e.getValue(), jobID);
+					}
+				}
 			}
 		}
-
-		return response;
-	}
-
-
-	@Override
-	public void updateLibraryCache(LibraryCacheUpdate update) throws IOException {
-		// Nothing to to here
-	}
-
-	public void executionStateChanged(final JobID jobID, final ExecutionVertexID id,
-			final ExecutionState newExecutionState, final String optionalDescription) {
-
-		// Don't propagate state CANCELING back to the job manager
-		if (newExecutionState == ExecutionState.CANCELING) {
-			return;
-		}
-
-		if (newExecutionState == ExecutionState.FINISHED || newExecutionState == ExecutionState.CANCELED
-				|| newExecutionState == ExecutionState.FAILED) {
-
-			// Unregister the task (free all buffers, remove all channels, task-specific class loaders, etc...)
-			unregisterTask(id);
-		}
-		// Get lock on the jobManager object and propagate the state change
-		synchronized (this.jobManager) {
+		catch (Throwable t) {
+			LOG.error("Could not instantiate task", t);
+			
 			try {
-				this.jobManager.updateTaskExecutionState(new TaskExecutionState(jobID, id, newExecutionState,
-					optionalDescription));
+				LibraryCacheManager.unregister(jobID);
 			} catch (IOException e) {
-				LOG.error(e);
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Unregistering the execution " + executionId + " caused an IOException");
+				}
 			}
+			
+			return new TaskOperationResult(vertexId, taskIndex, executionId, false, ExceptionUtils.stringifyException(t));
 		}
 	}
 
 	/**
-	 * Shuts the task manager down.
+	 * Unregisters a finished or aborted task.
+	 * 
+	 * @param executionId
+	 *        the ID of the task to be unregistered
 	 */
-	public void shutdown() {
+	private void unregisterTask(ExecutionAttemptID executionId) {
 
-		if (!this.shutdownStarted.compareAndSet(false, true)) {
+		// Task de-registration must be atomic
+		final Task task = this.runningTasks.remove(executionId);
+		if (task == null) {
+			LOG.error("Cannot find task with ID " + executionId + " to unregister");
 			return;
 		}
 
-		LOG.info("Shutting down TaskManager");
+		// remove the local tmp file for unregistered tasks.
+		for (Entry<String, DistributedCacheEntry> e: DistributedCache.readFileInfoFromConfig(task.getEnvironment().getJobConfiguration())) {
+			this.fileCache.deleteTmpFile(e.getKey(), e.getValue(), task.getJobID());
+		}
 		
-		// first, stop the heartbeat thread and wait for it to terminate
-		this.heartbeatThread.interrupt();
-		try {
-			this.heartbeatThread.join(1000);
-		} catch (InterruptedException e) {}
-
-		// Stop RPC proxy for the task manager
-		RPC.stopProxy(this.jobManager);
-
-		// Stop RPC proxy for the global input split assigner
-		RPC.stopProxy(this.globalInputSplitProvider);
+		// Unregister task from the byte buffered channel manager
+		this.channelManager.unregister(executionId, task);
 
-		// Stop RPC proxy for the lookup service
-		RPC.stopProxy(this.lookupService);
+		// Unregister task from profiling
+		task.unregisterProfiler(this.profiler);
 
-		// Stop RPC proxy for accumulator reports
-		RPC.stopProxy(this.accumulatorProtocolProxy);
-
-		// Shut down the own RPC server
-		this.taskManagerServer.stop();
+		// Unregister task from memory manager
+		task.unregisterMemoryManager(this.memoryManager);
 
-		// Stop profiling if enabled
-		if (this.profiler != null) {
-			this.profiler.shutdown();
-		}
-
-		// Shut down the channel manager
+		// Unregister task from library cache manager
 		try {
-			this.channelManager.shutdown();
+			LibraryCacheManager.unregister(task.getJobID());
 		} catch (IOException e) {
-			LOG.warn("ChannelManager did not shutdown properly: " + e.getMessage(), e);
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Unregistering the execution " + executionId + " caused an IOException");
+			}
 		}
+	}
 
-		// Shut down the memory manager
-		if (this.ioManager != null) {
-			this.ioManager.shutdown();
+	public void notifyExecutionStateChange(JobID jobID, ExecutionAttemptID executionId, ExecutionState2 newExecutionState, String optionalDescription) {
+		
+		// Get lock on the jobManager object and propagate the state change
+		boolean success = false;
+		try {
+			this.jobManager.updateTaskExecutionState(new TaskExecutionState(jobID, executionId, newExecutionState, optionalDescription));
 		}
-
-		if (this.memoryManager != null) {
-			this.memoryManager.shutdown();
+		catch (Throwable t) {
+			String msg = "Error sending task state update to JobManager.";
+			LOG.error(msg, t);
+			ExceptionUtils.rethrow(t, msg);
 		}
-
-		this.fileCache.shutdown();
-
-		// Shut down the executor service
-		if (this.executorService != null) {
-			this.executorService.shutdown();
-			try {
-				this.executorService.awaitTermination(5000L, TimeUnit.MILLISECONDS);
-			} catch (InterruptedException e) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug(e);
-				}
+		finally {
+			// in case of a failure, or when the tasks is in a finished state, then unregister the
+			// task (free all buffers, remove all channels, task-specific class loaders, etc...)
+			if (!success || newExecutionState == ExecutionState2.FINISHED || newExecutionState == ExecutionState2.CANCELED
+					|| newExecutionState == ExecutionState2.FAILED)
+			{
+				
+				unregisterTask(executionId);
 			}
 		}
-
-		this.shutdownComplete = true;
 	}
 
 	/**
-	 * Checks whether the task manager has already been shut down.
-	 * 
-	 * @return <code>true</code> if the task manager has already been shut down, <code>false</code> otherwise
+	 * Removes all tasks from this TaskManager.
 	 */
-	public boolean isShutDown() {
-		return this.shutdownComplete;
-	}
-
-	@Override
-	public void logBufferUtilization() {
-		this.channelManager.logBufferUtilization();
-	}
-
-	@Override
-	public void killTaskManager() throws IOException {
-		// Kill the entire JVM after a delay of 10ms, so this RPC will finish properly before
-		final Timer timer = new Timer();
-		final TimerTask timerTask = new TimerTask() {
-
-			@Override
-			public void run() {
-				System.exit(0);
-			}
-		};
-
-		timer.schedule(timerTask, 10L);
-	}
-
-
-	@Override
-	public void invalidateLookupCacheEntries(final Set<ChannelID> channelIDs) throws IOException {
-		this.channelManager.invalidateLookupCacheEntries(channelIDs);
-	}
-
 	public void cancelAndClearEverything() {
 		LOG.info("Cancelling all computations and discarding all cached data.");
+		for (Task t : runningTasks.values()) {
+			t.cancelExecution();
+			runningTasks.remove(t.getExecutionId());
+		}
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	//  Properties
+	//  Library caching
 	// --------------------------------------------------------------------------------------------
 	
-	public InstanceConnectionInfo getConnectionInfo() {
-		return this.localInstanceConnectionInfo;
-	}
-	
-	public ExecutionMode getExecutionMode() {
-		return this.executionMode;
-	}
-	
-	/**
-	 * Gets the ID under which the TaskManager is currently registered at its JobManager.
-	 * If the TaskManager has not been registered, yet, or if it lost contact, this is is null.
-	 * 
-	 * @return The ID under which the TaskManager is currently registered.
-	 */
-	public InstanceID getRegisteredId() {
-		return this.registeredId;
+	@Override
+	public LibraryCacheProfileResponse getLibraryCacheProfile(LibraryCacheProfileRequest request) throws IOException {
+
+		LibraryCacheProfileResponse response = new LibraryCacheProfileResponse(request);
+		String[] requiredLibraries = request.getRequiredLibraries();
+
+		for (int i = 0; i < requiredLibraries.length; i++) {
+			if (LibraryCacheManager.contains(requiredLibraries[i]) == null) {
+				response.setCached(i, false);
+			} else {
+				response.setCached(i, true);
+			}
+		}
+
+		return response;
 	}
 	
-	/**
-	 * Checks if the TaskManager is properly registered and ready to receive work.
-	 * 
-	 * @return True, if the TaskManager is registered, false otherwise.
-	 */
-	public boolean isRegistered() {
-		return this.registeredId != null;
+	@Override
+	public void updateLibraryCache(LibraryCacheUpdate update) throws IOException {
+		// Nothing to to here, because the libraries are added to the cache when the
+		// update is deserialized (WE SHOULD CHANGE THAT!!!)
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -1243,7 +872,187 @@ public class TaskManager implements TaskOperationProtocol {
 
 		return str.toString();
 	}
+
+	
+	// --------------------------------------------------------------------------------------------
+	//  Execution & Initialization
+	// --------------------------------------------------------------------------------------------
+	
+	public static TaskManager createTaskManager(ExecutionMode mode) throws Exception {
+		
+		// IMPORTANT! At this point, the GlobalConfiguration must have been read!
+		
+		final InetSocketAddress jobManagerAddress;
+		LOG.info("Reading location of job manager from configuration");
+					
+		final String address = GlobalConfiguration.getString(ConfigConstants.JOB_MANAGER_IPC_ADDRESS_KEY, null);
+		final int port = GlobalConfiguration.getInteger(ConfigConstants.JOB_MANAGER_IPC_PORT_KEY, ConfigConstants.DEFAULT_JOB_MANAGER_IPC_PORT);
+					
+		if (address == null) {
+			throw new Exception("Job manager address not configured in the GlobalConfiguration.");
+		}
+			
+		// Try to convert configured address to {@link InetAddress}
+		try {
+			final InetAddress tmpAddress = InetAddress.getByName(address);
+			jobManagerAddress = new InetSocketAddress(tmpAddress, port);
+		}
+		catch (UnknownHostException e) {
+			LOG.fatal("Could not resolve JobManager host name.");
+			throw new Exception("Could not resolve JobManager host name: " + e.getMessage(), e);
+		}
+		
+		return createTaskManager(mode, jobManagerAddress);
+	}
+	
+	public static TaskManager createTaskManager(ExecutionMode mode, InetSocketAddress jobManagerAddress) throws Exception {
+		// Determine our own public facing address and start the server
+		final InetAddress taskManagerAddress;
+		try {
+			taskManagerAddress = getTaskManagerAddress(jobManagerAddress);
+		}
+		catch (IOException e) {
+			throw new Exception("The TaskManager failed to determine the IP address of the interface that connects to the JobManager.", e);
+		}
+		
+		return createTaskManager(mode, jobManagerAddress, taskManagerAddress);
+	}
+	
+	
+	public static TaskManager createTaskManager(ExecutionMode mode, InetSocketAddress jobManagerAddress, InetAddress taskManagerAddress) throws Exception {
+		
+		// IMPORTANT! At this point, the GlobalConfiguration must have been read!
+		
+		LOG.info("Connecting to JobManager at: " + jobManagerAddress);
+		
+		// Create RPC connections to the JobManager
+		
+		JobManagerProtocol jobManager = null;
+		InputSplitProviderProtocol splitProvider = null;
+		ChannelLookupProtocol channelLookup = null;
+		AccumulatorProtocol accumulators = null;
+		
+		// try/finally block to close proxies if anything goes wrong
+		boolean success = false;
+		try {
+			// create the RPC call proxy to the job manager for jobs
+			try {
+				jobManager = RPC.getProxy(JobManagerProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
+			}
+			catch (IOException e) {
+				LOG.fatal("Could not connect to the JobManager: " + e.getMessage(), e);
+				throw new Exception("Failed to initialize connection to JobManager: " + e.getMessage(), e);
+			}
+			
+			// Try to create local stub of the global input split provider
+			try {
+				splitProvider = RPC.getProxy(InputSplitProviderProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
+			}
+			catch (IOException e) {
+				LOG.fatal(e.getMessage(), e);
+				throw new Exception("Failed to initialize connection to global input split provider: " + e.getMessage(), e);
+			}
+
+			// Try to create local stub for the lookup service
+			try {
+				channelLookup = RPC.getProxy(ChannelLookupProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
+			}
+			catch (IOException e) {
+				LOG.fatal(e.getMessage(), e);
+				throw new Exception("Failed to initialize channel lookup protocol. " + e.getMessage(), e);
+			}
+
+			// Try to create local stub for the accumulators
+			try {
+				accumulators = RPC.getProxy(AccumulatorProtocol.class, jobManagerAddress, NetUtils.getSocketFactory());
+			}
+			catch (IOException e) {
+				LOG.fatal("Failed to initialize accumulator protocol: " + e.getMessage(), e);
+				throw new Exception("Failed to initialize accumulator protocol: " + e.getMessage(), e);
+			}
+			
+			TaskManager tm = new TaskManager(mode, jobManager, splitProvider, channelLookup, accumulators, jobManagerAddress, taskManagerAddress);
+			success = true;
+			return tm;
+		}
+		finally {
+			if (!success) {
+				stopProxy(jobManager);
+				stopProxy(splitProvider);
+				stopProxy(channelLookup);
+				stopProxy(accumulators);
+			}
+		}
+	}
+	
+
+	// --------------------------------------------------------------------------------------------
+	//  Executable
+	// --------------------------------------------------------------------------------------------
 	
+	/**
+	 * Entry point for the TaskManager executable.
+	 * 
+	 * @param args Arguments from the command line
+	 * @throws IOException 
+	 */
+	@SuppressWarnings("static-access")
+	public static void main(String[] args) throws IOException {		
+		Option configDirOpt = OptionBuilder.withArgName("config directory").hasArg().withDescription(
+			"Specify configuration directory.").create("configDir");
+		// tempDir option is used by the YARN client.
+		Option tempDir = OptionBuilder.withArgName("temporary directory (overwrites configured option)")
+				.hasArg().withDescription(
+				"Specify temporary directory.").create(ARG_CONF_DIR);
+		configDirOpt.setRequired(true);
+		tempDir.setRequired(false);
+		Options options = new Options();
+		options.addOption(configDirOpt);
+		options.addOption(tempDir);
+		
+
+		CommandLineParser parser = new GnuParser();
+		CommandLine line = null;
+		try {
+			line = parser.parse(options, args);
+		} catch (ParseException e) {
+			System.err.println("CLI Parsing failed. Reason: " + e.getMessage());
+			System.exit(STARTUP_FAILURE_RETURN_CODE);
+		}
+
+		String configDir = line.getOptionValue(configDirOpt.getOpt(), null);
+		String tempDirVal = line.getOptionValue(tempDir.getOpt(), null);
+
+		// First, try to load global configuration
+		GlobalConfiguration.loadConfiguration(configDir);
+		if(tempDirVal != null // the YARN TM runner has set a value for the temp dir
+				// the configuration does not contain a temp directory
+				&& GlobalConfiguration.getString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, null) == null) {
+			Configuration c = GlobalConfiguration.getConfiguration();
+			c.setString(ConfigConstants.TASK_MANAGER_TMP_DIR_KEY, tempDirVal);
+			LOG.info("Setting temporary directory to "+tempDirVal);
+			GlobalConfiguration.includeConfiguration(c);
+		}
+		
+		// print some startup environment info, like user, code revision, etc
+		EnvironmentInformation.logEnvironmentInfo(LOG, "TaskManager");
+		
+		// Create a new task manager object
+		try {
+			createTaskManager(ExecutionMode.CLUSTER);
+		} catch (Exception e) {
+			LOG.fatal("Taskmanager startup failed: " + e.getMessage(), e);
+			System.exit(STARTUP_FAILURE_RETURN_CODE);
+		}
+		
+		// park the main thread to keep the JVM alive (all other threads may be daemon threads)
+		Object mon = new Object();
+		synchronized (mon) {
+			try {
+				mon.wait();
+			} catch (InterruptedException ex) {}
+		}
+	}
 	
 	// --------------------------------------------------------------------------------------------
 	// Miscellaneous Utilities
@@ -1268,30 +1077,169 @@ public class TaskManager implements TaskOperationProtocol {
 			if (!f.exists()) {
 				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' does not exist.");
 			}
-
 			if (!f.isDirectory()) {
 				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' is not a directory.");
 			}
-
 			if (!f.canWrite()) {
 				throw new Exception("Temporary file directory '" + f.getAbsolutePath() + "' is not writable.");
 			}
 		}
 	}
 	
-	public static class EmergencyShutdownExceptionHandler implements Thread.UncaughtExceptionHandler {
+	/**
+	 * Stops the given RPC protocol proxy, if it is not null.
+	 * This method never throws an exception, it only logs errors.
+	 * 
+	 * @param protocol The protocol proxy to stop.
+	 */
+	private static final void stopProxy(VersionedProtocol protocol) {
+		if (protocol != null) {
+			try {
+				RPC.stopProxy(protocol);
+			}
+			catch (Throwable t) {
+				LOG.error("Error while shutting down RPC proxy.", t);
+			}
+		}
+	}
+	
+	/**
+	 * Determines the IP address of the interface from which the TaskManager can connect to the given JobManager
+	 * IP address.
+	 * 
+	 * @param jobManagerAddress The socket address to connect to.
+	 * @return The IP address of the interface that connects to the JobManager.
+	 * @throws IOException If no connection could be established.
+	 */
+	private static InetAddress getTaskManagerAddress(InetSocketAddress jobManagerAddress) throws IOException {
+		AddressDetectionState strategy = AddressDetectionState.ADDRESS;
 
-		private final TaskManager tm;
-		
-		public EmergencyShutdownExceptionHandler(TaskManager tm) {
-			this.tm = tm;
+		while (true) {
+			Enumeration<NetworkInterface> e = NetworkInterface.getNetworkInterfaces();
+			while (e.hasMoreElements()) {
+				NetworkInterface n = e.nextElement();
+				Enumeration<InetAddress> ee = n.getInetAddresses();
+				while (ee.hasMoreElements()) {
+					InetAddress i = ee.nextElement();
+					switch (strategy) {
+					case ADDRESS:
+						if (hasCommonPrefix(jobManagerAddress.getAddress().getAddress(), i.getAddress())) {
+							if (tryToConnect(i, jobManagerAddress, strategy.getTimeout())) {
+								LOG.info("Determined " + i + " as the TaskTracker's own IP address");
+								return i;
+							}
+						}
+						break;
+					case FAST_CONNECT:
+					case SLOW_CONNECT:
+						boolean correct = tryToConnect(i, jobManagerAddress, strategy.getTimeout());
+						if (correct) {
+							LOG.info("Determined " + i + " as the TaskTracker's own IP address");
+							return i;
+						}
+						break;
+					default:
+						throw new RuntimeException("Unkown address detection strategy: " + strategy);
+					}
+				}
+			}
+			// state control
+			switch (strategy) {
+			case ADDRESS:
+				strategy = AddressDetectionState.FAST_CONNECT;
+				break;
+			case FAST_CONNECT:
+				strategy = AddressDetectionState.SLOW_CONNECT;
+				break;
+			case SLOW_CONNECT:
+				throw new RuntimeException("The TaskManager is unable to connect to the JobManager (Address: '"+jobManagerAddress+"').");
+			}
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Defaulting to detection strategy " + strategy);
+			}
+		}
+	}
+	
+	/**
+	 * Searches for an available free port and returns the port number.
+	 * 
+	 * @return An available port.
+	 * @throws RuntimeException Thrown, if no free port was found.
+	 */
+	private static final int getAvailablePort() {
+		for (int i = 0; i < 50; i++) {
+			ServerSocket serverSocket = null;
+			try {
+				serverSocket = new ServerSocket(0);
+				int port = serverSocket.getLocalPort();
+				if (port != 0) {
+					return port;
+				}
+			} catch (IOException e) {
+				LOG.debug("Unable to allocate port " + e.getMessage(), e);
+			} finally {
+				if (serverSocket != null) {
+					try { serverSocket.close(); } catch (Throwable t) {}
+				}
+			}
 		}
 		
-		@Override
-		public void uncaughtException(Thread t, Throwable e) {
-			LOG.fatal("Thread " + t.getName() + " caused an unrecoverable exception.", e);
-			tm.shutdown();
+		throw new RuntimeException("Could not find a free permitted port on the machine.");
+	}
+	
+	/**
+	 * Checks if two addresses have a common prefix (first 2 bytes).
+	 * Example: 192.168.???.???
+	 * Works also with ipv6, but accepts probably too many addresses
+	 */
+	private static boolean hasCommonPrefix(byte[] address, byte[] address2) {
+		return address[0] == address2[0] && address[1] == address2[1];
+	}
+
+	private static boolean tryToConnect(InetAddress fromAddress, SocketAddress toSocket, int timeout) throws IOException {
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Trying to connect to JobManager (" + toSocket + ") from local address " + fromAddress
+				+ " with timeout " + timeout);
+		}
+		boolean connectable = true;
+		Socket socket = null;
+		try {
+			socket = new Socket();
+			SocketAddress bindP = new InetSocketAddress(fromAddress, 0); // 0 = let the OS choose the port on this
+																			// machine
+			socket.bind(bindP);
+			socket.connect(toSocket, timeout);
+		} catch (Exception ex) {
+			LOG.info("Failed to connect to JobManager from address '" + fromAddress + "': " + ex.getMessage());
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Failed with exception", ex);
+			}
+			connectable = false;
+		} finally {
+			if (socket != null) {
+				socket.close();
+			}
 		}
+		return connectable;
+	}
+	
+	/**
+	 * The states of address detection mechanism.
+	 * There is only a state transition if the current state failed to determine the address.
+	 */
+	private enum AddressDetectionState {
+		ADDRESS(50), 		//detect own IP based on the JobManagers IP address. Look for common prefix
+		FAST_CONNECT(50),	//try to connect to the JobManager on all Interfaces and all their addresses.
+							//this state uses a low timeout (say 50 ms) for fast detection.
+		SLOW_CONNECT(1000);	//same as FAST_CONNECT, but with a timeout of 1000 ms (1s).
 		
+		
+		private int timeout;
+		AddressDetectionState(int timeout) {
+			this.timeout = timeout;
+		}
+		public int getTimeout() {
+			return timeout;
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java
new file mode 100644
index 0000000..f0f00a7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskOperationResult.java
@@ -0,0 +1,110 @@
+/**
+ * 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.taskmanager;
+
+import java.io.IOException;
+
+import org.apache.flink.core.io.IOReadableWritable;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.types.StringValue;
+
+import com.google.common.base.Preconditions;
+
+
+public class TaskOperationResult implements IOReadableWritable {
+
+	private JobVertexID vertexId;
+	
+	private int subtaskIndex;
+	
+	private ExecutionAttemptID executionId;
+	
+	private boolean success;
+	
+	private String description;
+
+
+	public TaskOperationResult() {
+		this(new JobVertexID(), -1, new ExecutionAttemptID(), false);
+	}
+	
+	public TaskOperationResult(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, boolean success) {
+		this(vertexId, subtaskIndex, executionId, success, null);
+	}
+	
+	public TaskOperationResult(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, boolean success, String description) {
+		Preconditions.checkNotNull(vertexId);
+		Preconditions.checkNotNull(executionId);
+		
+		this.vertexId = vertexId;
+		this.subtaskIndex = subtaskIndex;
+		this.executionId = executionId;
+		this.success = success;
+		this.description = description;
+	}
+
+
+	public JobVertexID getVertexId() {
+		return vertexId;
+	}
+	
+	public int getSubtaskIndex() {
+		return subtaskIndex;
+	}
+	
+	public ExecutionAttemptID getExecutionId() {
+		return executionId;
+	}
+	
+	public boolean isSuccess() {
+		return success;
+	}
+
+	public String getDescription() {
+		return description;
+	}
+
+	@Override
+	public void read(DataInputView in) throws IOException {
+		this.vertexId.read(in);
+		this.subtaskIndex = in.readInt();
+		this.success = in.readBoolean();
+		
+		if (in.readBoolean()) {
+			this.description = StringValue.readString(in);
+		}
+	}
+
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		this.vertexId.write(out);
+		out.writeInt(subtaskIndex);
+		out.writeBoolean(success);
+		
+		if (description != null) {
+			out.writeBoolean(true);
+			StringValue.writeString(description, out);
+		} else {
+			out.writeBoolean(false);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskSubmissionResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskSubmissionResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskSubmissionResult.java
deleted file mode 100644
index ae88f3b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskSubmissionResult.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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.taskmanager;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-
-/**
- * A <code>TaskSubmissionResult</code> is used to report the results
- * of a task submission. It contains the ID of the submitted task, a return code and
- * a description. In case of a submission error the description includes an error message.
- * 
- */
-public class TaskSubmissionResult extends AbstractTaskResult {
-
-	/**
-	 * Constructs a new task submission result.
-	 * 
-	 * @param vertexID
-	 *        the task ID this result belongs to
-	 * @param returnCode
-	 *        the return code of the submission
-	 */
-	public TaskSubmissionResult(ExecutionVertexID vertexID, ReturnCode returnCode) {
-		super(vertexID, returnCode);
-	}
-
-	/**
-	 * Constructs an empty task submission result.
-	 */
-	public TaskSubmissionResult() {
-		super();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnumUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnumUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnumUtils.java
index 9694f63..312c9ac 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnumUtils.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnumUtils.java
@@ -16,18 +16,16 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.util;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import org.apache.flink.core.io.StringRecord;
+import org.apache.flink.types.StringValue;
 
 /**
  * Auxiliary class to (de)serialize enumeration values.
- * 
  */
 public final class EnumUtils {
 
@@ -56,7 +54,7 @@ public final class EnumUtils {
 			return null;
 		}
 
-		return T.valueOf(enumType, StringRecord.readString(in));
+		return T.valueOf(enumType, StringValue.readString(in));
 	}
 
 	/**
@@ -75,7 +73,7 @@ public final class EnumUtils {
 			out.writeBoolean(false);
 		} else {
 			out.writeBoolean(true);
-			StringRecord.writeString(out, enumVal.name());
+			StringValue.writeString(enumVal.name(), out);
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
index 964a754..fa7ad1c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
@@ -19,20 +19,46 @@
 package org.apache.flink.runtime.util;
 
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 public class ExecutorThreadFactory implements ThreadFactory {
 	
+	private static final Log LOG = LogFactory.getLog(ExecutorThreadFactory.class);
+	
+	
+	private static final String THREAD_NAME_PREFIX = "Flink Executor Thread - ";
+	
+	private static final AtomicInteger COUNTER = new AtomicInteger(1);
+	
+	private static final ThreadGroup THREAD_GROUP = new ThreadGroup("Flink Executor Threads");
+	
+	private static final Thread.UncaughtExceptionHandler EXCEPTION_HANDLER = new LoggingExceptionHander();
+	
+	
 	public static final ExecutorThreadFactory INSTANCE = new ExecutorThreadFactory();
-
-	private static final String THREAD_NAME = "Flink Executor Thread";
 	
+	// --------------------------------------------------------------------------------------------
 	
 	private ExecutorThreadFactory() {}
 	
 	
 	public Thread newThread(Runnable target) {
-		Thread t = new Thread(target, THREAD_NAME);
+		Thread t = new Thread(THREAD_GROUP, target, THREAD_NAME_PREFIX + COUNTER.getAndIncrement());
 		t.setDaemon(true);
+		t.setUncaughtExceptionHandler(EXCEPTION_HANDLER);
 		return t;
 	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	private static final class LoggingExceptionHander implements Thread.UncaughtExceptionHandler {
+
+		@Override
+		public void uncaughtException(Thread t, Throwable e) {
+			LOG.error("Thread '" + t.getName() + "' produced an uncaught exception.", e);
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableArrayList.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableArrayList.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableArrayList.java
index d4db568..5bb9451 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableArrayList.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/SerializableArrayList.java
@@ -110,45 +110,11 @@ public class SerializableArrayList<E extends IOReadableWritable> extends ArrayLi
 
 
 	@Override
-	public boolean equals(final Object obj) {
-
+	public boolean equals(Object obj) {
 		if (!(obj instanceof SerializableArrayList<?>)) {
 			return false;
 		}
 
-		final SerializableArrayList<?> sal = (SerializableArrayList<?>) obj;
-
-		if (this.size() != sal.size()) {
-			return false;
-		}
-
-		final Iterator<E> it = iterator();
-		final Iterator<?> it2 = sal.iterator();
-		while (it.hasNext()) {
-
-			final E e = it.next();
-			final Object obj2 = it2.next();
-			if (!e.equals(obj2)) {
-				return false;
-			}
-		}
-
-		return true;
-	}
-
-
-	@Override
-	public int hashCode() {
-
-		int hashCode = Integer.MIN_VALUE;
-
-		if (!isEmpty()) {
-			final E e = get(0);
-			hashCode += Math.abs(e.getClass().hashCode());
-		}
-
-		hashCode += size();
-
-		return hashCode;
+		return (obj instanceof SerializableArrayList) && super.equals(obj);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
index bc3173f..9ae6e2b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/client/JobResultTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.client;
 
 import static org.junit.Assert.assertEquals;
@@ -36,7 +35,6 @@ import org.junit.Test;
 
 /**
  * This class contains test concerning all classes which are derived from {@link AbstractJobResult}.
- * 
  */
 public class JobResultTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptorTest.java
index d0b1504..ffbcf41 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptorTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.deployment;
 
 import static org.junit.Assert.assertEquals;
@@ -30,10 +29,7 @@ import org.apache.flink.runtime.testutils.ServerTestUtils;
 import org.apache.flink.util.StringUtils;
 import org.junit.Test;
 
-/**
- * This class contains unit tests for the {@link ChannelDeploymentDescriptor} class.
- * 
- */
+
 public class ChannelDeploymentDescriptorTest {
 
 	/**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptorTest.java
deleted file mode 100644
index 68d285a..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptorTest.java
+++ /dev/null
@@ -1,138 +0,0 @@
-/**
- * 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.deployment;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.flink.runtime.io.network.channels.ChannelID;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-import org.apache.flink.runtime.io.network.gates.GateID;
-import org.apache.flink.runtime.testutils.ServerTestUtils;
-import org.apache.flink.util.StringUtils;
-import org.junit.Test;
-
-/**
- * This class contains unit tests for the {@link GateDeploymentDescriptor} class.
- * 
- */
-public class GateDeploymentDescriptorTest {
-
-	/**
-	 * Tests the constructor of the {@link GateDeploymentDescriptor} class with valid arguments.
-	 */
-	@Test
-	public void testConstructorWithValidArguments() {
-
-		final GateID gateID = new GateID();
-		final ChannelType channelType = ChannelType.IN_MEMORY;
-		final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
-
-		final GateDeploymentDescriptor gdd = new GateDeploymentDescriptor(gateID, channelType, channels);
-
-		assertEquals(gateID, gdd.getGateID());
-		assertEquals(channelType, gdd.getChannelType());
-		assertEquals(channels.size(), gdd.getNumberOfChannelDescriptors());
-	}
-
-	/**
-	 * Tests the constructor of the {@link GateDeploymentDescriptor} class with valid arguments.
-	 */
-	@Test
-	public void testConstructorWithInvalidArguments() {
-
-		final GateID gateID = new GateID();
-		final ChannelType channelType = ChannelType.IN_MEMORY;
-		final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
-
-		boolean firstExceptionCaught = false;
-		boolean secondExceptionCaught = false;
-		boolean thirdExceptionCaught = false;
-
-		try {
-			new GateDeploymentDescriptor(null, channelType, channels);
-		} catch (IllegalArgumentException e) {
-			firstExceptionCaught = true;
-		}
-
-		try {
-			new GateDeploymentDescriptor(gateID, null, channels);
-		} catch (IllegalArgumentException e) {
-			secondExceptionCaught = true;
-		}
-
-		try {
-			new GateDeploymentDescriptor(gateID, channelType, null);
-		} catch (IllegalArgumentException e) {
-			thirdExceptionCaught = true;
-		}
-
-		if (!firstExceptionCaught) {
-			fail("First argument was illegal but not detected");
-		}
-
-		if (!secondExceptionCaught) {
-			fail("Second argument was illegal but not detected");
-		}
-
-
-		if (!thirdExceptionCaught) {
-			fail("Third argument was illegal but not detected");
-		}
-	}
-
-	/**
-	 * Tests the serialization/deserialization of the {@link GateDeploymentDescriptor} class.
-	 */
-	@Test
-	public void testSerialization() {
-
-		final GateID gateID = new GateID();
-		final ChannelType channelType = ChannelType.IN_MEMORY;
-		final List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(0);
-		final ChannelDeploymentDescriptor cdd = new ChannelDeploymentDescriptor(new ChannelID(), new ChannelID());
-		channels.add(cdd);
-
-		final GateDeploymentDescriptor orig = new GateDeploymentDescriptor(gateID, channelType,
-			channels);
-
-		GateDeploymentDescriptor copy = null;
-
-		try {
-			copy = ServerTestUtils.createCopy(orig);
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
-		}
-
-		assertFalse(orig.getGateID() == copy.getGateID());
-
-		assertEquals(orig.getGateID(), copy.getGateID());
-		assertEquals(orig.getChannelType(), copy.getChannelType());
-		assertEquals(orig.getNumberOfChannelDescriptors(), copy.getNumberOfChannelDescriptors());
-		assertEquals(orig.getChannelDescriptor(0).getOutputChannelID(), copy.getChannelDescriptor(0)
-			.getOutputChannelID());
-		assertEquals(orig.getChannelDescriptor(0).getInputChannelID(), copy.getChannelDescriptor(0).getInputChannelID());
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
index 3a1aa40..1f66f93 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/deployment/TaskDeploymentDescriptorTest.java
@@ -16,275 +16,64 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.deployment;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.operators.RegularPactTask;
-import org.apache.flink.runtime.testutils.ServerTestUtils;
-import org.apache.flink.runtime.util.SerializableArrayList;
-import org.apache.flink.util.StringUtils;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
 import org.junit.Test;
 
-/**
- * This class contains unit tests for the {@link TaskDeploymentDescriptor} class.
- * 
- */
 public class TaskDeploymentDescriptorTest {
-	/**
-	 * Tests the constructor of the {@link TaskDeploymentDescriptor} class with valid arguments.
-	 */
-	@Test
-	public void testConstructorWithValidArguments() {
-
-		final JobID jobID = new JobID();
-		final ExecutionVertexID vertexID = new ExecutionVertexID();
-		final String taskName = "task name";
-		final int indexInSubtaskGroup = 0;
-		final int currentNumberOfSubtasks = 1;
-		final Configuration jobConfiguration = new Configuration();
-		final Configuration taskConfiguration = new Configuration();
-		final Class<? extends AbstractInvokable> invokableClass =  RegularPactTask.class;
-		final SerializableArrayList<GateDeploymentDescriptor> outputGates = new SerializableArrayList<GateDeploymentDescriptor>(
-			0);
-		final SerializableArrayList<GateDeploymentDescriptor> inputGates = new SerializableArrayList<GateDeploymentDescriptor>(
-			0);
-
-		final TaskDeploymentDescriptor tdd = new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-			indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-			invokableClass, outputGates, inputGates);
 
-		assertEquals(jobID, tdd.getJobID());
-		assertEquals(vertexID, tdd.getVertexID());
-		assertEquals(taskName, tdd.getTaskName());
-		assertEquals(indexInSubtaskGroup, tdd.getIndexInSubtaskGroup());
-		assertEquals(currentNumberOfSubtasks, tdd.getCurrentNumberOfSubtasks());
-		assertEquals(jobConfiguration, tdd.getJobConfiguration());
-		assertEquals(taskConfiguration, tdd.getTaskConfiguration());
-		assertEquals(invokableClass, tdd.getInvokableClass());
-		assertEquals(outputGates.size(), tdd.getNumberOfOutputGateDescriptors());
-		assertEquals(inputGates.size(), tdd.getNumberOfInputGateDescriptors());
-	}
-
-	/**
-	 * Tests the constructor of the {@link GateDeploymentDescriptor} class with valid arguments.
-	 */
-	@Test
-	public void testConstructorWithInvalidArguments() {
-
-		final JobID jobID = new JobID();
-		final ExecutionVertexID vertexID = new ExecutionVertexID();
-		final String taskName = "task name";
-		final int indexInSubtaskGroup = 0;
-		final int currentNumberOfSubtasks = 1;
-		final Configuration jobConfiguration = new Configuration();
-		final Configuration taskConfiguration = new Configuration();
-		final Class<? extends AbstractInvokable> invokableClass = RegularPactTask.class;
-		final SerializableArrayList<GateDeploymentDescriptor> outputGates = new SerializableArrayList<GateDeploymentDescriptor>(
-			0);
-		final SerializableArrayList<GateDeploymentDescriptor> inputGates = new SerializableArrayList<GateDeploymentDescriptor>(
-			0);
-
-		boolean firstExceptionCaught = false;
-		boolean secondExceptionCaught = false;
-		boolean thirdExceptionCaught = false;
-		boolean forthExceptionCaught = false;
-		boolean fifthExceptionCaught = false;
-		boolean sixthExceptionCaught = false;
-		boolean seventhExceptionCaught = false;
-		boolean eighthExceptionCaught = false;
-		boolean ninethExeceptionCaught = false;
-		boolean tenthExceptionCaught = false;
-
-		try {
-			new TaskDeploymentDescriptor(null, vertexID, taskName,
-				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-				invokableClass, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			firstExceptionCaught = true;
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, null, taskName,
-				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-				invokableClass, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			secondExceptionCaught = true;
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, null,
-				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-				invokableClass, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			thirdExceptionCaught = true;
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-				-1, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-				invokableClass, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			forthExceptionCaught = true;
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-				indexInSubtaskGroup, -1, jobConfiguration, taskConfiguration,
-				invokableClass, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			fifthExceptionCaught = true;
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-				indexInSubtaskGroup, currentNumberOfSubtasks, null, taskConfiguration,
-				invokableClass, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			sixthExceptionCaught = true;
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, null,
-				invokableClass, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			seventhExceptionCaught = true;
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-				null, outputGates, inputGates);
-		} catch (IllegalArgumentException e) {
-			eighthExceptionCaught = true;
-			
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-				invokableClass, null, inputGates);
-		} catch (IllegalArgumentException e) {
-			ninethExeceptionCaught = true;
-			
-		}
-
-		try {
-			new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-				invokableClass, outputGates, null);
-		} catch (IllegalArgumentException e) {
-			tenthExceptionCaught = true;
-		}
-
-		if (!firstExceptionCaught) {
-			fail("First argument was illegal but not detected");
-		}
-
-		if (!secondExceptionCaught) {
-			fail("Second argument was illegal but not detected");
-		}
-
-		if (!thirdExceptionCaught) {
-			fail("Third argument was illegal but not detected");
-		}
-
-		if (!forthExceptionCaught) {
-			fail("Forth argument was illegal but not detected");
-		}
-
-		if (!fifthExceptionCaught) {
-			fail("Fifth argument was illegal but not detected");
-		}
-
-		if (!sixthExceptionCaught) {
-			fail("Sixth argument was illegal but not detected");
-		}
-
-		if (!seventhExceptionCaught) {
-			fail("Seventh argument was illegal but not detected");
-		}
-
-		if (!eighthExceptionCaught) {
-			fail("Eighth argument was illegal but not detected");
-		}
-
-		if (!ninethExeceptionCaught) {
-			fail("Nineth argument was illegal but not detected");
-		}
-
-		if (!tenthExceptionCaught) {
-			fail("Tenth argument was illegal but not detected");
-		}
-
-	}
-
-	/**
-	 * Tests the serialization/deserialization of the {@link TaskDeploymentDescriptor} class.
-	 */
 	@Test
 	public void testSerialization() {
-
-		final JobID jobID = new JobID();
-		final ExecutionVertexID vertexID = new ExecutionVertexID();
-		final String taskName = "task name";
-		final int indexInSubtaskGroup = 0;
-		final int currentNumberOfSubtasks = 1;
-		final Configuration jobConfiguration = new Configuration();
-		final Configuration taskConfiguration = new Configuration();
-		final Class<? extends AbstractInvokable> invokableClass = RegularPactTask.class;
-		final SerializableArrayList<GateDeploymentDescriptor> outputGates = new SerializableArrayList<GateDeploymentDescriptor>(
-			0);
-		final SerializableArrayList<GateDeploymentDescriptor> inputGates = new SerializableArrayList<GateDeploymentDescriptor>(
-			0);
-
-		final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, taskName,
-			indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
-			invokableClass, outputGates, inputGates);
-
-		TaskDeploymentDescriptor copy = null;
-
-		try {
-			LibraryCacheManager.register(jobID, new String[] {});
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
-		}
-
-		try {
-			copy = ServerTestUtils.createCopy(orig);
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
-		}
-
-		assertFalse(orig.getJobID() == copy.getJobID());
-		assertFalse(orig.getVertexID() == copy.getVertexID());
-		assertFalse(orig.getTaskName() == copy.getTaskName());
-		assertFalse(orig.getJobConfiguration() == copy.getJobConfiguration());
-		assertFalse(orig.getTaskConfiguration() == copy.getTaskConfiguration());
-
-		assertEquals(orig.getJobID(), copy.getJobID());
-		assertEquals(orig.getVertexID(), copy.getVertexID());
-		assertEquals(orig.getTaskName(), copy.getTaskName());
-		assertEquals(orig.getIndexInSubtaskGroup(), copy.getIndexInSubtaskGroup());
-		assertEquals(orig.getCurrentNumberOfSubtasks(), copy.getCurrentNumberOfSubtasks());
-		assertEquals(orig.getNumberOfOutputGateDescriptors(), copy.getNumberOfOutputGateDescriptors());
-		assertEquals(orig.getNumberOfInputGateDescriptors(), copy.getNumberOfInputGateDescriptors());
-
 		try {
-			LibraryCacheManager.register(jobID, new String[] {});
-		} catch (IOException ioe) {
-			fail(StringUtils.stringifyException(ioe));
+			final JobID jobID = new JobID();
+			final JobVertexID vertexID = new JobVertexID();
+			final ExecutionAttemptID execId = new ExecutionAttemptID();
+			final String taskName = "task name";
+			final int indexInSubtaskGroup = 0;
+			final int currentNumberOfSubtasks = 1;
+			final Configuration jobConfiguration = new Configuration();
+			final Configuration taskConfiguration = new Configuration();
+			final Class<? extends AbstractInvokable> invokableClass = RegularPactTask.class;
+			final List<GateDeploymentDescriptor> outputGates = new ArrayList<GateDeploymentDescriptor>(0);
+			final List<GateDeploymentDescriptor> inputGates = new ArrayList<GateDeploymentDescriptor>(0);
+	
+			final TaskDeploymentDescriptor orig = new TaskDeploymentDescriptor(jobID, vertexID, execId, taskName,
+				indexInSubtaskGroup, currentNumberOfSubtasks, jobConfiguration, taskConfiguration,
+				invokableClass.getName(), outputGates, inputGates, new String[] { "jar1", "jar2" }, 47);
+	
+			final TaskDeploymentDescriptor copy = CommonTestUtils.createCopyWritable(orig);
+	
+			assertFalse(orig.getJobID() == copy.getJobID());
+			assertFalse(orig.getVertexID() == copy.getVertexID());
+			assertFalse(orig.getTaskName() == copy.getTaskName());
+			assertFalse(orig.getJobConfiguration() == copy.getJobConfiguration());
+			assertFalse(orig.getTaskConfiguration() == copy.getTaskConfiguration());
+	
+			assertEquals(orig.getJobID(), copy.getJobID());
+			assertEquals(orig.getVertexID(), copy.getVertexID());
+			assertEquals(orig.getTaskName(), copy.getTaskName());
+			assertEquals(orig.getIndexInSubtaskGroup(), copy.getIndexInSubtaskGroup());
+			assertEquals(orig.getCurrentNumberOfSubtasks(), copy.getCurrentNumberOfSubtasks());
+			assertEquals(orig.getOutputGates(), copy.getOutputGates());
+			assertEquals(orig.getInputGates(), copy.getInputGates());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
 		}
 	}
 }


[08/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerUtils.java
deleted file mode 100644
index bb8bee7..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManagerUtils.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * 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.jobmanager;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.runtime.ExecutionMode;
-import org.apache.flink.runtime.instance.InstanceManager;
-import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
-import org.apache.flink.util.StringUtils;
-
-/**
- * This class provides static utility methods for the {@link JobManager}.
- * 
- */
-public class JobManagerUtils {
-
-	/**
-	 * The logging object used by the utility methods.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(JobManagerUtils.class);
-
-	/**
-	 * Private constructor.
-	 */
-	private JobManagerUtils() {
-	}
-
-	/**
-	 * Tries to locate a class with given name and to
-	 * instantiate a {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler} object from it.
-	 * 
-	 * @param schedulerClassName
-	 *        the name of the class to instantiate the scheduler object from
-	 * @param deploymentManager
-	 *        the deployment manager which shall be passed on to the scheduler
-	 * @param instanceManager
-	 *        the instance manager which shall be passed on to the scheduler
-	 * @return the {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler} object instantiated from the class with the provided name
-	 */
-	@SuppressWarnings("unchecked")
-	static DefaultScheduler loadScheduler(final String schedulerClassName, final DeploymentManager deploymentManager,
-			final InstanceManager instanceManager) {
-
-		Class<? extends DefaultScheduler> schedulerClass;
-		try {
-			schedulerClass = (Class<? extends DefaultScheduler>) Class.forName(schedulerClassName);
-		} catch (ClassNotFoundException e) {
-			LOG.error("Cannot find class " + schedulerClassName + ": " + StringUtils.stringifyException(e));
-			return null;
-		}
-
-		Constructor<? extends DefaultScheduler> constructor;
-
-		try {
-
-			Class<?>[] constructorArgs = { DeploymentManager.class, InstanceManager.class };
-			constructor = schedulerClass.getConstructor(constructorArgs);
-		} catch (NoSuchMethodException e) {
-			LOG.error("Cannot create scheduler: " + StringUtils.stringifyException(e));
-			return null;
-		} catch (SecurityException e) {
-			LOG.error("Cannot create scheduler: " + StringUtils.stringifyException(e));
-			return null;
-		}
-
-		DefaultScheduler scheduler;
-
-		try {
-			scheduler = constructor.newInstance(deploymentManager, instanceManager);
-		} catch (InstantiationException e) {
-			LOG.error("Cannot create scheduler: " + StringUtils.stringifyException(e));
-			return null;
-		} catch (IllegalAccessException e) {
-			LOG.error("Cannot create scheduler: " + StringUtils.stringifyException(e));
-			return null;
-		} catch (IllegalArgumentException e) {
-			LOG.error("Cannot create scheduler: " + StringUtils.stringifyException(e));
-			return null;
-		} catch (InvocationTargetException e) {
-			LOG.error("Cannot create scheduler: " + StringUtils.stringifyException(e));
-			return null;
-		}
-
-		return scheduler;
-	}
-
-	/**
-	 * Tries to read the class name of the {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler} implementation from the global configuration which
-	 * is set to be used for the provided execution mode.
-	 * 
-	 * @param executionMode The Nephele execution mode.
-	 * @return the class name of the {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler} implementation to be used or <code>null</code> if no
-	 *         implementation is configured for the given execution mode
-	 */
-	static String getSchedulerClassName(ExecutionMode executionMode) {
-		return "org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler";
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
index 3cae48b..06f0eab 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
@@ -184,13 +184,7 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 								slot = sharingUnit.getTaskAssignment().addSlotWithTask(slot, task.getJobVertexId());
 							}
 							
-							// try to run the task 
-							if (slot.runTask(task.getTaskVertex())) {
-								return slot;
-							} else {
-								// did not assign, so we recycle the resource
-								slot.releaseSlot();
-							}
+							return slot;
 						}
 					}
 					catch (InstanceDiedException e) {
@@ -279,12 +273,18 @@ public class DefaultScheduler implements InstanceListener, SlotAvailablilityList
 				
 				try {
 					AllocatedSlot newSlot = instance.allocateSlot(task.getTaskVertex().getJobId());
-					if (newSlot != null && newSlot.runTask(task.getTaskVertex())) {
+					if (newSlot != null) {
 						
 						// success, remove from the task queue and notify the future
 						taskQueue.poll();
 						if (queued.getFuture() != null) {
-							queued.getFuture().setSlot(newSlot);
+							try {
+								queued.getFuture().setSlot(newSlot);
+							}
+							catch (Throwable t) {
+								LOG.error("Error calling allocation future for task " + task.getTaskVertex().getSimpleName(), t);
+								task.getTaskVertex().fail(t);
+							}
 						}
 					}
 				}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java
index d8bb852..7dfdc95 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java
@@ -1,17 +1,20 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.jobmanager.scheduler;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
index 2b76d26..0fa866a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
@@ -1,26 +1,31 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.jobmanager.scheduler;
 
-public class NoResourceAvailableException extends Exception {
+import org.apache.flink.runtime.JobException;
+
+public class NoResourceAvailableException extends JobException {
 
 	private static final long serialVersionUID = -2249953165298717803L;
 
 	public NoResourceAvailableException() {
-		super();
+		super("Not enough resources available for the Job.");
 	}
 	
 	public NoResourceAvailableException(ScheduledUnit unit) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java
index bad3b88..c4fd2d0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java
@@ -21,7 +21,8 @@ package org.apache.flink.runtime.jobmanager.scheduler;
 import org.apache.flink.runtime.instance.Instance;
 
 /**
- * A SlotRecycler handles allocated slots that have been released.
+ * A SlotAvailablilityListener can be notified when new {@link org.apache.flink.runtime.instance.AllocatedSlot}s become available
+ * on an {@link org.apache.flink.runtime.instance.Instance}.
  */
 public interface SlotAvailablilityListener {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
index 9f008ef..b8b2ed1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.jobmanager.scheduler;
 
+import java.util.Collections;
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -26,7 +27,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 /**
  * A slot sharing units defines which different task (from different job vertices) can be
  * deployed together within a slot. This is a soft permission, in contrast to the hard constraint
- * defined by the {@link CoLocationHint}.
+ * defined by a co-location hint.
  */
 public class SlotSharingGroup {
 	
@@ -44,12 +45,16 @@ public class SlotSharingGroup {
 	}
 	
 	
-	public void addVertexGroup(JobVertexID id) {
+	public void addVertexToGroup(JobVertexID id) {
 		this.ids.add(id);
 	}
 	
+	public void removeVertexFromGroup(JobVertexID id) {
+		this.ids.remove(id);
+	}
+	
 	public Set<JobVertexID> getJobVertexIds() {
-		return ids;
+		return Collections.unmodifiableSet(ids);
 	}
 	
 	
@@ -65,6 +70,6 @@ public class SlotSharingGroup {
 	
 	@Override
 	public String toString() {
-		return this.ids.toString();
+		return "SlotSharingGroup " + this.ids.toString();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java
deleted file mode 100644
index 44dae64..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultInputSplitAssigner.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.io.InputSplitAssigner;
-
-/**
- * This is the default implementation of the {@link InputSplitAssigner} interface. The default input split assigner
- * simply returns all input splits of an input vertex in the order they were originally computed.
- */
-public class DefaultInputSplitAssigner implements InputSplitAssigner {
-
-	/** The logging object used to report information and errors. */
-	private static final Logger LOG = LoggerFactory.getLogger(DefaultInputSplitAssigner.class);
-
-	/** The list of all splits */
-	private final List<InputSplit> splits = new ArrayList<InputSplit>();
-
-
-	public DefaultInputSplitAssigner(InputSplit[] splits) {
-		Collections.addAll(this.splits, splits);
-	}
-	
-	public DefaultInputSplitAssigner(Collection<? extends InputSplit> splits) {
-		this.splits.addAll(splits);
-	}
-	
-	
-	@Override
-	public InputSplit getNextInputSplit(String host) {
-		InputSplit next = null;
-		
-		// keep the synchronized part short
-		synchronized (this.splits) {
-			if (this.splits.size() > 0) {
-				next = this.splits.remove(this.splits.size() - 1);
-			}
-		}
-		
-		if (LOG.isDebugEnabled()) {
-			if (next == null) {
-				LOG.debug("Assigning split " + next + " to " + host);
-			} else {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("No more input splits available");
-				}
-			}
-		}
-		return next;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java
deleted file mode 100644
index 958fa4c..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/InputSplitWrapper.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import java.io.IOException;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.jobgraph.JobID;
-
-/**
- * An input split wrapper object wraps an input split for RPC calls. In particular, the input split wrapper ensures that
- * the right class loader is used to instantiate the wrapped input split object.
- */
-public final class InputSplitWrapper implements IOReadableWritable {
-
-	/**
-	 * The ID of the job this input split belongs to.
-	 */
-	private JobID jobID;
-
-	/**
-	 * The wrapped input split.
-	 */
-	private InputSplit inputSplit;
-
-	/**
-	 * Constructs a new input split wrapper.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the input split belongs to
-	 * @param inputSplit
-	 *        the input split to be wrapped
-	 */
-	public InputSplitWrapper(JobID jobID, InputSplit inputSplit) {
-		if (jobID == null) {
-			throw new NullPointerException();
-		}
-
-		this.jobID = jobID;
-		this.inputSplit = inputSplit;
-	}
-
-	/**
-	 * Default constructor for serialization/deserialization.
-	 */
-	public InputSplitWrapper() {
-		this.jobID = new JobID();
-	}
-
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-
-		// Write the job ID
-		this.jobID.write(out);
-
-		if (this.inputSplit == null) {
-			out.writeBoolean(false);
-		} else {
-
-			out.writeBoolean(true);
-
-			// Write the name of the class
-			StringRecord.writeString(out, this.inputSplit.getClass().getName());
-
-			// Write out the input split itself
-			this.inputSplit.write(out);
-		}
-	}
-
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public void read(DataInputView in) throws IOException {
-
-		// Read the job ID
-		this.jobID.read(in);
-
-		if (in.readBoolean()) {
-
-			// Find class loader for this job
-			final ClassLoader cl = LibraryCacheManager.getClassLoader(this.jobID);
-			if (cl == null) {
-				throw new IOException("Cannot find class loader for job " + this.jobID);
-			}
-
-			// Read the name of the class
-			final String className = StringRecord.readString(in);
-
-			// Try to locate the class using the job's class loader
-			Class<? extends InputSplit> splitClass = null;
-			try {
-				splitClass = (Class<? extends InputSplit>) Class.forName(className, true, cl);
-			} catch (ClassNotFoundException e) {
-				throw new IOException(e);
-			}
-
-			try {
-				this.inputSplit = splitClass.newInstance();
-			} catch (InstantiationException e) {
-				throw new IOException(e);
-			} catch (IllegalAccessException e) {
-				throw new IOException(e);
-			}
-
-			// Read the input split itself
-			this.inputSplit.read(in);
-		} else {
-			this.inputSplit = null;
-		}
-	}
-
-	/**
-	 * Returns the wrapped input split. The wrapped input split may also be <code>null</code> in case no more input
-	 * splits shall be consumed by the requesting task.
-	 * 
-	 * @return the wrapped input split, possibly <code>null</code>
-	 */
-	public InputSplit getInputSplit() {
-		return this.inputSplit;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java
deleted file mode 100644
index 08b5b65..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableInputSplitAssigner.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/**
- * 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.jobmanager.splitassigner;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Locale;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.apache.flink.core.io.LocatableInputSplit;
-
-/**
- * The locatable input split assigner assigns to each host splits that are local, before assigning
- * splits that are not local. 
- */
-public final class LocatableInputSplitAssigner implements InputSplitAssigner {
-
-	private static final Logger LOG = LoggerFactory.getLogger(LocatableInputSplitAssigner.class);
-
-
-	private final Set<LocatableInputSplit> unassigned = new HashSet<LocatableInputSplit>();
-	
-	private final ConcurrentHashMap<String, List<LocatableInputSplit>> localPerHost = new ConcurrentHashMap<String, List<LocatableInputSplit>>();
-	
-	private int localAssignments;		// lock protected by the unassigned set lock
-	
-	private int remoteAssignments;		// lock protected by the unassigned set lock
-
-	// --------------------------------------------------------------------------------------------
-	
-	public LocatableInputSplitAssigner(Collection<LocatableInputSplit> splits) {
-		this.unassigned.addAll(splits);
-	}
-	
-	public LocatableInputSplitAssigner(LocatableInputSplit[] splits) {
-		Collections.addAll(this.unassigned, splits);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public LocatableInputSplit getNextInputSplit(String host) {
-		// for a null host, we return an arbitrary split
-		if (host == null) {
-			
-			synchronized (this.unassigned) {
-				Iterator<LocatableInputSplit> iter = this.unassigned.iterator();
-				if (iter.hasNext()) {
-					LocatableInputSplit next = iter.next();
-					iter.remove();
-					
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("Assigning arbitrary split to null host.");
-					}
-					
-					remoteAssignments++;
-					return next;
-				} else {
-					if (LOG.isDebugEnabled()) {
-						LOG.debug("No more input splits remaining.");
-					}
-					return null;
-				}
-			}
-		}
-		
-		host = host.toLowerCase(Locale.US);
-		
-		// for any non-null host, we take the list of non-null splits
-		List<LocatableInputSplit> localSplits = this.localPerHost.get(host);
-		
-		// if we have no list for this host yet, create one
-		if (localSplits == null) {
-			localSplits = new ArrayList<LocatableInputSplit>(16);
-			
-			// lock the list, to be sure that others have to wait for that host's local list
-			synchronized (localSplits) {
-				List<LocatableInputSplit> prior = this.localPerHost.putIfAbsent(host, localSplits);
-				
-				// if someone else beat us in the case to create this list, then we do not populate this one, but
-				// simply work with that other list
-				if (prior == null) {
-					// we are the first, we populate
-					
-					// first, copy the remaining splits to release the lock on the set early
-					// because that is shared among threads
-					LocatableInputSplit[] remaining;
-					synchronized (this.unassigned) {
-						remaining = (LocatableInputSplit[]) this.unassigned.toArray(new LocatableInputSplit[this.unassigned.size()]);
-					}
-					
-					for (LocatableInputSplit is : remaining) {
-						if (isLocal(host, is.getHostnames())) {
-							localSplits.add(is);
-						}
-					}
-				}
-				else {
-					// someone else was faster
-					localSplits = prior;
-				}
-			}
-		}
-		
-		// at this point, we have a list of local splits (possibly empty)
-		// we need to make sure no one else operates in the current list (that protects against
-		// list creation races) and that the unassigned set is consistent
-		// NOTE: we need to obtain the locks in this order, strictly!!!
-		synchronized (localSplits) {
-			int size = localSplits.size();
-			if (size > 0) {
-				synchronized (this.unassigned) {
-					do {
-						--size;
-						LocatableInputSplit split = localSplits.remove(size);
-						if (this.unassigned.remove(split)) {
-							
-							if (LOG.isDebugEnabled()) {
-								LOG.debug("Assigning local split to host " + host);
-							}
-							
-							localAssignments++;
-							return split;
-						}
-					} while (size > 0);
-				}
-			}
-		}
-		
-		// we did not find a local split, return any
-		synchronized (this.unassigned) {
-			Iterator<LocatableInputSplit> iter = this.unassigned.iterator();
-			if (iter.hasNext()) {
-				LocatableInputSplit next = iter.next();
-				iter.remove();
-				
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Assigning remote split to host " + host);
-				}
-				
-				remoteAssignments++;
-				return next;
-			} else {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("No more input splits remaining.");
-				}
-				return null;
-			}
-		}
-	}
-	
-	private static final boolean isLocal(String host, String[] hosts) {
-		if (host == null || hosts == null) {
-			return false;
-		}
-		
-		for (String h : hosts) {
-			if (h != null && host.equals(h.toLowerCase())) {
-				return true;
-			}
-		}
-		
-		return false;
-	}
-	
-	public int getNumberOfLocalAssignments() {
-		return localAssignments;
-	}
-	
-	public int getNumberOfRemoteAssignments() {
-		return remoteAssignments;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java
index 4e5acc1..c7d8f0a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertex.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.managementgraph;
 
 import java.io.IOException;
@@ -27,16 +26,13 @@ import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.util.EnumUtils;
+import org.apache.flink.runtime.execution.ExecutionState2;
+import org.apache.flink.types.StringValue;
 import org.apache.flink.util.StringUtils;
 
 /**
  * This class implements a management vertex of a {@link ManagementGraph}. A management vertex is derived from the type
  * of vertices Nephele uses in its internal scheduling structures.
- * <p>
- * This class is not thread-safe.
- * 
  */
 public final class ManagementVertex extends ManagementAttachment implements IOReadableWritable {
 
@@ -63,7 +59,7 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	/**
 	 * The current execution state of the vertex represented by this management vertex.
 	 */
-	private ExecutionState executionState = ExecutionState.CREATED;
+	private ExecutionState2 executionState = ExecutionState2.CREATED;
 
 	/**
 	 * The name of the instance the vertex represented by this management vertex currently runs on.
@@ -240,7 +236,7 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	 * @param executionState
 	 *        the current execution state of this vertex
 	 */
-	public void setExecutionState(final ExecutionState executionState) {
+	public void setExecutionState(ExecutionState2 executionState) {
 		this.executionState = executionState;
 	}
 
@@ -249,7 +245,7 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	 * 
 	 * @return the current execution state of this management vertex
 	 */
-	public ExecutionState getExecutionState() {
+	public ExecutionState2 getExecutionState() {
 		return this.executionState;
 	}
 
@@ -275,7 +271,7 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	public void read(final DataInputView in) throws IOException {
 
 		// Read the execution state
-		this.executionState = EnumUtils.readEnum(in, ExecutionState.class);
+		this.executionState = ExecutionState2.values()[in.readInt()];
 
 		// Read number of input gates
 		int numberOfInputGates = in.readInt();
@@ -295,17 +291,11 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 
 	@Override
 	public void write(final DataOutputView out) throws IOException {
-
-		// Write the execution state
-		EnumUtils.writeEnum(out, this.executionState);
-
-		// Write out number of input gates
+		out.writeInt(this.executionState.ordinal());
 		out.writeInt(this.inputGates.size());
-
-		// Write out number of output gates
 		out.writeInt(this.outputGates.size());
 
-		StringRecord.writeString(out, this.instanceName);
+		StringValue.writeString(this.instanceName, out);
 	}
 	
 	@Override
@@ -314,9 +304,9 @@ public final class ManagementVertex extends ManagementAttachment implements IORe
 	}
 	
 	/**
-	 * Returns Json representation of this ManagementVertex
+	 * Returns JSON representation of this ManagementVertex
 	 * 
-	 * @return
+	 * @return A JSON representation of this ManagementVertex
 	 */
 	public String toJson() {
 		StringBuilder json = new StringBuilder("");

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
index 92955e8..6e7277d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
@@ -33,4 +33,8 @@ public final class ManagementVertexID extends AbstractID {
 	public ManagementVertexID(AbstractID toCopy) {
 		super(toCopy);
 	}
+	
+	public ManagementVertexID(long lowerPart, long upperPart) {
+		super(lowerPart, upperPart);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
index 641d4b2..2150cb6 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CollectorMapDriver.java
@@ -36,6 +36,7 @@ import org.apache.flink.util.MutableObjectIterator;
  * @param <IT> The mapper's input data type.
  * @param <OT> The mapper's output data type.
  */
+@SuppressWarnings("deprecation")
 public class CollectorMapDriver<IT, OT> implements PactDriver<GenericCollectorMap<IT, OT>, OT> {
 	
 	private PactTaskContext<GenericCollectorMap<IT, OT>, OT> taskContext;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
index 9fedab2..e1b16ed 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
@@ -38,7 +38,7 @@ import org.apache.flink.util.MutableObjectIterator;
  * The CrossTask builds the Cartesian product of the pairs of its two inputs. Each element (pair of pairs) is handed to
  * the <code>cross()</code> method of the CrossFunction.
  * 
- * @see org.apache.flink.api.java.functions.CrossFunction
+ * @see org.apache.flink.api.common.functions.CrossFunction
  */
 public class CrossDriver<T1, T2, OT> implements PactDriver<CrossFunction<T1, T2, OT>, OT> {
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java
index ee2b295..153a764 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/MergeMatchIterator.java
@@ -147,14 +147,14 @@ public class MergeMatchIterator<T1, T2, O> implements JoinTaskIterator<T1, T2, O
 	 * 
 	 * @throws Exception Forwards all exceptions from the user code and the I/O system.
 	 * 
-	 * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey()
+	 * @see org.apache.flink.runtime.operators.util.JoinTaskIterator#callWithNextKey(FlatJoinFunction, Collector)
 	 */
 	@Override
 	public boolean callWithNextKey(final FlatJoinFunction<T1, T2, O> matchFunction, final Collector<O> collector)
 	throws Exception
 	{
 		if (!this.iterator1.nextKey() || !this.iterator2.nextKey()) {
-			// consume all remanining keys (hack to prevent remaining inputs during iterations, lets get rid of this soon)
+			// consume all remaining keys (hack to prevent remaining inputs during iterations, lets get rid of this soon)
 			while (this.iterator1.nextKey());
 			while (this.iterator2.nextKey());
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
index 358719d..2eaff60 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
@@ -792,9 +792,6 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 		 */
 		public void run() {
 			try {
-				if (this.parentTask != null) {
-					this.parentTask.userThreadStarted(this);
-				}
 				go();
 			}
 			catch (Throwable t) {
@@ -802,9 +799,6 @@ public class UnilateralSortMerger<E> implements Sorter<E> {
 					+ t.getMessage(), t));
 			}
 			finally {
-				if (this.parentTask != null) {
-					this.parentTask.userThreadFinished(this);
-				}
 			}
 		}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/TaskManagerProfiler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/TaskManagerProfiler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/TaskManagerProfiler.java
index 3722c15..d76d377 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/TaskManagerProfiler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/TaskManagerProfiler.java
@@ -16,22 +16,20 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.taskmanager.Task;
 
 /**
  * This interface must be implemented by profiling components
  * for the task manager manager.
- * 
  */
 public interface TaskManagerProfiler {
 
 	/**
-	 * Registers an {@link ExecutionListener} object for profiling.
+	 * Registers an {@link org.apache.flink.runtime.execution.ExecutionListener} object for profiling.
 	 * 
 	 * @param task
 	 *        task to be register a profiling listener for
@@ -41,13 +39,13 @@ public interface TaskManagerProfiler {
 	void registerExecutionListener(Task task, Configuration jobConfiguration);
 
 	/**
-	 * Unregisters all previously register {@link ExecutionListener} objects for
+	 * Unregisters all previously register {@link org.apache.flink.runtime.execution.ExecutionListener} objects for
 	 * the vertex identified by the given ID.
 	 * 
 	 * @param id
-	 *        the ID of the vertex to unregister the {@link ExecutionListener} objects for
+	 *        the ID of the vertex to unregister the {@link org.apache.flink.runtime.execution.ExecutionListener} objects for
 	 */
-	void unregisterExecutionListener(ExecutionVertexID id);
+	void unregisterExecutionListener(ExecutionAttemptID id);
 
 	/**
 	 * Shuts done the task manager's profiling component

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
index bf6105e..77c4f5b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentListenerImpl.java
@@ -16,16 +16,16 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.runtime.execution.ExecutionListener;
-import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.execution.ExecutionState2;
 import org.apache.flink.runtime.execution.RuntimeEnvironment;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 public class EnvironmentListenerImpl implements ExecutionListener {
 
@@ -35,65 +35,30 @@ public class EnvironmentListenerImpl implements ExecutionListener {
 
 	private final RuntimeEnvironment environment;
 
-	public EnvironmentListenerImpl(final TaskManagerProfilerImpl taskManagerProfiler,
-			final RuntimeEnvironment environment) {
-
+	public EnvironmentListenerImpl(TaskManagerProfilerImpl taskManagerProfiler, RuntimeEnvironment environment) {
 		this.taskManagerProfiler = taskManagerProfiler;
 		this.environment = environment;
 	}
 
 
 	@Override
-	public void executionStateChanged(final JobID jobID, final ExecutionVertexID vertexID,
-			final ExecutionState newExecutionState, final String optionalMessage) {
+	public void executionStateChanged(JobID jobID, JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId, ExecutionState2 newExecutionState, String optionalMessage) {
 
 		switch (newExecutionState) {
 		case RUNNING:
-			this.taskManagerProfiler.registerMainThreadForCPUProfiling(this.environment,
-				this.environment.getExecutingThread(), vertexID);
+			this.taskManagerProfiler.registerMainThreadForCPUProfiling(this.environment, this.environment.getExecutingThread(), executionId);
 			break;
-		case FINISHING:
+			
 		case FINISHED:
 		case CANCELING:
 		case CANCELED:
 		case FAILED:
-			this.taskManagerProfiler.unregisterMainThreadFromCPUProfiling(this.environment,
-				this.environment.getExecutingThread());
+			this.taskManagerProfiler.unregisterMainThreadFromCPUProfiling(this.environment, this.environment.getExecutingThread());
 			break;
+			
 		default:
-			LOG.error("Unexpected state transition to " + newExecutionState + " for vertex " + vertexID);
+			LOG.error(String.format("Unexpected state transition to %s for vertex %s (%d) attempt %s", newExecutionState, vertexId, subtaskIndex, executionId));
 			break;
 		}
 	}
-
-
-	@Override
-	public void userThreadFinished(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-
-		// Make sure the user thread is not the task's main thread
-		if (this.environment.getExecutingThread() == userThread) {
-			return;
-		}
-
-		this.taskManagerProfiler.unregisterUserThreadFromCPUProfiling(this.environment, userThread);
-	}
-
-
-	@Override
-	public void userThreadStarted(final JobID jobID, final ExecutionVertexID vertexID, final Thread userThread) {
-
-		// Make sure the user thread is not the task's main thread
-		if (this.environment.getExecutingThread() == userThread) {
-			return;
-		}
-
-		this.taskManagerProfiler.registerUserThreadForCPUProfiling(this.environment, userThread);
-	}
-
-
-	@Override
-	public int getPriority() {
-
-		return 1;
-	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
index 1f78138..7eac666 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/EnvironmentThreadSet.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl;
 
 import java.lang.management.ThreadInfo;
@@ -25,7 +24,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.profiling.impl.types.InternalExecutionVertexThreadProfilingData;
 
@@ -79,15 +78,15 @@ public class EnvironmentThreadSet {
 
 	private final Thread mainThread;
 
-	private final ExecutionVertexID executionVertexID;
+	private final ExecutionAttemptID executionId;
 
 	private final Map<Thread, CPUUtilizationSnapshot> userThreads = new HashMap<Thread, CPUUtilizationSnapshot>();
 
 	private CPUUtilizationSnapshot mainThreadSnapshot = null;
 
-	public EnvironmentThreadSet(ThreadMXBean tmx, Thread mainThread, ExecutionVertexID executionVertexID) {
+	public EnvironmentThreadSet(ThreadMXBean tmx, Thread mainThread, ExecutionAttemptID executionId) {
 		this.mainThread = mainThread;
-		this.executionVertexID = executionVertexID;
+		this.executionId = executionId;
 
 		this.mainThreadSnapshot = createCPUUtilizationSnapshot(tmx, mainThread, System.currentTimeMillis());
 	}
@@ -206,7 +205,7 @@ public class EnvironmentThreadSet {
 				sumWaiTime /= (divisor + 1);
 			}
 
-			return new InternalExecutionVertexThreadProfilingData(jobID, this.executionVertexID, (int) mainInterval,
+			return new InternalExecutionVertexThreadProfilingData(jobID, this.executionId, (int) mainInterval,
 				sumUsrTime, sumSysTime, sumBlkTime, sumWaiTime);
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
index 97a5b50..c7efb8c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/JobProfilingData.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl;
 
 import java.util.HashMap;
@@ -27,14 +26,13 @@ import java.util.Set;
 
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertexIterator;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.instance.DummyInstance;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.profiling.impl.types.InternalInstanceProfilingData;
 import org.apache.flink.runtime.profiling.types.InstanceSummaryProfilingEvent;
 
+
 public class JobProfilingData {
 
 	private final ExecutionGraph executionGraph;
@@ -111,8 +109,7 @@ public class JobProfilingData {
 	private InstanceSummaryProfilingEvent constructInstanceSummary(long timestamp) {
 
 		final int numberOfInstances = this.collectedInstanceProfilingData.size();
-		final Iterator<InstanceConnectionInfo> instanceIterator = this.collectedInstanceProfilingData.keySet()
-			.iterator();
+		final Iterator<InstanceConnectionInfo> instanceIterator = this.collectedInstanceProfilingData.keySet().iterator();
 
 		long freeMemorySum = 0;
 		long totalMemorySum = 0;
@@ -134,8 +131,7 @@ public class JobProfilingData {
 		// Sum up the individual values
 		while (instanceIterator.hasNext()) {
 
-			final InternalInstanceProfilingData profilingData = this.collectedInstanceProfilingData
-				.get(instanceIterator.next());
+			final InternalInstanceProfilingData profilingData = this.collectedInstanceProfilingData.get(instanceIterator.next());
 
 			freeMemorySum += profilingData.getFreeMemory();
 			ioWaitCPUSum += profilingData.getIOWaitCPU();

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java
index 1d8d80f..f80e624 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/TaskManagerProfilerImpl.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl;
 
 import org.slf4j.Logger;
@@ -24,7 +23,7 @@ import org.slf4j.LoggerFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.ipc.RPC;
 import org.apache.flink.runtime.net.NetUtils;
@@ -107,11 +106,11 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro
 	public void registerExecutionListener(final Task task, final Configuration jobConfiguration) {
 
 		// Register profiling hook for the environment
-		task.registerExecutionListener(new EnvironmentListenerImpl(this, task.getRuntimeEnvironment()));
+		task.registerExecutionListener(new EnvironmentListenerImpl(this, task.getEnvironment()));
 	}
 
 	@Override
-	public void unregisterExecutionListener(ExecutionVertexID id) {
+	public void unregisterExecutionListener(ExecutionAttemptID id) {
 		/*
 		 * Nothing to do here, the task will unregister itself when its
 		 * execution state has either switched to FINISHED, CANCELLED,
@@ -121,7 +120,6 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro
 
 	@Override
 	public void shutdown() {
-
 		// Stop the timer task
 		this.timer.cancel();
 	}
@@ -174,8 +172,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro
 		}
 	}
 
-	public void registerMainThreadForCPUProfiling(Environment environment, Thread thread,
-			ExecutionVertexID executionVertexID) {
+	public void registerMainThreadForCPUProfiling(Environment environment, Thread thread, ExecutionAttemptID executionID) {
 
 		synchronized (this.monitoredThreads) {
 			LOG.debug("Registering thread " + thread.getName() + " for CPU monitoring");
@@ -184,7 +181,7 @@ public class TaskManagerProfilerImpl extends TimerTask implements TaskManagerPro
 					+ environment.getTaskName());
 			}
 
-			this.monitoredThreads.put(environment, new EnvironmentThreadSet(this.tmx, thread, executionVertexID));
+			this.monitoredThreads.put(environment, new EnvironmentThreadSet(this.tmx, thread, executionID));
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
index 9f78caa..f2a23e3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexProfilingData.java
@@ -16,53 +16,49 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl.types;
 
 import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
 
 public abstract class InternalExecutionVertexProfilingData implements InternalProfilingData {
 
-	private final ExecutionVertexID executionVertexID;
+	private final ExecutionAttemptID executionId;
 
 	private final JobID jobId;
 
+	
 	public InternalExecutionVertexProfilingData() {
 		this.jobId = new JobID();
-		this.executionVertexID = new ExecutionVertexID();
+		this.executionId = new ExecutionAttemptID();
 	}
 
-	public InternalExecutionVertexProfilingData(JobID jobID, ExecutionVertexID executionVertexID) {
+	public InternalExecutionVertexProfilingData(JobID jobID, ExecutionAttemptID executionId) {
 		this.jobId = jobID;
-		this.executionVertexID = executionVertexID;
+		this.executionId = executionId;
 	}
 
-	public ExecutionVertexID getExecutionVertexID() {
-
-		return this.executionVertexID;
+	public ExecutionAttemptID getExecutionAttemptId() {
+		return this.executionId;
 	}
 
 	public JobID getJobID() {
-
 		return this.jobId;
 	}
 
 	@Override
 	public void read(DataInputView in) throws IOException {
-
 		this.jobId.read(in);
-		this.executionVertexID.read(in);
+		this.executionId.read(in);
 	}
 
 	@Override
 	public void write(DataOutputView out) throws IOException {
-
 		this.jobId.write(out);
-		this.executionVertexID.write(out);
+		this.executionId.write(out);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
index 93d1fa6..892968d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalExecutionVertexThreadProfilingData.java
@@ -16,14 +16,13 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl.types;
 
 import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
 
 public class InternalExecutionVertexThreadProfilingData extends InternalExecutionVertexProfilingData {
@@ -38,10 +37,10 @@ public class InternalExecutionVertexThreadProfilingData extends InternalExecutio
 
 	private int waitedTime = 0;
 
-	public InternalExecutionVertexThreadProfilingData(JobID jobID, ExecutionVertexID executionVertexID,
-			int profilingInterval, int userTime, int systemTime, int blockedTime, int waitedTime) {
-
-		super(jobID, executionVertexID);
+	public InternalExecutionVertexThreadProfilingData(JobID jobID, ExecutionAttemptID executionId,
+			int profilingInterval, int userTime, int systemTime, int blockedTime, int waitedTime)
+	{
+		super(jobID, executionId);
 
 		this.profilingInterval = profilingInterval;
 		this.userTime = userTime;
@@ -50,12 +49,10 @@ public class InternalExecutionVertexThreadProfilingData extends InternalExecutio
 		this.waitedTime = waitedTime;
 	}
 
-	public InternalExecutionVertexThreadProfilingData() {
-	}
+	public InternalExecutionVertexThreadProfilingData() {}
 
 	@Override
 	public void read(DataInputView in) throws IOException {
-
 		super.read(in);
 
 		this.profilingInterval = in.readInt();
@@ -67,7 +64,6 @@ public class InternalExecutionVertexThreadProfilingData extends InternalExecutio
 
 	@Override
 	public void write(DataOutputView out) throws IOException {
-
 		super.write(out);
 
 		out.writeInt(this.profilingInterval);
@@ -78,27 +74,22 @@ public class InternalExecutionVertexThreadProfilingData extends InternalExecutio
 	}
 
 	public int getBlockedTime() {
-
 		return this.blockedTime;
 	}
 
 	public int getProfilingInterval() {
-
 		return this.profilingInterval;
 	}
 
 	public int getSystemTime() {
-
 		return this.systemTime;
 	}
 
 	public int getUserTime() {
-
 		return this.userTime;
 	}
 
 	public int getWaitedTime() {
-
 		return this.waitedTime;
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java
index 4fbfd6e..3d00161 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalInputGateProfilingData.java
@@ -16,19 +16,18 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl.types;
 
 import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
 
 public class InternalInputGateProfilingData implements InternalProfilingData {
 
-	private final ExecutionVertexID executionVertexID;
+	private final ExecutionAttemptID executionID;
 
 	private final JobID jobId;
 
@@ -40,16 +39,17 @@ public class InternalInputGateProfilingData implements InternalProfilingData {
 
 	public InternalInputGateProfilingData() {
 		this.jobId = new JobID();
-		this.executionVertexID = new ExecutionVertexID();
+		this.executionID = new ExecutionAttemptID();
 		this.gateIndex = 0;
 		this.profilingInternval = 0;
 		this.noRecordsAvailableCounter = 0;
 	}
 
-	public InternalInputGateProfilingData(JobID jobID, ExecutionVertexID executionVertexID, int gateIndex,
-			int profilingInterval, int noRecordsAvailableCounter) {
+	public InternalInputGateProfilingData(JobID jobID, ExecutionAttemptID executionID, int gateIndex,
+			int profilingInterval, int noRecordsAvailableCounter)
+	{
 		this.jobId = jobID;
-		this.executionVertexID = executionVertexID;
+		this.executionID = executionID;
 		this.gateIndex = gateIndex;
 		this.profilingInternval = profilingInterval;
 		this.noRecordsAvailableCounter = noRecordsAvailableCounter;
@@ -57,9 +57,8 @@ public class InternalInputGateProfilingData implements InternalProfilingData {
 
 	@Override
 	public void read(DataInputView in) throws IOException {
-
 		this.jobId.read(in);
-		this.executionVertexID.read(in);
+		this.executionID.read(in);
 		this.gateIndex = in.readInt();
 		this.profilingInternval = in.readInt();
 		this.noRecordsAvailableCounter = in.readInt();
@@ -67,9 +66,8 @@ public class InternalInputGateProfilingData implements InternalProfilingData {
 
 	@Override
 	public void write(DataOutputView out) throws IOException {
-
 		this.jobId.write(out);
-		this.executionVertexID.write(out);
+		this.executionID.write(out);
 		out.writeInt(this.gateIndex);
 		out.writeInt(this.profilingInternval);
 		out.writeInt(this.noRecordsAvailableCounter);
@@ -79,8 +77,8 @@ public class InternalInputGateProfilingData implements InternalProfilingData {
 		return this.jobId;
 	}
 
-	public ExecutionVertexID getExecutionVertexID() {
-		return this.executionVertexID;
+	public ExecutionAttemptID getExecutionID() {
+		return this.executionID;
 	}
 
 	public int getGateIndex() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java
index 579aa08..c05fbe7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalOutputGateProfilingData.java
@@ -16,19 +16,18 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl.types;
 
 import java.io.IOException;
 
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.jobgraph.JobID;
 
 public class InternalOutputGateProfilingData implements InternalProfilingData {
 
-	private final ExecutionVertexID executionVertexID;
+	private final ExecutionAttemptID executionID;
 
 	private final JobID jobId;
 
@@ -40,16 +39,16 @@ public class InternalOutputGateProfilingData implements InternalProfilingData {
 
 	public InternalOutputGateProfilingData() {
 		this.jobId = new JobID();
-		this.executionVertexID = new ExecutionVertexID();
+		this.executionID = new ExecutionAttemptID();
 		this.gateIndex = 0;
 		this.profilingInternval = 0;
 		this.channelCapacityExhaustedCounter = 0;
 	}
 
-	public InternalOutputGateProfilingData(JobID jobID, ExecutionVertexID executionVertexID, int gateIndex,
+	public InternalOutputGateProfilingData(JobID jobID, ExecutionAttemptID executionVertexID, int gateIndex,
 			int profilingInterval, int channelCapacityExhaustedCounter) {
 		this.jobId = jobID;
-		this.executionVertexID = executionVertexID;
+		this.executionID = executionVertexID;
 		this.gateIndex = gateIndex;
 		this.profilingInternval = profilingInterval;
 		this.channelCapacityExhaustedCounter = channelCapacityExhaustedCounter;
@@ -57,9 +56,8 @@ public class InternalOutputGateProfilingData implements InternalProfilingData {
 
 	@Override
 	public void read(DataInputView in) throws IOException {
-
 		this.jobId.read(in);
-		this.executionVertexID.read(in);
+		this.executionID.read(in);
 		this.gateIndex = in.readInt();
 		this.profilingInternval = in.readInt();
 		this.channelCapacityExhaustedCounter = in.readInt();
@@ -67,9 +65,8 @@ public class InternalOutputGateProfilingData implements InternalProfilingData {
 
 	@Override
 	public void write(DataOutputView out) throws IOException {
-
 		this.jobId.write(out);
-		this.executionVertexID.write(out);
+		this.executionID.write(out);
 		out.writeInt(this.gateIndex);
 		out.writeInt(this.profilingInternval);
 		out.writeInt(this.channelCapacityExhaustedCounter);
@@ -79,8 +76,8 @@ public class InternalOutputGateProfilingData implements InternalProfilingData {
 		return this.jobId;
 	}
 
-	public ExecutionVertexID getExecutionVertexID() {
-		return this.executionVertexID;
+	public ExecutionAttemptID getExecutionID() {
+		return this.executionID;
 	}
 
 	public int getGateIndex() {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalProfilingData.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalProfilingData.java b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalProfilingData.java
index 7edcc55..c656741 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalProfilingData.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/profiling/impl/types/InternalProfilingData.java
@@ -16,11 +16,8 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.profiling.impl.types;
 
 import org.apache.flink.core.io.IOReadableWritable;
 
-public interface InternalProfilingData extends IOReadableWritable {
-
-}
+public interface InternalProfilingData extends IOReadableWritable {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
index a2577a9..b080606 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
@@ -16,18 +16,15 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.protocols;
 
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.runtime.event.job.AbstractEvent;
 import org.apache.flink.runtime.event.job.RecentJobEvent;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.managementgraph.ManagementGraph;
-import org.apache.flink.runtime.managementgraph.ManagementVertexID;
 
 /**
  * This protocol provides extended management capabilities beyond the
@@ -71,29 +68,6 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
 	List<AbstractEvent> getEvents(JobID jobID) throws IOException;
 
 	/**
-	 * Kills the task with the given vertex ID.
-	 *
-	 * @param jobID
-	 *        the ID of the job the vertex to be killed belongs to
-	 * @param id
-	 *        the vertex ID which identified the task be killed
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the kill request
-	 */
-	void killTask(JobID jobID, ManagementVertexID id) throws IOException;
-
-	/**
-	 * Triggers all task managers involved in processing the job with the given job ID to write the utilization of
-	 * their read and write buffers to their log files. This method is primarily for debugging purposes.
-	 * 
-	 * @param jobID
-	 *        the ID of the job to print the buffer distribution for
-	 * @throws IOException
-	 *         throws if an error occurs while transmitting the request
-	 */
-	void logBufferUtilization(JobID jobID) throws IOException;
-
-	/**
 	 * Returns the number of available slots among the registered task managers
 	 * @return number of available slots
 	 * @throws IOException

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/InputSplitProviderProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/InputSplitProviderProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/InputSplitProviderProtocol.java
index 3f6b165..dcc9446 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/InputSplitProviderProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/InputSplitProviderProtocol.java
@@ -16,38 +16,19 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.protocols;
 
 import java.io.IOException;
 
+import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.core.protocols.VersionedProtocol;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobmanager.splitassigner.InputSplitWrapper;
-import org.apache.flink.runtime.types.IntegerRecord;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 /**
- * The input split provider protocol is used to facilitate RPC calls related to the lazy split assignment which Nephele
- * applies to provide better load balancing properties.
- * 
+ * The input split provider protocol is used to facilitate RPC calls related to the lazy split assignment.
  */
 public interface InputSplitProviderProtocol extends VersionedProtocol {
 
-	/**
-	 * Requests the next split to be consumed by the task with the given execution vertex ID.
-	 * 
-	 * @param jobID
-	 *        the ID of the job the task to retrieve the next input split for belongs to
-	 * @param vertexID
-	 *        the ID of the task to retrieve the next input split for
-	 * @param sequenceNumber
-	 *        a sequence number, starting at 0 and increased by the task on each request
-	 * @return a wrapper containing the next input split. The wrapped input split may also be <code>null</code> in case
-	 *         no more input splits shall be consumed by the task with the given execution vertex ID
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while retrieving the new input split
-	 */
-	InputSplitWrapper requestNextInputSplit(JobID jobID, ExecutionVertexID vertexID, IntegerRecord sequenceNumber)
-			throws IOException;
+	InputSplit requestNextInputSplit(JobID jobID, JobVertexID vertex) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagementProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagementProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagementProtocol.java
index 1f47d9c..bbb31a1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagementProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagementProtocol.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.protocols;
 
 import java.io.IOException;
@@ -30,9 +29,7 @@ import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.types.IntegerRecord;
 
 /**
- * The JobManagementProtocol specifies methods required to manage
- * Nephele jobs from a job client.
- * 
+ * The JobManagementProtocol specifies methods required to manage jobs from a job client.
  */
 public interface JobManagementProtocol extends VersionedProtocol {
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
index 8937782..1800e3f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
@@ -19,19 +19,15 @@
 package org.apache.flink.runtime.protocols;
 
 import java.io.IOException;
-import java.util.List;
-import java.util.Set;
 
 import org.apache.flink.core.protocols.VersionedProtocol;
 import org.apache.flink.runtime.deployment.TaskDeploymentDescriptor;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileRequest;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheProfileResponse;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheUpdate;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.io.network.channels.ChannelID;
-import org.apache.flink.runtime.taskmanager.TaskCancelResult;
-import org.apache.flink.runtime.taskmanager.TaskKillResult;
-import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.taskmanager.TaskOperationResult;
 
 /**
  * The task submission protocol is implemented by the task manager and allows the job manager
@@ -40,83 +36,11 @@ import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
  */
 public interface TaskOperationProtocol extends VersionedProtocol {
 
-	/**
-	 * Submits a list of tasks to the task manager.
-	 * 
-	 * @param tasks
-	 *        the tasks to be submitted
-	 * @return the result of the task submission
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
-	List<TaskSubmissionResult> submitTasks(List<TaskDeploymentDescriptor> tasks) throws IOException;
+	TaskOperationResult submitTask(TaskDeploymentDescriptor task) throws IOException;
 
-	/**
-	 * Advises the task manager to cancel the task with the given ID.
-	 * 
-	 * @param id
-	 *        the ID of the task to cancel
-	 * @return the result of the task cancel attempt
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
-	TaskCancelResult cancelTask(ExecutionVertexID id) throws IOException;
+	TaskOperationResult cancelTask(JobVertexID vertexId, int subtaskIndex, ExecutionAttemptID executionId) throws IOException;
 
-	/**
-	 * Advises the task manager to kill the task with the given ID.
-	 *
-	 * @param id
-	 *        the ID of the task to kill
-	 * @return the result of the task kill attempt
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
-	TaskKillResult killTask(ExecutionVertexID id) throws IOException;
-
-	/**
-	 * Queries the task manager about the cache status of the libraries stated in the {@link LibraryCacheProfileRequest}
-	 * object.
-	 * 
-	 * @param request
-	 *        a {@link LibraryCacheProfileRequest} containing a list of libraries whose cache status is to be determined
-	 * @return a {@link LibraryCacheProfileResponse} containing the cache status for each library included in the
-	 *         request
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
 	LibraryCacheProfileResponse getLibraryCacheProfile(LibraryCacheProfileRequest request) throws IOException;
-
-	/**
-	 * Updates the task manager's library cache.
-	 * 
-	 * @param update
-	 *        a {@link LibraryCacheUpdate} object used to transmit the library data
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
+	
 	void updateLibraryCache(LibraryCacheUpdate update) throws IOException;
-
-	/**
-	 * Invalidates the entries identified by the given channel IDs from the task manager's receiver lookup cache.
-	 * 
-	 * @param channelIDs
-	 *        the channel IDs identifying the cache entries to invalidate
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
-	void invalidateLookupCacheEntries(Set<ChannelID> channelIDs) throws IOException;
-
-	/**
-	 * Triggers the task manager write the current utilization of its read and write buffers to its logs.
-	 * This method is primarily for debugging purposes.
-	 */
-	void logBufferUtilization();
-
-	/**
-	 * Kills the task manager. This method is mainly intended to test and debug Nephele's fault tolerance mechanisms.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
-	 */
-	void killTaskManager() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/AbstractTaskResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/AbstractTaskResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/AbstractTaskResult.java
deleted file mode 100644
index e4c1fab..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/AbstractTaskResult.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * 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.taskmanager;
-
-import java.io.IOException;
-
-import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.io.StringRecord;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.util.EnumUtils;
-
-/**
- * An <code>AbstractTaskResult</code> is used to report the results
- * of a task related operation. It contains the ID of the corresponding task, a return code and
- * a description. In case of an error the description includes an error message.
- * 
- */
-public abstract class AbstractTaskResult implements IOReadableWritable {
-
-	public enum ReturnCode {
-		SUCCESS, DEPLOYMENT_ERROR, IPC_ERROR, NO_INSTANCE, ILLEGAL_STATE, TASK_NOT_FOUND, INSUFFICIENT_RESOURCES
-	};
-
-	private ExecutionVertexID vertexID;
-
-	private ReturnCode returnCode;
-
-	private String description;
-
-	/**
-	 * Constructs a new abstract task result.
-	 * 
-	 * @param vertexID
-	 *        the task ID this result belongs to
-	 * @param returnCode
-	 *        the return code of the operation
-	 */
-	public AbstractTaskResult(ExecutionVertexID vertexID, ReturnCode returnCode) {
-		this.vertexID = vertexID;
-		this.returnCode = returnCode;
-	}
-
-	/**
-	 * Constructs an empty abstract task result.
-	 */
-	public AbstractTaskResult() {
-		this.vertexID = null;
-		this.returnCode = ReturnCode.SUCCESS;
-	}
-
-	/**
-	 * Sets a description for this abstract task result.
-	 * 
-	 * @param description
-	 *        the description to be set
-	 */
-	public void setDescription(String description) {
-		this.description = description;
-	}
-
-	/**
-	 * Returns the description for this abstract task result.
-	 * 
-	 * @return the description for this abstract task result or <code>null</code> if no description has yet been set
-	 */
-	public String getDescription() {
-		return this.description;
-	}
-
-	/**
-	 * Returns the ID of the task this result belongs to.
-	 * 
-	 * @return the ID of the task this result belongs to
-	 */
-	public ExecutionVertexID getVertexID() {
-		return this.vertexID;
-	}
-
-	/**
-	 * Returns the return code of the result.
-	 * 
-	 * @return the return code of the result
-	 */
-	public ReturnCode getReturnCode() {
-		return this.returnCode;
-	}
-
-
-	@Override
-	public void read(DataInputView in) throws IOException {
-
-		// Read the jobID
-		boolean isNotNull = in.readBoolean();
-		if (isNotNull) {
-			this.vertexID = new ExecutionVertexID();
-			this.vertexID.read(in);
-		}
-
-		// Read the return code
-		this.returnCode = EnumUtils.readEnum(in, ReturnCode.class);
-
-		// Read the description
-		this.description = StringRecord.readString(in);
-	}
-
-
-	@Override
-	public void write(DataOutputView out) throws IOException {
-
-		// Write jobID
-		if (this.vertexID == null) {
-			out.writeBoolean(false);
-		} else {
-			out.writeBoolean(true);
-			this.vertexID.write(out);
-		}
-
-		// Write return code
-		EnumUtils.writeEnum(out, this.returnCode);
-
-		// Write the description
-		StringRecord.writeString(out, this.description);
-	}
-
-}


[24/63] [abbrv] Redesign Scheduler part 2

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureAction.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureAction.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureAction.java
new file mode 100644
index 0000000..28a984f
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFutureAction.java
@@ -0,0 +1,34 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+
+/**
+ * An action that is invoked once a {@link SlotAllocationFuture} is triggered.
+ */
+public interface SlotAllocationFutureAction {
+
+	/**
+	 * This method is called as soon as the SlotAllocationFuture is triggered.
+	 * 
+	 * @param slot The slot that has been allocated.
+	 */
+	void slotAllocated(AllocatedSlot slot);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java
new file mode 100644
index 0000000..bad3b88
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAvailablilityListener.java
@@ -0,0 +1,29 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import org.apache.flink.runtime.instance.Instance;
+
+/**
+ * A SlotRecycler handles allocated slots that have been released.
+ */
+public interface SlotAvailablilityListener {
+
+	void newSlotAvailable(Instance instance);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
new file mode 100644
index 0000000..9f008ef
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroup.java
@@ -0,0 +1,70 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+/**
+ * A slot sharing units defines which different task (from different job vertices) can be
+ * deployed together within a slot. This is a soft permission, in contrast to the hard constraint
+ * defined by the {@link CoLocationHint}.
+ */
+public class SlotSharingGroup {
+	
+	private final Set<JobVertexID> ids = new TreeSet<JobVertexID>();
+	
+	private SlotSharingGroupAssignment taskAssignment;
+	
+	
+	public SlotSharingGroup() {}
+	
+	public SlotSharingGroup(JobVertexID ... sharedVertices) {
+		for (JobVertexID id : sharedVertices) {
+			this.ids.add(id);
+		}
+	}
+	
+	
+	public void addVertexGroup(JobVertexID id) {
+		this.ids.add(id);
+	}
+	
+	public Set<JobVertexID> getJobVertexIds() {
+		return ids;
+	}
+	
+	
+	public SlotSharingGroupAssignment getTaskAssignment() {
+		if (this.taskAssignment == null) {
+			this.taskAssignment = new SlotSharingGroupAssignment();
+		}
+		
+		return this.taskAssignment;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return this.ids.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
new file mode 100644
index 0000000..e7968ed
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
@@ -0,0 +1,270 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+public class SlotSharingGroupAssignment {
+	
+	/** All slots currently allocated to this sharing group */
+	private final Set<SharedSlot> allSlots = new LinkedHashSet<SharedSlot>();
+	
+	/** The slots available per vertex type (jid), keyed by instance, to make them locatable */
+	private final Map<JobVertexID, Map<Instance, List<SharedSlot>>> availableSlotsPerJid = new LinkedHashMap<JobVertexID, Map<Instance, List<SharedSlot>>>();
+	
+	/** The tasks that are waiting, per vertex type (jid) */
+	private final Map<JobVertexID, Queue<ExecutionVertex2>> pendingTasks = new HashMap<JobVertexID, Queue<ExecutionVertex2>>();
+	
+	
+	// --------------------------------------------------------------------------------------------
+	
+	
+	public SubSlot addSlotWithTask(AllocatedSlot slot, JobVertexID jid) {
+		
+		final SharedSlot sharedSlot = new SharedSlot(slot, this);
+		final Instance location = slot.getInstance();
+		
+		synchronized (allSlots) {
+			// add to the total bookkeeping
+			allSlots.add(sharedSlot);
+			
+			// allocate us a sub slot to return
+			SubSlot subslot = sharedSlot.allocateSubSlot(jid);
+			boolean entryForNewJidExists = false;
+			
+			// let the other vertex types know about this one as well
+			
+			for (Map.Entry<JobVertexID, Map<Instance, List<SharedSlot>>> entry : availableSlotsPerJid.entrySet()) {
+				
+				if (entry.getKey().equals(jid)) {
+					entryForNewJidExists = true;
+					continue;
+				}
+				
+				Map<Instance, List<SharedSlot>> available = entry.getValue();
+				putIntoMultiMap(available, location, sharedSlot);
+			}
+			
+			// make sure an empty entry exists for this jid, if no other entry exists
+			if (!entryForNewJidExists) {
+				availableSlotsPerJid.put(jid, new LinkedHashMap<Instance, List<SharedSlot>>());
+			}
+			
+			return subslot;
+		}
+	}
+	
+	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex2 vertex) {
+		synchronized (allSlots) {
+			return getSlotForTaskInternal(jid, vertex.getPreferredLocations());
+		}
+	}
+	
+	public boolean sharedSlotAvailableForJid(SharedSlot slot, JobVertexID jid, boolean lastSubSlot) {
+		if (slot == null || jid == null) {
+			throw new NullPointerException();
+		}
+		
+		synchronized (allSlots) {
+			if (!allSlots.contains(slot)) {
+				throw new IllegalArgumentException("Slot was not associated with this SlotSharingGroup before.");
+			}
+			
+			if (lastSubSlot) {
+				// this was the last sub slot. unless there is something pending for this jid
+				// remove this from the availability list of all jids and 
+				// return that this one is good to release
+				allSlots.remove(slot);
+				
+				Instance location = slot.getAllocatedSlot().getInstance();
+				
+				for (Map.Entry<JobVertexID, Map<Instance, List<SharedSlot>>> mapEntry : availableSlotsPerJid.entrySet()) {
+					if (mapEntry.getKey().equals(jid)) {
+						continue;
+					}
+					
+					Map<Instance, List<SharedSlot>> map = mapEntry.getValue();
+					List<SharedSlot> list = map.get(location);
+					if (list == null || !list.remove(slot)) {
+						throw new IllegalStateException("SharedSlot was not available to another vertex type that it was not allocated for before.");
+					}
+					if (list.isEmpty()) {
+						map.remove(location);
+					}
+				}
+				
+				return true;
+			}
+			
+			Map<Instance, List<SharedSlot>> slotsForJid = availableSlotsPerJid.get(jid);
+			
+			// sanity check
+			if (slotsForJid == null) {
+				throw new IllegalStateException("Trying to return a slot for jid " + jid + 
+						" when available slots indicated that all slots were available.");
+			}
+			
+			putIntoMultiMap(slotsForJid, slot.getAllocatedSlot().getInstance(), slot);
+			
+			// do not release, we are still depending on this shared slot
+			return false;
+		}
+	}
+	
+	
+	/**
+	 * NOTE: This method is not synchronized by itself, needs to be synchronized externally.
+	 * 
+	 * @param jid
+	 * @return An allocated sub slot, or {@code null}, if no slot is available.
+	 */
+	private AllocatedSlot getSlotForTaskInternal(JobVertexID jid, Iterable<Instance> preferredLocations) {
+		if (allSlots.isEmpty()) {
+			return null;
+		}
+		
+		Map<Instance, List<SharedSlot>> slotsForJid = availableSlotsPerJid.get(jid);
+		
+		// get the available slots for the vertex type (jid)
+		if (slotsForJid == null) {
+			// no task is yet scheduled for that jid, so all slots are available
+			slotsForJid = new LinkedHashMap<Instance, List<SharedSlot>>();
+			availableSlotsPerJid.put(jid, slotsForJid);
+			
+			for (SharedSlot availableSlot : allSlots) {
+				putIntoMultiMap(slotsForJid, availableSlot.getAllocatedSlot().getInstance(), availableSlot);
+			}
+		}
+		else if (slotsForJid.isEmpty()) {
+			return null;
+		}
+		
+		// check whether we can schedule the task to a preferred location
+		if (preferredLocations != null) {
+			for (Instance location : preferredLocations) {
+				SharedSlot slot = removeFromMultiMap(slotsForJid, location);
+				if (slot != null) {
+					return slot.allocateSubSlot(jid);
+				}
+			}
+		}
+		
+		// schedule the task to any available location
+		SharedSlot slot = pollFromMultiMap(slotsForJid);
+		if (slot != null) {
+			return slot.allocateSubSlot(jid);
+		}
+		else {
+			return null;
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  State
+	// --------------------------------------------------------------------------------------------
+	
+	public int getNumberOfSlots() {
+		return allSlots.size();
+	}
+	
+	public int getNumberOfAvailableSlotsForJid(JobVertexID jid) {
+		synchronized (allSlots) {
+			Map<Instance, List<SharedSlot>> available = availableSlotsPerJid.get(jid);
+			
+			if (available != null) {
+				Set<SharedSlot> set = new HashSet<SharedSlot>();
+				
+				for (List<SharedSlot> list : available.values()) {
+					for (SharedSlot slot : list) {
+						set.add(slot);
+					}
+				}
+				
+				return set.size();
+			} else {
+				return allSlots.size();
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Utilities
+	// --------------------------------------------------------------------------------------------
+
+	private static final void putIntoMultiMap(Map<Instance, List<SharedSlot>> map, Instance location, SharedSlot slot) {
+		List<SharedSlot> slotsForInstance = map.get(location);
+		if (slotsForInstance == null) {
+			slotsForInstance = new ArrayList<SharedSlot>();
+			map.put(location, slotsForInstance);
+		}
+		slotsForInstance.add(slot);
+	}
+	
+	private static final SharedSlot removeFromMultiMap(Map<Instance, List<SharedSlot>> map, Instance location) {
+		List<SharedSlot> slotsForLocation = map.get(location);
+		
+		if (slotsForLocation == null) {
+			return null;
+		}
+		else {
+			SharedSlot slot = slotsForLocation.remove(slotsForLocation.size() - 1);
+			if (slotsForLocation.isEmpty()) {
+				map.remove(location);
+			}
+			
+			return slot;
+		}
+	}
+	
+	private static final SharedSlot pollFromMultiMap(Map<Instance, List<SharedSlot>> map) {
+		Iterator<Map.Entry<Instance, List<SharedSlot>>> iter = map.entrySet().iterator();
+		
+		while (iter.hasNext()) {
+			List<SharedSlot> slots = iter.next().getValue();
+			
+			if (slots.isEmpty()) {
+				iter.remove();
+			}
+			else if (slots.size() == 1) {
+				SharedSlot slot = slots.remove(0);
+				iter.remove();
+				return slot;
+			}
+			else {
+				return slots.remove(slots.size() - 1);
+			}
+		}
+		
+		return null;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
new file mode 100644
index 0000000..003239d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
@@ -0,0 +1,69 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+public class SubSlot extends AllocatedSlot {
+
+	private final SharedSlot sharedSlot;
+	
+	private final JobVertexID jid;
+	
+	private final int subSlotNumber;
+	
+	
+	public SubSlot(SharedSlot sharedSlot, int subSlotNumber, JobVertexID jid) {
+		super(sharedSlot.getAllocatedSlot().getJobID(),
+				sharedSlot.getAllocatedSlot().getInstance(),
+				sharedSlot.getAllocatedSlot().getSlotNumber());
+		
+		this.sharedSlot = sharedSlot;
+		this.jid = jid;
+		this.subSlotNumber = subSlotNumber;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public void releaseSlot() {
+		// cancel everything, if there is something. since this is atomically status based,
+		// it will not happen twice if another attempt happened before or concurrently
+		cancel();
+		
+		if (markReleased()) {
+			this.sharedSlot.returnAllocatedSlot(this);
+		}
+	}
+	
+	public SharedSlot getSharedSlot() {
+		return this.sharedSlot;
+	}
+	
+	public JobVertexID getJobVertexId() {
+		return jid;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "SubSlot " + subSlotNumber + " (" + super.toString() + ')';
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java
index a31e390..7e5ad3e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/SetupInfoServlet.java
@@ -123,7 +123,7 @@ public class SetupInfoServlet extends HttpServlet {
 				objInner.put("ipcPort", k.ipcPort());
 				objInner.put("dataPort", k.dataPort());
 				objInner.put("timeSinceLastHeartbeat", time / 1000);
-				objInner.put("slotsNumber", instance.getNumberOfSlots());
+				objInner.put("slotsNumber", instance.getTotalNumberOfSlots());
 				objInner.put("freeSlots", instance.getNumberOfAvailableSlots());
 				objInner.put("cpuCores", instance.getResources().getNumberOfCPUCores());
 				objInner.put("physicalMemory", instance.getResources().getSizeOfPhysicalMemory() >>> 20);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
index 83aac03..a2577a9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
@@ -83,16 +83,6 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
 	void killTask(JobID jobID, ManagementVertexID id) throws IOException;
 
 	/**
-	 * Kills the instance with the given name (i.e. shuts down its task manager).
-	 * 
-	 * @param instanceName
-	 *        the name of the instance to be killed
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the kill request
-	 */
-	void killInstance(StringRecord instanceName) throws IOException;
-
-	/**
 	 * Triggers all task managers involved in processing the job with the given job ID to write the utilization of
 	 * their read and write buffers to their log files. This method is primarily for debugging purposes.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
index 33571f5..8937782 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/TaskOperationProtocol.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.protocols;
 
 import java.io.IOException;
@@ -38,7 +37,6 @@ import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
  * The task submission protocol is implemented by the task manager and allows the job manager
  * to submit and cancel tasks, as well as to query the task manager for cached libraries and submit
  * these if necessary.
- * 
  */
 public interface TaskOperationProtocol extends VersionedProtocol {
 
@@ -111,9 +109,6 @@ public interface TaskOperationProtocol extends VersionedProtocol {
 	/**
 	 * Triggers the task manager write the current utilization of its read and write buffers to its logs.
 	 * This method is primarily for debugging purposes.
-	 * 
-	 * @throws IOException
-	 *         thrown if an error occurs while transmitting the request
 	 */
 	void logBufferUtilization();
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ExecutorThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ExecutorThreadFactory.java
deleted file mode 100644
index 13dd316..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/ExecutorThreadFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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.taskmanager;
-
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class ExecutorThreadFactory implements ThreadFactory {
-	
-	public static final ExecutorThreadFactory INSTANCE = new ExecutorThreadFactory();
-
-	private static final String THREAD_NAME = "Nephele Executor Thread ";
-	
-	private final AtomicInteger threadNumber = new AtomicInteger(1);
-	
-	
-	private ExecutorThreadFactory() {}
-	
-	
-	public Thread newThread(Runnable target) {
-		Thread t = new Thread(target, THREAD_NAME + threadNumber.getAndIncrement());
-		t.setDaemon(true);
-		return t;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
index 2d30cf2..f074f3c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
@@ -97,6 +97,7 @@ import org.apache.flink.runtime.protocols.JobManagerProtocol;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
 import org.apache.flink.runtime.types.IntegerRecord;
 import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.util.SerializableArrayList;
 import org.apache.flink.util.StringUtils;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/runtime/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/runtime/ExecutorThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/runtime/ExecutorThreadFactory.java
deleted file mode 100644
index e711eb2..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/runtime/ExecutorThreadFactory.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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.taskmanager.runtime;
-
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class ExecutorThreadFactory implements ThreadFactory {
-	
-	public static final ExecutorThreadFactory INSTANCE = new ExecutorThreadFactory();
-
-	private static final String THREAD_NAME = "Flink Executor Thread ";
-	
-	private final AtomicInteger threadNumber = new AtomicInteger(1);
-	
-	
-	private ExecutorThreadFactory() {}
-	
-	
-	public Thread newThread(Runnable target) {
-		Thread t = new Thread(target, THREAD_NAME + threadNumber.getAndIncrement());
-		t.setDaemon(true);
-		return t;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
new file mode 100644
index 0000000..964a754
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ExecutorThreadFactory.java
@@ -0,0 +1,38 @@
+/**
+ * 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.util;
+
+import java.util.concurrent.ThreadFactory;
+
+public class ExecutorThreadFactory implements ThreadFactory {
+	
+	public static final ExecutorThreadFactory INSTANCE = new ExecutorThreadFactory();
+
+	private static final String THREAD_NAME = "Flink Executor Thread";
+	
+	
+	private ExecutorThreadFactory() {}
+	
+	
+	public Thread newThread(Runnable target) {
+		Thread t = new Thread(target, THREAD_NAME);
+		t.setDaemon(true);
+		return t;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/util/NativeCodeLoader.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NativeCodeLoader.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NativeCodeLoader.java
deleted file mode 100644
index 200337e..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NativeCodeLoader.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * 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.util;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * Utility class to deal with native libraries.
- * 
- */
-public final class NativeCodeLoader {
-
-	/**
-	 * Set of the native libraries which are already loaded.
-	 */
-	private static Set<String> loadedLibrarySet = new HashSet<String>();
-
-	/**
-	 * Directory prefix for native libraries inside JAR files.
-	 */
-	private static final String JAR_PREFIX = "META-INF/lib/";
-
-	/**
-	 * Size of temporary buffer to extract native libraries in bytes.
-	 */
-	private static final int BUFSIZE = 8192;
-
-	/**
-	 * Empty private constructor to avoid instantiation.
-	 */
-	private NativeCodeLoader() {
-	}
-
-	/**
-	 * Loads a native library from a file.
-	 * 
-	 * @param directory
-	 *        the directory in which the library is supposed to be located
-	 * @param filename
-	 *        filename of the library to be loaded
-	 * @throws IOException
-	 *         thrown if an internal native library cannot be extracted
-	 */
-	public static void loadLibraryFromFile(final String directory, final String filename) throws IOException {
-
-		final String libraryPath = directory + File.separator + filename;
-
-		synchronized (loadedLibrarySet) {
-
-			final File outputFile = new File(directory, filename);
-			if (!outputFile.exists()) {
-
-				// Try to extract the library from the system resources
-				final ClassLoader cl = ClassLoader.getSystemClassLoader();
-				final InputStream in = cl.getResourceAsStream(JAR_PREFIX + filename);
-				if (in == null) {
-					throw new IOException("Unable to extract native library " + filename + " to " + directory);
-				}
-
-				final OutputStream out = new FileOutputStream(outputFile);
-				copy(in, out);
-			}
-
-			System.load(libraryPath);
-			loadedLibrarySet.add(filename);
-		}
-	}
-
-	/**
-	 * Copies all data from the given input stream to the given output stream.
-	 * 
-	 * @param in
-	 *        the input stream to read data from
-	 * @param out
-	 *        the output stream to write data to
-	 * @throws IOException
-	 *         thrown if an I/O error occurs while copying the data
-	 */
-	private static void copy(final InputStream in, final OutputStream out) throws IOException {
-
-		final byte[] buf = new byte[BUFSIZE];
-		int len = 0;
-		while (true) {
-			len = in.read(buf);
-			if (len <= 0) {
-				break;
-			}
-			out.write(buf, 0, len);
-		}
-	}
-
-	/**
-	 * Checks if a native library is already loaded.
-	 * 
-	 * @param libraryName
-	 *        the filename of the library to check
-	 * @return <code>true</code> if the library is already loaded, <code>false</code> otherwise
-	 */
-	public static boolean isLibraryLoaded(final String libraryName) {
-
-		synchronized (loadedLibrarySet) {
-			return loadedLibrarySet.contains(libraryName);
-		}
-
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
index 7b3da6a..e55012b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
@@ -16,20 +16,19 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
-import java.io.IOException;
+import org.junit.Test;
 
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
-import org.junit.Test;
 
 /**
  * This class contains tests for the {@link org.apache.flink.runtime.AbstractID} class.
- * 
  */
 public class AbstractIDTest {
 	/**
@@ -51,7 +50,6 @@ public class AbstractIDTest {
 	 */
 	@Test
 	public void testSerialization() {
-
 		final ChannelID origID = new ChannelID();
 		try {
 			final ChannelID copyID = (ChannelID) CommonTestUtils.createCopyWritable(origID);
@@ -59,8 +57,73 @@ public class AbstractIDTest {
 			assertEquals(origID.hashCode(), copyID.hashCode());
 			assertEquals(origID, copyID);
 
-		} catch (IOException e) {
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCompare() {
+		try {
+			AbstractID id1 = new AbstractID(0, 0);
+			AbstractID id2 = new AbstractID(1, 0);
+			AbstractID id3 = new AbstractID(0, 1);
+			AbstractID id4 = new AbstractID(-1, 0);
+			AbstractID id5 = new AbstractID(0, -1);
+			AbstractID id6 = new AbstractID(-1, -1);
+			
+			AbstractID id7 = new AbstractID(Long.MAX_VALUE, Long.MAX_VALUE);
+			AbstractID id8 = new AbstractID(Long.MIN_VALUE, Long.MIN_VALUE);
+			AbstractID id9 = new AbstractID(Long.MAX_VALUE, Long.MIN_VALUE);
+			AbstractID id10 = new AbstractID(Long.MIN_VALUE, Long.MAX_VALUE);
+			
+			// test self equality
+			assertEquals(0, id1.compareTo(CommonTestUtils.createCopyWritable(id1)));
+			assertEquals(0, id2.compareTo(CommonTestUtils.createCopyWritable(id2)));
+			assertEquals(0, id3.compareTo(CommonTestUtils.createCopyWritable(id3)));
+			assertEquals(0, id4.compareTo(CommonTestUtils.createCopyWritable(id4)));
+			assertEquals(0, id5.compareTo(CommonTestUtils.createCopyWritable(id5)));
+			assertEquals(0, id6.compareTo(CommonTestUtils.createCopyWritable(id6)));
+			assertEquals(0, id7.compareTo(CommonTestUtils.createCopyWritable(id7)));
+			assertEquals(0, id8.compareTo(CommonTestUtils.createCopyWritable(id8)));
+			assertEquals(0, id9.compareTo(CommonTestUtils.createCopyWritable(id9)));
+			assertEquals(0, id10.compareTo(CommonTestUtils.createCopyWritable(id10)));
+			
+			// test order
+			assertCompare(id1, id2, -1);
+			assertCompare(id1, id3, -1);
+			assertCompare(id1, id4, 1);
+			assertCompare(id1, id5, 1);
+			assertCompare(id1, id6, 1);
+			assertCompare(id2, id5, 1);
+			assertCompare(id3, id5, 1);
+			assertCompare(id2, id6, 1);
+			assertCompare(id3, id6, 1);
+			assertCompare(id1, id7, -1);
+			assertCompare(id1, id8, 1);
+			assertCompare(id7, id8, 1);
+			assertCompare(id9, id10, -1);
+			assertCompare(id7, id9, 1);
+			assertCompare(id7, id10, 1);
+			assertCompare(id8, id9, -1);
+			assertCompare(id8, id10, -1);
+		}
+		catch (Exception e) {
 			e.printStackTrace();
+			fail(e.getMessage());
 		}
 	}
+	
+	private static void assertCompare(AbstractID a, AbstractID b, int signum) {
+		int cmpAB = a.compareTo(b);
+		int cmpBA = b.compareTo(a);
+		
+		int sgnAB = cmpAB > 0 ? 1 : (cmpAB < 0 ? -1 : 0);
+		int sgnBA = cmpBA > 0 ? 1 : (cmpBA < 0 ? -1 : 0);
+		
+		assertEquals(signum, sgnAB);
+		assertTrue(sgnAB == -sgnBA);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/instance/HardwareTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/HardwareTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/HardwareTest.java
index 13834b1..b123a7f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/HardwareTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/HardwareTest.java
@@ -1,17 +1,20 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.instance;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java
index 2b83c84..0374d62 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceConnectionInfoTest.java
@@ -1,17 +1,20 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.instance;
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
new file mode 100644
index 0000000..bcb0cf5
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/InstanceTest.java
@@ -0,0 +1,174 @@
+/**
+ * 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.instance;
+
+import static org.junit.Assert.*;
+
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.junit.Test;
+
+/**
+ * Tests for the {@link Instance} class.
+ */
+public class InstanceTest {
+
+	@Test
+	public void testAllocatingAndCancellingSlots() {
+		try {
+			HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024);
+			InetAddress address = InetAddress.getByName("127.0.0.1");
+			InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10000, 10001);
+			
+			Instance instance = new Instance(connection, new InstanceID(), hardwareDescription, 4);
+			
+			assertEquals(4, instance.getTotalNumberOfSlots());
+			assertEquals(4, instance.getNumberOfAvailableSlots());
+			assertEquals(0, instance.getNumberOfAllocatedSlots());
+			
+			AllocatedSlot slot1 = instance.allocateSlot(new JobID());
+			AllocatedSlot slot2 = instance.allocateSlot(new JobID());
+			AllocatedSlot slot3 = instance.allocateSlot(new JobID());
+			AllocatedSlot slot4 = instance.allocateSlot(new JobID());
+			
+			assertNotNull(slot1);
+			assertNotNull(slot2);
+			assertNotNull(slot3);
+			assertNotNull(slot4);
+			
+			assertEquals(0, instance.getNumberOfAvailableSlots());
+			assertEquals(4, instance.getNumberOfAllocatedSlots());
+			assertEquals(6, slot1.getSlotNumber() + slot2.getSlotNumber() + 
+					slot3.getSlotNumber() + slot4.getSlotNumber());
+			
+			// no more slots
+			assertNull(instance.allocateSlot(new JobID()));
+			try {
+				instance.returnAllocatedSlot(slot2);
+				fail("instance accepted a non-cancelled slot.");
+			} catch (IllegalArgumentException e) {
+				// good
+			}
+			
+			// release the slots. this returns them to the instance
+			slot1.releaseSlot();
+			slot2.releaseSlot();
+			assertFalse(instance.returnAllocatedSlot(slot1));
+			assertFalse(instance.returnAllocatedSlot(slot2));
+			
+			// cancel some slots. this does not release them, yet
+			slot3.cancel();
+			slot4.cancel();
+			assertTrue(instance.returnAllocatedSlot(slot3));
+			assertTrue(instance.returnAllocatedSlot(slot4));
+			
+			assertEquals(4, instance.getNumberOfAvailableSlots());
+			assertEquals(0, instance.getNumberOfAllocatedSlots());
+			
+			assertFalse(instance.returnAllocatedSlot(slot1));
+			assertFalse(instance.returnAllocatedSlot(slot2));
+			assertFalse(instance.returnAllocatedSlot(slot3));
+			assertFalse(instance.returnAllocatedSlot(slot4));
+			
+			assertEquals(4, instance.getNumberOfAvailableSlots());
+			assertEquals(0, instance.getNumberOfAllocatedSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testInstanceDies() {
+		try {
+			HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024);
+			InetAddress address = InetAddress.getByName("127.0.0.1");
+			InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10000, 10001);
+			
+			Instance instance = new Instance(connection, new InstanceID(), hardwareDescription, 3);
+			
+			assertEquals(3, instance.getNumberOfAvailableSlots());
+			
+			AllocatedSlot slot1 = instance.allocateSlot(new JobID());
+			AllocatedSlot slot2 = instance.allocateSlot(new JobID());
+			AllocatedSlot slot3 = instance.allocateSlot(new JobID());
+			
+			instance.markDead();
+			
+			assertEquals(0, instance.getNumberOfAllocatedSlots());
+			assertEquals(0, instance.getNumberOfAvailableSlots());
+			
+			assertTrue(slot1.isCanceled());
+			assertTrue(slot2.isCanceled());
+			assertTrue(slot3.isCanceled());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCancelAllSlots() {
+		try {
+			HardwareDescription hardwareDescription = new HardwareDescription(4, 2L*1024*1024*1024, 1024*1024*1024, 512*1024*1024);
+			InetAddress address = InetAddress.getByName("127.0.0.1");
+			InstanceConnectionInfo connection = new InstanceConnectionInfo(address, 10000, 10001);
+			
+			Instance instance = new Instance(connection, new InstanceID(), hardwareDescription, 3);
+			
+			assertEquals(3, instance.getNumberOfAvailableSlots());
+			
+			AllocatedSlot slot1 = instance.allocateSlot(new JobID());
+			AllocatedSlot slot2 = instance.allocateSlot(new JobID());
+			AllocatedSlot slot3 = instance.allocateSlot(new JobID());
+			
+			instance.cancelAndReleaseAllSlots();
+			
+			assertEquals(3, instance.getNumberOfAvailableSlots());
+			
+			assertTrue(slot1.isCanceled());
+			assertTrue(slot2.isCanceled());
+			assertTrue(slot3.isCanceled());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	/**
+	 * It is crucial for some portions of the code that instance objects do not override equals and
+	 * are only considered equal, if the references are equal.
+	 */
+	@Test
+	public void testInstancesReferenceEqual() {
+		try {
+			Method m = Instance.class.getMethod("equals", Object.class);
+			assertTrue(m.getDeclaringClass() == Object.class);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
index b047222..bce692d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/instance/LocalInstanceManagerTest.java
@@ -25,7 +25,6 @@ import java.util.List;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.GlobalConfiguration;
-import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.runtime.ExecutionMode;
 import org.apache.flink.runtime.accumulators.AccumulatorEvent;
 import org.apache.flink.runtime.client.JobCancelResult;
@@ -239,9 +238,6 @@ public class LocalInstanceManagerTest {
 		public void killTask(JobID jobID, ManagementVertexID id) {}
 
 		@Override
-		public void killInstance(StringRecord instanceName) {}
-
-		@Override
 		public void logBufferUtilization(JobID jobID) {}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java
deleted file mode 100644
index 3a24f97..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultSchedulerTest.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/**
- * 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.jobmanager.scheduler;
-
-import static org.junit.Assert.*;
-
-import java.net.InetAddress;
-import java.net.UnknownHostException;
-
-import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
-import org.apache.flink.runtime.instance.AllocatedSlot;
-import org.apache.flink.runtime.instance.HardwareDescription;
-import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.InstanceConnectionInfo;
-import org.apache.flink.runtime.instance.InstanceID;
-import org.apache.flink.runtime.jobgraph.JobID;
-import org.junit.Test;
-
-/**
- * This class checks the functionality of the {@link org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler} class
- */
-public class DefaultSchedulerTest {
-
-	private int portNum = 10000;
-	
-	@Test
-	public void testAddAndRemoveInstance() {
-		try {
-			DefaultScheduler scheduler = new DefaultScheduler();
-			
-			Instance i1 = getRandomInstance(2);
-			Instance i2 = getRandomInstance(2);
-			Instance i3 = getRandomInstance(2);
-			
-			assertEquals(0, scheduler.getNumberOfAvailableInstances());
-			scheduler.newInstanceAvailable(i1);
-			assertEquals(1, scheduler.getNumberOfAvailableInstances());
-			scheduler.newInstanceAvailable(i2);
-			assertEquals(2, scheduler.getNumberOfAvailableInstances());
-			scheduler.newInstanceAvailable(i3);
-			assertEquals(3, scheduler.getNumberOfAvailableInstances());
-			
-			// cannot add available instance again
-			try {
-				scheduler.newInstanceAvailable(i2);
-				fail("Scheduler accepted instance twice");
-			}
-			catch (IllegalArgumentException e) {
-				// bueno!
-			}
-			
-			// some instances die
-			assertEquals(3, scheduler.getNumberOfAvailableInstances());
-			scheduler.instanceDied(i2);
-			assertEquals(2, scheduler.getNumberOfAvailableInstances());
-			
-			// try to add a dead instance
-			try {
-				scheduler.newInstanceAvailable(i2);
-				fail("Scheduler accepted dead instance");
-			}
-			catch (IllegalArgumentException e) {
-				// stimmt
-				
-			}
-						
-			scheduler.instanceDied(i1);
-			assertEquals(1, scheduler.getNumberOfAvailableInstances());
-			scheduler.instanceDied(i3);
-			assertEquals(0, scheduler.getNumberOfAvailableInstances());
-			
-			assertFalse(i1.isAlive());
-			assertFalse(i2.isAlive());
-			assertFalse(i3.isAlive());
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	
-	@Test
-	public void testAssignToSlots() {
-		try {
-			final JobID jobId = new JobID();
-			
-			DefaultScheduler scheduler = new DefaultScheduler();
-
-			scheduler.newInstanceAvailable(getRandomInstance(2));
-			scheduler.newInstanceAvailable(getRandomInstance(2));
-			scheduler.newInstanceAvailable(getRandomInstance(2));
-			
-			ResourceId id1 = new ResourceId();
-			ResourceId id2 = new ResourceId();
-			ResourceId id3 = new ResourceId();
-			ResourceId id4 = new ResourceId();
-			ResourceId id5 = new ResourceId();
-			ResourceId id6 = new ResourceId();
-			
-			AllocatedSlot s1 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id1), true);
-			AllocatedSlot s2 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id2), true);
-			AllocatedSlot s3 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id3), true);
-			AllocatedSlot s4 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id4), true);
-			AllocatedSlot s5 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id5), true);
-			AllocatedSlot s6 = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id6), true);
-			
-			// no more slots available, the next call should throw an exception
-			try {
-				scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), new ResourceId()), true);
-				fail("Scheduler accepted scheduling request without available resource.");
-			}
-			catch (NoResourceAvailableException e) {
-				// expected
-			}
-			
-			// schedule something into the same slots as before
-			AllocatedSlot s1s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id1), true);
-			AllocatedSlot s2s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id2), true);
-			AllocatedSlot s3s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id3), true);
-			AllocatedSlot s4s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id4), true);
-			AllocatedSlot s5s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id5), true);
-			AllocatedSlot s6s = scheduler.getResourceToScheduleUnit(new ScheduledUnit(jobId, getDummyVertex(), id6), true);
-			
-			assertEquals(s1, s1s);
-			assertEquals(s2, s2s);
-			assertEquals(s3, s3s);
-			assertEquals(s4, s4s);
-			assertEquals(s5, s5s);
-			assertEquals(s6, s6s);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			fail(e.getMessage());
-		}
-	}
-	
-	
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private Instance getRandomInstance(int numSlots) {
-		InetAddress address;
-		try {
-			address = InetAddress.getByName("127.0.0.1");
-		} catch (UnknownHostException e) {
-			throw new RuntimeException("Test could not create IP address for localhost loopback.");
-		}
-		
-		int ipcPort = portNum++;
-		int dataPort = portNum++;
-		
-		InstanceConnectionInfo ci = new InstanceConnectionInfo(address, ipcPort, dataPort);
-		
-		final long GB = 1024L*1024*1024;
-		HardwareDescription resources = new HardwareDescription(4, 4*GB, 3*GB, 2*GB);
-		
-		return new Instance(ci, new InstanceID(), resources, numSlots);
-	}
-	
-	private ExecutionVertex2 getDummyVertex() {
-		return new ExecutionVertex2();
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
new file mode 100644
index 0000000..10fca8a
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerIsolatedTasksTest.java
@@ -0,0 +1,385 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.areAllDistinct;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getDummyVertex;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance;
+
+import static org.junit.Assert.*;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.instance.Instance;
+
+/**
+ * Tests for the {@link DefaultScheduler} when scheduling individual tasks.
+ */
+public class SchedulerIsolatedTasksTest {
+	
+	@Test
+	public void testAddAndRemoveInstance() {
+		try {
+			DefaultScheduler scheduler = new DefaultScheduler();
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			Instance i3 = getRandomInstance(2);
+			
+			assertEquals(0, scheduler.getNumberOfAvailableInstances());
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+			scheduler.newInstanceAvailable(i1);
+			assertEquals(1, scheduler.getNumberOfAvailableInstances());
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			scheduler.newInstanceAvailable(i2);
+			assertEquals(2, scheduler.getNumberOfAvailableInstances());
+			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+			scheduler.newInstanceAvailable(i3);
+			assertEquals(3, scheduler.getNumberOfAvailableInstances());
+			assertEquals(6, scheduler.getNumberOfAvailableSlots());
+			
+			// cannot add available instance again
+			try {
+				scheduler.newInstanceAvailable(i2);
+				fail("Scheduler accepted instance twice");
+			}
+			catch (IllegalArgumentException e) {
+				// bueno!
+			}
+			
+			// some instances die
+			assertEquals(3, scheduler.getNumberOfAvailableInstances());
+			assertEquals(6, scheduler.getNumberOfAvailableSlots());
+			scheduler.instanceDied(i2);
+			assertEquals(2, scheduler.getNumberOfAvailableInstances());
+			assertEquals(4, scheduler.getNumberOfAvailableSlots());
+			
+			// try to add a dead instance
+			try {
+				scheduler.newInstanceAvailable(i2);
+				fail("Scheduler accepted dead instance");
+			}
+			catch (IllegalArgumentException e) {
+				// stimmt
+				
+			}
+						
+			scheduler.instanceDied(i1);
+			assertEquals(1, scheduler.getNumberOfAvailableInstances());
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			scheduler.instanceDied(i3);
+			assertEquals(0, scheduler.getNumberOfAvailableInstances());
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+			
+			assertFalse(i1.isAlive());
+			assertFalse(i2.isAlive());
+			assertFalse(i3.isAlive());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testScheduleImmediately() {
+		try {
+			DefaultScheduler scheduler = new DefaultScheduler();
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+			
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			scheduler.newInstanceAvailable(getRandomInstance(1));
+			scheduler.newInstanceAvailable(getRandomInstance(2));
+			assertEquals(5, scheduler.getNumberOfAvailableSlots());
+			
+			// schedule something into all slots
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			
+			// the slots should all be different
+			assertTrue(areAllDistinct(s1, s2, s3, s4, s5));
+			
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+				fail("Scheduler accepted scheduling request without available resource.");
+			}
+			catch (NoResourceAvailableException e) {
+				// pass!
+			}
+			
+			// release some slots again
+			s3.releaseSlot();
+			s4.releaseSlot();
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			// now we can schedule some more slots
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+			
+			assertTrue(areAllDistinct(s1, s2, s3, s4, s5, s6, s7));
+			
+			// release all
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s5.releaseSlot();
+			s6.releaseSlot();
+			s7.releaseSlot();
+			
+			assertEquals(5, scheduler.getNumberOfAvailableSlots());
+			
+			// check that slots that are released twice (accidentally) do not mess things up
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s5.releaseSlot();
+			s6.releaseSlot();
+			s7.releaseSlot();
+			
+			assertEquals(5, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testScheduleQueueing() {
+		final int NUM_INSTANCES = 50;
+		final int NUM_SLOTS_PER_INSTANCE = 3;
+		final int NUM_TASKS_TO_SCHEDULE = 2000;
+		
+		try {
+			DefaultScheduler scheduler = new DefaultScheduler();
+			
+			for (int i = 0;i < NUM_INSTANCES; i++) {
+				scheduler.newInstanceAvailable(getRandomInstance((int) (Math.random() * NUM_SLOTS_PER_INSTANCE) + 1));
+			}
+			
+			assertEquals(NUM_INSTANCES, scheduler.getNumberOfAvailableInstances());
+			final int totalSlots = scheduler.getNumberOfAvailableSlots();
+			
+			// all slots we ever got.
+			List<SlotAllocationFuture> allAllocatedSlots = new ArrayList<SlotAllocationFuture>();
+			
+			// slots that need to be released
+			final Set<AllocatedSlot> toRelease = new HashSet<AllocatedSlot>();
+			
+			// flag to track errors in the concurrent thread
+			final AtomicBoolean errored = new AtomicBoolean(false);
+			
+			
+			SlotAllocationFutureAction action = new SlotAllocationFutureAction() {
+				@Override
+				public void slotAllocated(AllocatedSlot slot) {
+					synchronized (toRelease) {
+						toRelease.add(slot);
+						toRelease.notifyAll();
+					}
+				}
+			};
+			
+			// thread to asynchronously release slots
+			Runnable disposer = new Runnable() {
+				
+				@Override
+				public void run() {
+					try {
+						int recycled = 0;
+						while (recycled < NUM_TASKS_TO_SCHEDULE) {
+							synchronized (toRelease) {
+								while (toRelease.isEmpty()) {
+									toRelease.wait();
+								}
+								
+								Iterator<AllocatedSlot> iter = toRelease.iterator();
+								AllocatedSlot next = iter.next();
+								iter.remove();
+								
+								next.releaseSlot();
+								recycled++;
+							}
+						}
+					}
+					catch (Throwable t) {
+						errored.set(true);
+					}
+				}
+			};
+			
+			Thread disposeThread = new Thread(disposer);
+			disposeThread.start();
+			
+			for (int i = 0; i < NUM_TASKS_TO_SCHEDULE; i++) {
+				SlotAllocationFuture future = scheduler.scheduleQueued(new ScheduledUnit(getDummyVertex()));
+				future.setFutureAction(action);
+				allAllocatedSlots.add(future);
+			}
+
+			disposeThread.join();
+			
+			assertFalse("The slot releasing thread caused an error.", errored.get());
+			
+			List<AllocatedSlot> slotsAfter = new ArrayList<AllocatedSlot>();
+			for (SlotAllocationFuture future : allAllocatedSlots) {
+				slotsAfter.add(future.waitTillAllocated());
+			}
+			
+			// the slots should all be different
+			assertTrue(areAllDistinct(slotsAfter.toArray()));
+			
+			assertEquals(totalSlots, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testScheduleWithDyingInstances() {
+		try {
+			DefaultScheduler scheduler = new DefaultScheduler();
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			Instance i3 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i3);
+			
+			List<AllocatedSlot> slots = new ArrayList<AllocatedSlot>();
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
+			slots.add(scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex())));
+			
+			i2.markDead();
+			
+			for (AllocatedSlot slot : slots) {
+				if (slot.getInstance() == i2) {
+					assertTrue(slot.isCanceled());
+				} else {
+					assertFalse(slot.isCanceled());
+				}
+				
+				slot.releaseSlot();
+			}
+			
+			assertEquals(3, scheduler.getNumberOfAvailableSlots());
+			
+			i1.markDead();
+			i3.markDead();
+			
+			// cannot get another slot, since all instances are dead
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getDummyVertex()));
+				fail("Scheduler served a slot from a dead instance");
+			}
+			catch (NoResourceAvailableException e) {
+				// fine
+			}
+			catch (Exception e) {
+				fail("Wrong exception type.");
+			}
+			
+			// now the latest, the scheduler should have noticed (through the lazy mechanisms)
+			// that all instances have vanished
+			assertEquals(0, scheduler.getNumberOfInstancesWithAvailableSlots());
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSchedulingLocation() {
+		try {
+			DefaultScheduler scheduler = new DefaultScheduler();
+			
+			Instance i1 = getRandomInstance(2);
+			Instance i2 = getRandomInstance(2);
+			Instance i3 = getRandomInstance(2);
+			
+			scheduler.newInstanceAvailable(i1);
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i3);
+			
+			// schedule something on an arbitrary instance
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Collections.<Instance>emptyList())));
+			
+			// figure out how we use the location hints
+			Instance first = s1.getInstance();
+			Instance second = first != i1 ? i1 : i2;
+			Instance third = first == i3 ? i2 : i3;
+			
+			// something that needs to go to the first instance again
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Collections.singletonList(s1.getInstance()))));
+			assertEquals(first, s2.getInstance());
+
+			// first or second --> second, because first is full
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, second))));
+			assertEquals(second, s3.getInstance());
+			
+			// first or third --> third (because first is full)
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third))));
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third))));
+			assertEquals(third, s4.getInstance());
+			assertEquals(third, s5.getInstance());
+			
+			// first or third --> second, because all others are full
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third))));
+			assertEquals(second, s6.getInstance());
+			
+			// release something on the first and second instance
+			s2.releaseSlot();
+			s6.releaseSlot();
+			
+			AllocatedSlot s7 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(Arrays.asList(first, third))));
+			assertEquals(first, s7.getInstance());
+			
+			assertEquals(1, scheduler.getNumberOfUnconstrainedAssignments());
+			assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(5, scheduler.getNumberOfLocalizedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}
\ No newline at end of file


[33/63] [abbrv] git commit: Better error messages at TaskManager startup and registration

Posted by se...@apache.org.
Better error messages at TaskManager startup and registration


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/8998a30e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/8998a30e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/8998a30e

Branch: refs/heads/master
Commit: 8998a30ea036e6770c9a6b6442ffe01e64720c66
Parents: d5d3a08
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Sep 15 16:51:12 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:49 2014 +0200

----------------------------------------------------------------------
 .../apache/flink/runtime/taskmanager/TaskManager.java    | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/8998a30e/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
index 1fd5a71..d6494a7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/TaskManager.java
@@ -51,7 +51,6 @@ import org.apache.commons.cli.Option;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.cli.Options;
 import org.apache.commons.cli.ParseException;
-
 import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.api.common.cache.DistributedCache.DistributedCacheEntry;
 import org.apache.flink.configuration.ConfigConstants;
@@ -97,7 +96,6 @@ import org.apache.flink.runtime.util.EnvironmentInformation;
 import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.util.ExceptionUtils;
 import org.apache.flink.util.StringUtils;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -763,8 +761,12 @@ public class TaskManager implements TaskOperationProtocol {
 					try {
 						resultId = this.jobManager.registerTaskManager(this.localInstanceConnectionInfo,
 								this.hardwareDescription, this.numberOfSlots);
+						
+						if (resultId == null) {
+							throw new Exception("Registration attempt refused by JobManager.");
+						}
 					}
-					catch (IOException e) {
+					catch (Exception e) {
 						// this may be if the job manager was not yet online
 						// if this has happened for a while, report it. if it has just happened
 						// at the very beginning, this may not mean anything (JM still in startup)
@@ -780,9 +782,6 @@ public class TaskManager implements TaskOperationProtocol {
 						// success
 						this.registeredId = resultId;
 						break;
-					} else {
-						// this is bad. The job manager refused us. report and try again later
-						LOG.error("Registration attempt refused by JobManager.");
 					}
 		
 					try {


[16/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/FormatUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/FormatUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/io/FormatUtil.java
deleted file mode 100644
index 2f89a80..0000000
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/FormatUtil.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/**
- * 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.api.common.io;
-
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FileInputSplit;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.fs.FileSystem.WriteMode;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.util.ReflectionUtil;
-
-/**
- * Provides convenience methods to deal with I/O operations related to {@link InputFormat} and {@link OutputFormat}.
- */
-public class FormatUtil {
-
-
-	/**
-	 * Creates an {@link InputFormat} from a given class for the specified file. The optional {@link Configuration}
-	 * initializes the format.
-	 * 
-	 * @param <T>
-	 *        the class of the InputFormat
-	 * @param inputFormatClass
-	 *        the class of the InputFormat
-	 * @param path
-	 *        the path of the file
-	 * @param configuration
-	 *        optional configuration of the InputFormat
-	 * @return the created {@link InputFormat}
-	 * @throws IOException
-	 *         if an I/O error occurred while accessing the file or initializing the InputFormat.
-	 */
-	public static <T, F extends FileInputFormat<T>> F openInput(
-			Class<F> inputFormatClass, String path, Configuration configuration)
-		throws IOException
-	{
-		configuration = configuration == null ? new Configuration() : configuration;
-
-		Path normalizedPath = normalizePath(new Path(path));
-		final F inputFormat = ReflectionUtil.newInstance(inputFormatClass);
-
-		inputFormat.setFilePath(normalizedPath);
-		inputFormat.setOpenTimeout(0);
-		inputFormat.configure(configuration);
-
-		final FileSystem fs = FileSystem.get(normalizedPath.toUri());
-		FileStatus fileStatus = fs.getFileStatus(normalizedPath);
-
-		BlockLocation[] blocks = fs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen());
-		inputFormat.open(new FileInputSplit(0, new Path(path), 0, fileStatus.getLen(), blocks[0].getHosts()));
-		return inputFormat;
-	}
-
-	/**
-	 * Creates {@link InputFormat}s from a given class for the specified file(s). The optional {@link Configuration}
-	 * initializes the formats.
-	 * 
-	 * @param <T>
-	 *        the class of the InputFormat
-	 * @param inputFormatClass
-	 *        the class of the InputFormat
-	 * @param path
-	 *        the path of the file or to the directory containing the splits
-	 * @param configuration
-	 *        optional configuration of the InputFormat
-	 * @return the created {@link InputFormat}s for each file in the specified path
-	 * @throws IOException
-	 *         if an I/O error occurred while accessing the files or initializing the InputFormat.
-	 */
-	@SuppressWarnings("unchecked")
-	public static <T, F extends FileInputFormat<T>> List<F> openAllInputs(
-			Class<F> inputFormatClass, String path, Configuration configuration) throws IOException {
-		Path nephelePath = new Path(path);
-		FileSystem fs = nephelePath.getFileSystem();
-		FileStatus fileStatus = fs.getFileStatus(nephelePath);
-		if (!fileStatus.isDir()) {
-			return Arrays.asList(openInput(inputFormatClass, path, configuration));
-		}
-		FileStatus[] list = fs.listStatus(nephelePath);
-		List<F> formats = new ArrayList<F>();
-		for (int index = 0; index < list.length; index++) {
-			formats.add(openInput(inputFormatClass, list[index].getPath().toString(), configuration));
-		}
-		return formats;
-	}
-
-	/**
-	 * Creates an {@link InputFormat} from a given class. The optional {@link Configuration}
-	 * initializes the format.
-	 * 
-	 * @param <T>
-	 *        the class of the InputFormat
-	 * @param inputFormatClass
-	 *        the class of the InputFormat
-	 * @param configuration
-	 *        optional configuration of the InputFormat
-	 * @return the created {@link InputFormat}
-	 * @throws IOException
-	 *         if an I/O error occurred while accessing the file or initializing the InputFormat.
-	 */
-	public static <T, IS extends InputSplit, F extends InputFormat<T, IS>> F openInput(
-			Class<F> inputFormatClass, Configuration configuration) throws IOException {
-		configuration = configuration == null ? new Configuration() : configuration;
-
-		final F inputFormat = ReflectionUtil.newInstance(inputFormatClass);
-		inputFormat.configure(configuration);
-		final IS[] splits = inputFormat.createInputSplits(1);
-		inputFormat.open(splits[0]);
-		return inputFormat;
-	}
-	
-	/**
-	 * Creates an {@link OutputFormat} from a given class for the specified file. The optional {@link Configuration}
-	 * initializes the format.
-	 * 
-	 * @param <T>
-	 *        the class of the OutputFormat
-	 * @param outputFormatClass
-	 *        the class of the OutputFormat
-	 * @param path
-	 *        the path of the file or to the directory containing the splits
-	 * @param configuration
-	 *        optional configuration of the OutputFormat
-	 * @return the created {@link OutputFormat}
-	 * @throws IOException
-	 *         if an I/O error occurred while accessing the file or initializing the OutputFormat.
-	 */
-	public static <T, F extends FileOutputFormat<? extends T>> F openOutput(
-			Class<F> outputFormatClass, String path, Configuration configuration) 
-		throws IOException
-	{
-		final F outputFormat = ReflectionUtil.newInstance(outputFormatClass);
-		outputFormat.setOutputFilePath(new Path(path));
-		outputFormat.setWriteMode(WriteMode.OVERWRITE);
-	
-		configuration = configuration == null ? new Configuration() : configuration;
-		
-		outputFormat.configure(configuration);
-		outputFormat.open(0, 1);
-		return outputFormat;
-	}
-
-	/**
-	 * Fixes the path if it denotes a local (relative) file without the proper protocol prefix.
-	 */
-	private static Path normalizePath(Path path) {
-		URI uri = path.toUri();
-		if (uri.getScheme() == null) {
-			try {
-				uri = new URI("file", uri.getHost(), uri.getPath(), uri.getFragment());
-				path = new Path(uri.toString());
-			} catch (URISyntaxException e) {
-				throw new IllegalArgumentException("path is invalid", e);
-			}
-		}
-		return path;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java
index 0ddeb64..c108471 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/GenericInputFormat.java
@@ -50,7 +50,6 @@ public abstract class GenericInputFormat<OT> implements InputFormat<OT, GenericI
 		return cachedStatistics;
 	}
 
-
 	@Override
 	public GenericInputSplit[] createInputSplits(int numSplits) throws IOException {
 		if (numSplits < 1) {
@@ -66,8 +65,8 @@ public abstract class GenericInputFormat<OT> implements InputFormat<OT, GenericI
 	}
 	
 	@Override
-	public Class<? extends GenericInputSplit> getInputSplitType() {
-		return GenericInputSplit.class;
+	public DefaultInputSplitAssigner getInputSplitAssigner(GenericInputSplit[] splits) {
+		return new DefaultInputSplitAssigner(splits);
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java b/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
index 1bb7815..5eaa657 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/InitializeOnMaster.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.common.io;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java
index cb4019c..6845237 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/InputFormat.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.common.io;
 
 import java.io.IOException;
@@ -25,6 +24,8 @@ import java.io.Serializable;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.core.io.InputSplitSource;
 
 /**
  * The base interface for data sources that produces records.
@@ -59,7 +60,7 @@ import org.apache.flink.core.io.InputSplit;
  * @param <OT> The type of the produced records.
  * @param <T> The type of input split.
  */
-public interface InputFormat<OT, T extends InputSplit> extends Serializable {
+public interface InputFormat<OT, T extends InputSplit> extends InputSplitSource<T>, Serializable {
 	
 	/**
 	 * Configures this input format. Since input formats are instantiated generically and hence parameterless, 
@@ -95,6 +96,7 @@ public interface InputFormat<OT, T extends InputSplit> extends Serializable {
 	 * 
 	 * @throws IOException Thrown, when the creation of the splits was erroneous.
 	 */
+	@Override
 	T[] createInputSplits(int minNumSplits) throws IOException;
 	
 	/**
@@ -102,7 +104,8 @@ public interface InputFormat<OT, T extends InputSplit> extends Serializable {
 	 * 
 	 * @return The type of the input splits.
 	 */
-	Class<? extends T> getInputSplitType();
+	@Override
+	InputSplitAssigner getInputSplitAssigner(T[] inputSplits);
 	
 	// --------------------------------------------------------------------------------------------
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.java b/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.java
new file mode 100644
index 0000000..6243681
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/LocatableInputSplitAssigner.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.api.common.io;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.apache.flink.core.io.LocatableInputSplit;
+
+/**
+ * The locatable input split assigner assigns to each host splits that are local, before assigning
+ * splits that are not local. 
+ */
+public final class LocatableInputSplitAssigner implements InputSplitAssigner {
+
+	private static final Logger LOG = LoggerFactory.getLogger(LocatableInputSplitAssigner.class);
+
+
+	private final Set<LocatableInputSplit> unassigned = new HashSet<LocatableInputSplit>();
+	
+	private final ConcurrentHashMap<String, List<LocatableInputSplit>> localPerHost = new ConcurrentHashMap<String, List<LocatableInputSplit>>();
+	
+	private int localAssignments;		// lock protected by the unassigned set lock
+	
+	private int remoteAssignments;		// lock protected by the unassigned set lock
+
+	// --------------------------------------------------------------------------------------------
+	
+	public LocatableInputSplitAssigner(Collection<LocatableInputSplit> splits) {
+		this.unassigned.addAll(splits);
+	}
+	
+	public LocatableInputSplitAssigner(LocatableInputSplit[] splits) {
+		Collections.addAll(this.unassigned, splits);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public LocatableInputSplit getNextInputSplit(String host) {
+		// for a null host, we return an arbitrary split
+		if (host == null) {
+			
+			synchronized (this.unassigned) {
+				Iterator<LocatableInputSplit> iter = this.unassigned.iterator();
+				if (iter.hasNext()) {
+					LocatableInputSplit next = iter.next();
+					iter.remove();
+					
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Assigning arbitrary split to null host.");
+					}
+					
+					remoteAssignments++;
+					return next;
+				} else {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("No more input splits remaining.");
+					}
+					return null;
+				}
+			}
+		}
+		
+		host = host.toLowerCase(Locale.US);
+		
+		// for any non-null host, we take the list of non-null splits
+		List<LocatableInputSplit> localSplits = this.localPerHost.get(host);
+		
+		// if we have no list for this host yet, create one
+		if (localSplits == null) {
+			localSplits = new ArrayList<LocatableInputSplit>(16);
+			
+			// lock the list, to be sure that others have to wait for that host's local list
+			synchronized (localSplits) {
+				List<LocatableInputSplit> prior = this.localPerHost.putIfAbsent(host, localSplits);
+				
+				// if someone else beat us in the case to create this list, then we do not populate this one, but
+				// simply work with that other list
+				if (prior == null) {
+					// we are the first, we populate
+					
+					// first, copy the remaining splits to release the lock on the set early
+					// because that is shared among threads
+					LocatableInputSplit[] remaining;
+					synchronized (this.unassigned) {
+						remaining = (LocatableInputSplit[]) this.unassigned.toArray(new LocatableInputSplit[this.unassigned.size()]);
+					}
+					
+					for (LocatableInputSplit is : remaining) {
+						if (isLocal(host, is.getHostnames())) {
+							localSplits.add(is);
+						}
+					}
+				}
+				else {
+					// someone else was faster
+					localSplits = prior;
+				}
+			}
+		}
+		
+		// at this point, we have a list of local splits (possibly empty)
+		// we need to make sure no one else operates in the current list (that protects against
+		// list creation races) and that the unassigned set is consistent
+		// NOTE: we need to obtain the locks in this order, strictly!!!
+		synchronized (localSplits) {
+			int size = localSplits.size();
+			if (size > 0) {
+				synchronized (this.unassigned) {
+					do {
+						--size;
+						LocatableInputSplit split = localSplits.remove(size);
+						if (this.unassigned.remove(split)) {
+							
+							if (LOG.isDebugEnabled()) {
+								LOG.debug("Assigning local split to host " + host);
+							}
+							
+							localAssignments++;
+							return split;
+						}
+					} while (size > 0);
+				}
+			}
+		}
+		
+		// we did not find a local split, return any
+		synchronized (this.unassigned) {
+			Iterator<LocatableInputSplit> iter = this.unassigned.iterator();
+			if (iter.hasNext()) {
+				LocatableInputSplit next = iter.next();
+				iter.remove();
+				
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("Assigning remote split to host " + host);
+				}
+				
+				remoteAssignments++;
+				return next;
+			} else {
+				if (LOG.isDebugEnabled()) {
+					LOG.debug("No more input splits remaining.");
+				}
+				return null;
+			}
+		}
+	}
+	
+	private static final boolean isLocal(String host, String[] hosts) {
+		if (host == null || hosts == null) {
+			return false;
+		}
+		
+		for (String h : hosts) {
+			if (h != null && host.equals(h.toLowerCase())) {
+				return true;
+			}
+		}
+		
+		return false;
+	}
+	
+	public int getNumberOfLocalAssignments() {
+		return localAssignments;
+	}
+	
+	public int getNumberOfRemoteAssignments() {
+		return remoteAssignments;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java b/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
index 157df71..d4fce5b 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/NonParallelInput.java
@@ -18,10 +18,9 @@
 
 package org.apache.flink.api.common.io;
 
-
 /**
  * This interface acts as a marker for input formats for inputs which cannot be split.
- * Data sources with a Sequential input formats are always executed with a degree-of-parallelism
+ * Data sources with a non-parallel input formats are always executed with a degree-of-parallelism
  * of one.
  * 
  * @see InputFormat

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java b/flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java
index cd02ac9..ddf9cbc 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/io/OutputFormat.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.common.io;
 
 import java.io.IOException;
@@ -24,7 +23,6 @@ import java.io.Serializable;
 
 import org.apache.flink.configuration.Configuration;
 
-
 /**
  * The base interface for outputs that consumes records. The output format
  * describes how to store the final records, for example in a file.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
index c4a69ba..ebee5d0 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileInputSplit.java
@@ -151,19 +151,12 @@ public class FileInputSplit extends LocatableInputSplit {
 		if (obj == this) {
 			return true;
 		}
-		else if (obj != null && super.equals(obj) && obj instanceof FileInputSplit) {
+		else if (obj != null && obj instanceof FileInputSplit && super.equals(obj)) {
 			FileInputSplit other = (FileInputSplit) obj;
 			
-			if (this.file != null) {
-				if (!this.file.equals(other.file)) {
-					return false;
-				}
-			}
-			else if (other.file != null) {
-				return false;
-			}
-			
-			return this.start == other.start && this.length == other.length;
+			return this.start == other.start &&
+					this.length == other.length &&
+					(this.file == null ? other.file == null : (other.file != null && this.file.equals(other.file)));
 		}
 		else {
 			return false;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java b/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
index 850ba1c..52018a1 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/GenericInputSplit.java
@@ -78,7 +78,7 @@ public class GenericInputSplit implements InputSplit, java.io.Serializable {
 	}
 
 	@Override
-	public void read(final DataInputView in) throws IOException {
+	public void read(DataInputView in) throws IOException {
 		this.partitionNumber = in.readInt();
 		this.totalNumberOfPartitions = in.readInt();
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/core/io/IOReadableWritable.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/IOReadableWritable.java b/flink-core/src/main/java/org/apache/flink/core/io/IOReadableWritable.java
index c054fb8..1a51207 100644
--- a/flink-core/src/main/java/org/apache/flink/core/io/IOReadableWritable.java
+++ b/flink-core/src/main/java/org/apache/flink/core/io/IOReadableWritable.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.io;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/core/io/InputSplitSource.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/io/InputSplitSource.java b/flink-core/src/main/java/org/apache/flink/core/io/InputSplitSource.java
new file mode 100644
index 0000000..256b9c7
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/io/InputSplitSource.java
@@ -0,0 +1,48 @@
+/**
+ * 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.core.io;
+
+/**
+ * InputSplitSources create {@link InputSplit}s that define portions of data to be produced
+ * by {@link org.apache.flink.api.common.io.InputFormat}s.
+ *
+ * @param <T> The type of the input splits created by the source.
+ */
+public interface InputSplitSource<T extends InputSplit> extends java.io.Serializable {
+
+	/**
+	 * Computes the input splits. The given minimum number of splits is a hint as to how
+	 * many splits are desired.
+	 *
+	 * @param minNumSplits Number of minimal input splits, as a hint.
+	 * @return An array of input splits.
+	 * 
+	 * @throws Exception Exceptions when creating the input splits may be forwarded and will cause the
+	 *                   execution to permanently fail.
+	 */
+	T[] createInputSplits(int minNumSplits) throws Exception;
+	
+	/**
+	 * Returns the assigner for the input splits. Assigner determines which parallel instance of the
+	 * input format gets which input split.
+	 *
+	 * @return The input split assigner.
+	 */
+	InputSplitAssigner getInputSplitAssigner(T[] inputSplits);
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java
new file mode 100644
index 0000000..25835f5
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStream.java
@@ -0,0 +1,43 @@
+/**
+ * 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.core.memory;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public final class DataInputViewStream extends InputStream {
+
+	private final DataInputView inputView;
+	
+	
+	public DataInputViewStream(DataInputView inputView) {
+		this.inputView = inputView;
+	}
+
+	
+	@Override
+	public int read() throws IOException {
+		return inputView.readByte();
+	}
+	
+	public int read(byte b[], int off, int len) throws IOException {
+		inputView.readFully(b, off, len);
+		return len;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStream.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStream.java
new file mode 100644
index 0000000..f3188aa
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStream.java
@@ -0,0 +1,43 @@
+/**
+ * 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.core.memory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+public final class DataOutputViewStream extends OutputStream {
+
+	private final DataOutputView outputView;
+	
+	
+	public DataOutputViewStream(DataOutputView outputView) {
+		this.outputView = outputView;
+	}
+
+
+	@Override
+	public void write(int b) throws IOException {
+		outputView.write(b);
+	}
+	
+	@Override
+	public void write(byte[] b, int off, int len) throws IOException {
+		outputView.write(b, off, len);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java b/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
index 00ad8e5..7e891bb 100644
--- a/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ClassUtils.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.util;
 
 import java.util.HashMap;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
new file mode 100644
index 0000000..8f9822e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/util/ExceptionUtils.java
@@ -0,0 +1,78 @@
+/**
+ * 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.util;
+
+import java.io.PrintWriter;
+import java.io.StringWriter;
+
+public class ExceptionUtils {
+
+	/**
+	 * Makes a string representation of the exception's stack trace.
+	 * 
+	 * @param e The exception to stringify.
+	 * @return A string with exception name and call stack.
+	 */
+	public static String stringifyException(final Throwable e) {
+		final StringWriter stm = new StringWriter();
+		final PrintWriter wrt = new PrintWriter(stm);
+		e.printStackTrace(wrt);
+		wrt.close();
+		return stm.toString();
+	}
+	
+	/**
+	 * Throws the given {@code Throwable} in scenarios where the signatures do not allow you to
+	 * throw arbitrary Throwables. Errors and RuntimeExceptions are thrown directly, other exceptions
+	 * are packed into runtime exceptions
+	 * 
+	 * @param t The throwable to be thrown.
+	 */
+	public static void rethrow(Throwable t) {
+		if (t instanceof Error) {
+			throw (Error) t;
+		}
+		else if (t instanceof RuntimeException) {
+			throw (RuntimeException) t;
+		}
+		else {
+			throw new RuntimeException(t);
+		}
+	}
+	
+	/**
+	 * Throws the given {@code Throwable} in scenarios where the signatures do not allow you to
+	 * throw arbitrary Throwables. Errors and RuntimeExceptions are thrown directly, other exceptions
+	 * are packed into a parent RuntimeEception.
+	 * 
+	 * @param t The throwable to be thrown.
+	 * @param parentMessage The message for the parent RuntimeException, if one is needed.
+	 */
+	public static void rethrow(Throwable t, String parentMessage) {
+		if (t instanceof Error) {
+			throw (Error) t;
+		}
+		else if (t instanceof RuntimeException) {
+			throw (RuntimeException) t;
+		}
+		else {
+			throw new RuntimeException(parentMessage, t);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java b/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
index 2bdbc60..755e188 100644
--- a/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/util/NumberSequenceIterator.java
@@ -21,13 +21,10 @@ package org.apache.flink.util;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 
-
-
-/**
- *
- */
 /**
- *
+ * The {@code NumberSequenceIterator} is an iterator that returns a sequence of numbers (as {@code Long})s.
+ * The iterator is splittable (as defined by {@link SplittableIterator}, i.e., it can be divided into multiple
+ * iterators that each return a subsequence of the number sequence.
  */
 public class NumberSequenceIterator implements SplittableIterator<Long> {
 	
@@ -38,7 +35,12 @@ public class NumberSequenceIterator implements SplittableIterator<Long> {
 	private long current;
 	
 	
-	
+	/**
+	 * Internal constructor to allow for empty iterators.
+	 * 
+	 * @param from The first number returned by the iterator.
+	 * @param to The last number returned by the iterator.
+	 */
 	public NumberSequenceIterator(long from, long to) {
 		if (from > to) {
 			throw new IllegalArgumentException("The 'to' value must not be smaller than the 'from' value.");
@@ -52,11 +54,11 @@ public class NumberSequenceIterator implements SplittableIterator<Long> {
 	/**
 	 * Internal constructor to allow for empty iterators.
 	 * 
-	 * @param from
-	 * @param to
-	 * @param mark
+	 * @param from The first number returned by the iterator.
+	 * @param to The last number returned by the iterator.
+	 * @param unused A dummy parameter to disambiguate the constructor.
 	 */
-	private NumberSequenceIterator(long from, long to, boolean mark) {
+	private NumberSequenceIterator(long from, long to, boolean unused) {
 		this.current = from;
 		this.to = to;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/util/SimpleStringUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/SimpleStringUtils.java b/flink-core/src/main/java/org/apache/flink/util/SimpleStringUtils.java
index c3e6362..cfbd14e 100644
--- a/flink-core/src/main/java/org/apache/flink/util/SimpleStringUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/SimpleStringUtils.java
@@ -16,14 +16,12 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.util;
 
 import java.io.Serializable;
 
 import org.apache.flink.types.StringValue;
 
-
 /**
  * Utility class for efficient string operations on strings. All methods in this class are
  * written to be optimized for efficiency and work only on strings whose characters are

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java b/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java
index a15c31c..7d50d12 100644
--- a/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java
+++ b/flink-core/src/main/java/org/apache/flink/util/TraversableOnceException.java
@@ -18,10 +18,17 @@
 
 package org.apache.flink.util;
 
+/**
+ * An exception, indicating that an {@link java.lang.Iterable} can only be traversed once, but has been attempted
+ * to traverse an additional time.
+ */
 public class TraversableOnceException extends RuntimeException {
 
 	private static final long serialVersionUID = 7636881584773577290L;
 
+	/**
+	 * Creates a new exception with a default message.
+	 */
 	public TraversableOnceException() {
 		super("The Iterable can be iterated over only once. Only the first call to 'iterator()' will succeed.");
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/test/java/org/apache/flink/api/common/io/FormatUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/io/FormatUtil.java b/flink-core/src/test/java/org/apache/flink/api/common/io/FormatUtil.java
new file mode 100644
index 0000000..2f89a80
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/api/common/io/FormatUtil.java
@@ -0,0 +1,186 @@
+/**
+ * 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.api.common.io;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.BlockLocation;
+import org.apache.flink.core.fs.FileInputSplit;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.util.ReflectionUtil;
+
+/**
+ * Provides convenience methods to deal with I/O operations related to {@link InputFormat} and {@link OutputFormat}.
+ */
+public class FormatUtil {
+
+
+	/**
+	 * Creates an {@link InputFormat} from a given class for the specified file. The optional {@link Configuration}
+	 * initializes the format.
+	 * 
+	 * @param <T>
+	 *        the class of the InputFormat
+	 * @param inputFormatClass
+	 *        the class of the InputFormat
+	 * @param path
+	 *        the path of the file
+	 * @param configuration
+	 *        optional configuration of the InputFormat
+	 * @return the created {@link InputFormat}
+	 * @throws IOException
+	 *         if an I/O error occurred while accessing the file or initializing the InputFormat.
+	 */
+	public static <T, F extends FileInputFormat<T>> F openInput(
+			Class<F> inputFormatClass, String path, Configuration configuration)
+		throws IOException
+	{
+		configuration = configuration == null ? new Configuration() : configuration;
+
+		Path normalizedPath = normalizePath(new Path(path));
+		final F inputFormat = ReflectionUtil.newInstance(inputFormatClass);
+
+		inputFormat.setFilePath(normalizedPath);
+		inputFormat.setOpenTimeout(0);
+		inputFormat.configure(configuration);
+
+		final FileSystem fs = FileSystem.get(normalizedPath.toUri());
+		FileStatus fileStatus = fs.getFileStatus(normalizedPath);
+
+		BlockLocation[] blocks = fs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen());
+		inputFormat.open(new FileInputSplit(0, new Path(path), 0, fileStatus.getLen(), blocks[0].getHosts()));
+		return inputFormat;
+	}
+
+	/**
+	 * Creates {@link InputFormat}s from a given class for the specified file(s). The optional {@link Configuration}
+	 * initializes the formats.
+	 * 
+	 * @param <T>
+	 *        the class of the InputFormat
+	 * @param inputFormatClass
+	 *        the class of the InputFormat
+	 * @param path
+	 *        the path of the file or to the directory containing the splits
+	 * @param configuration
+	 *        optional configuration of the InputFormat
+	 * @return the created {@link InputFormat}s for each file in the specified path
+	 * @throws IOException
+	 *         if an I/O error occurred while accessing the files or initializing the InputFormat.
+	 */
+	@SuppressWarnings("unchecked")
+	public static <T, F extends FileInputFormat<T>> List<F> openAllInputs(
+			Class<F> inputFormatClass, String path, Configuration configuration) throws IOException {
+		Path nephelePath = new Path(path);
+		FileSystem fs = nephelePath.getFileSystem();
+		FileStatus fileStatus = fs.getFileStatus(nephelePath);
+		if (!fileStatus.isDir()) {
+			return Arrays.asList(openInput(inputFormatClass, path, configuration));
+		}
+		FileStatus[] list = fs.listStatus(nephelePath);
+		List<F> formats = new ArrayList<F>();
+		for (int index = 0; index < list.length; index++) {
+			formats.add(openInput(inputFormatClass, list[index].getPath().toString(), configuration));
+		}
+		return formats;
+	}
+
+	/**
+	 * Creates an {@link InputFormat} from a given class. The optional {@link Configuration}
+	 * initializes the format.
+	 * 
+	 * @param <T>
+	 *        the class of the InputFormat
+	 * @param inputFormatClass
+	 *        the class of the InputFormat
+	 * @param configuration
+	 *        optional configuration of the InputFormat
+	 * @return the created {@link InputFormat}
+	 * @throws IOException
+	 *         if an I/O error occurred while accessing the file or initializing the InputFormat.
+	 */
+	public static <T, IS extends InputSplit, F extends InputFormat<T, IS>> F openInput(
+			Class<F> inputFormatClass, Configuration configuration) throws IOException {
+		configuration = configuration == null ? new Configuration() : configuration;
+
+		final F inputFormat = ReflectionUtil.newInstance(inputFormatClass);
+		inputFormat.configure(configuration);
+		final IS[] splits = inputFormat.createInputSplits(1);
+		inputFormat.open(splits[0]);
+		return inputFormat;
+	}
+	
+	/**
+	 * Creates an {@link OutputFormat} from a given class for the specified file. The optional {@link Configuration}
+	 * initializes the format.
+	 * 
+	 * @param <T>
+	 *        the class of the OutputFormat
+	 * @param outputFormatClass
+	 *        the class of the OutputFormat
+	 * @param path
+	 *        the path of the file or to the directory containing the splits
+	 * @param configuration
+	 *        optional configuration of the OutputFormat
+	 * @return the created {@link OutputFormat}
+	 * @throws IOException
+	 *         if an I/O error occurred while accessing the file or initializing the OutputFormat.
+	 */
+	public static <T, F extends FileOutputFormat<? extends T>> F openOutput(
+			Class<F> outputFormatClass, String path, Configuration configuration) 
+		throws IOException
+	{
+		final F outputFormat = ReflectionUtil.newInstance(outputFormatClass);
+		outputFormat.setOutputFilePath(new Path(path));
+		outputFormat.setWriteMode(WriteMode.OVERWRITE);
+	
+		configuration = configuration == null ? new Configuration() : configuration;
+		
+		outputFormat.configure(configuration);
+		outputFormat.open(0, 1);
+		return outputFormat;
+	}
+
+	/**
+	 * Fixes the path if it denotes a local (relative) file without the proper protocol prefix.
+	 */
+	private static Path normalizePath(Path path) {
+		URI uri = path.toUri();
+		if (uri.getScheme() == null) {
+			try {
+				uri = new URI("file", uri.getHost(), uri.getPath(), uri.getFragment());
+				path = new Path(uri.toString());
+			} catch (URISyntaxException e) {
+				throw new IllegalArgumentException("path is invalid", e);
+			}
+		}
+		return path;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/test/java/org/apache/flink/core/io/DefaultSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/io/DefaultSplitAssignerTest.java b/flink-core/src/test/java/org/apache/flink/core/io/DefaultSplitAssignerTest.java
new file mode 100644
index 0000000..b88fcc5
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/io/DefaultSplitAssignerTest.java
@@ -0,0 +1,125 @@
+/**
+ * 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.core.io;
+
+import static org.junit.Assert.*;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.junit.Test;
+
+
+public class DefaultSplitAssignerTest {
+
+	@Test
+	public void testSerialSplitAssignment() {
+		try {
+			final int NUM_SPLITS = 50;
+			
+			Set<InputSplit> splits = new HashSet<InputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new GenericInputSplit(i, NUM_SPLITS));
+			}
+			
+			DefaultInputSplitAssigner ia = new DefaultInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit("")) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit(""));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignment() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			Set<InputSplit> splits = new HashSet<InputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new GenericInputSplit(i, NUM_SPLITS));
+			}
+			
+			final DefaultInputSplitAssigner ia = new DefaultInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					String host = "";
+					GenericInputSplit split;
+					while ((split = (GenericInputSplit) ia.getNextInputSplit(host)) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit(""));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java b/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java
new file mode 100644
index 0000000..aa56c1c
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/io/LocatableSplitAssignerTest.java
@@ -0,0 +1,385 @@
+/**
+ * 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.core.io;
+
+import static org.junit.Assert.*;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.LocatableInputSplit;
+import org.apache.flink.util.LogUtils;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+
+public class LocatableSplitAssignerTest {
+	
+	@Test
+	public void testSerialSplitAssignmentWithNullHost() {
+		try {
+			final int NUM_SPLITS = 50;
+			final String[][] hosts = new String[][] {
+					new String[] { "localhost" },
+					new String[0],
+					null
+			};
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i%3]));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit(null)) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit(""));
+			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
+			assertEquals(0, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialSplitAssignmentAllForSameHost() {
+		try {
+			final int NUM_SPLITS = 50;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, "testhost"));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit("testhost")) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit(""));
+			
+			assertEquals(0, ia.getNumberOfRemoteAssignments());
+			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialSplitAssignmentAllForRemoteHost() {
+		try {
+			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
+			final int NUM_SPLITS = 10 * hosts.length;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i % hosts.length]));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit("testhost")) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit("anotherHost"));
+			
+			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
+			assertEquals(0, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialSplitAssignmentMixedLocalHost() {
+		try {
+			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
+			final int NUM_SPLITS = 10 * hosts.length;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i % hosts.length]));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			int i = 0;
+			while ((is = ia.getNextInputSplit(hosts[i++ % hosts.length])) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit("anotherHost"));
+			
+			assertEquals(0, ia.getNumberOfRemoteAssignments());
+			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignmentNullHost() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			final String[][] hosts = new String[][] {
+					new String[] { "localhost" },
+					new String[0],
+					null
+			};
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i%3]));
+			}
+			
+			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					LocatableInputSplit split;
+					while ((split = ia.getNextInputSplit(null)) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit(""));
+			
+			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
+			assertEquals(0, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignmentForSingleHost() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, "testhost"));
+			}
+			
+			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					LocatableInputSplit split;
+					while ((split = ia.getNextInputSplit("testhost")) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit("testhost"));
+			
+			assertEquals(0, ia.getNumberOfRemoteAssignments());
+			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignmentForMultipleHosts() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i%hosts.length]));
+			}
+			
+			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					final String threadHost = hosts[(int) (Math.random() * hosts.length)];
+					
+					LocatableInputSplit split;
+					while ((split = ia.getNextInputSplit(threadHost)) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit("testhost"));
+			
+			// at least one fraction of hosts needs be local, no matter how bad the thread races
+			assertTrue(ia.getNumberOfLocalAssignments() >= NUM_SPLITS / hosts.length);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessFixedLengthInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessFixedLengthInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessFixedLengthInputFormatTest.java
index 3518927..a5d2b91 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessFixedLengthInputFormatTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessFixedLengthInputFormatTest.java
@@ -22,7 +22,7 @@ package org.apache.flink.api.java.record.io;
 import java.io.IOException;
 
 import org.junit.Assert;
-
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.api.java.record.io.ExternalProcessFixedLengthInputFormat;
 import org.apache.flink.api.java.record.io.ExternalProcessInputFormat;
@@ -286,8 +286,8 @@ private ExternalProcessFixedLengthInputFormat<ExternalProcessInputSplit> format;
 		}
 
 		@Override
-		public Class<GenericInputSplit> getInputSplitType() {
-			return GenericInputSplit.class;
+		public DefaultInputSplitAssigner getInputSplitAssigner(GenericInputSplit[] splits) {
+			return new DefaultInputSplitAssigner(splits);
 		}
 
 		@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormatTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormatTest.java
index 1b86ecb..96f6664 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormatTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/record/io/ExternalProcessInputFormatTest.java
@@ -16,13 +16,12 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.java.record.io;
 
 import java.io.IOException;
 
 import org.junit.Assert;
-
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.api.java.record.io.ExternalProcessInputFormat;
 import org.apache.flink.api.java.record.io.ExternalProcessInputSplit;
@@ -225,10 +224,10 @@ public class ExternalProcessInputFormatTest {
 		}
 
 		@Override
-		public Class<GenericInputSplit> getInputSplitType() {
-			return GenericInputSplit.class;
+		public DefaultInputSplitAssigner getInputSplitAssigner(GenericInputSplit[] splits) {
+			return new DefaultInputSplitAssigner(splits);
 		}
-
+		
 		@Override
 		public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
 			return null;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java
index 2ff49e0..7ec5a6a 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/type/extractor/TypeExtractorInputFormatsTest.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.*;
 
 import java.io.IOException;
 
+import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
 import org.apache.flink.api.common.io.GenericInputFormat;
 import org.apache.flink.api.common.io.InputFormat;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
@@ -141,7 +142,7 @@ public class TypeExtractorInputFormatsTest {
 		public InputSplit[] createInputSplits(int minNumSplits) { return null; }
 
 		@Override
-		public Class<? extends InputSplit> getInputSplitType() { return null; }
+		public DefaultInputSplitAssigner getInputSplitAssigner(InputSplit[] splits) { return null; }
 
 		@Override
 		public void open(InputSplit split) {}
@@ -211,7 +212,7 @@ public class TypeExtractorInputFormatsTest {
 		public InputSplit[] createInputSplits(int minNumSplits) { return null; }
 
 		@Override
-		public Class<? extends InputSplit> getInputSplitType() { return null; }
+		public DefaultInputSplitAssigner getInputSplitAssigner(InputSplit[] splits) { return null; }
 
 		@Override
 		public void open(InputSplit split) {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
index c187961..1fa9491 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
@@ -41,6 +41,7 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 
 	/** The size of the ID in byte */
 	public static final int SIZE = 2 * SIZE_OF_LONG;
+	
 
 	/** The upper part of the actual ID */
 	private long upperPart;
@@ -48,11 +49,13 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 	/** The lower part of the actual ID */
 	private long lowerPart;
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Constructs a new ID with a specific bytes value.
 	 */
 	public AbstractID(byte[] bytes) {
-		if (bytes.length != SIZE) {
+		if (bytes == null || bytes.length != SIZE) {
 			throw new IllegalArgumentException("Argument bytes must by an array of " + SIZE + " bytes");
 		}
 
@@ -80,6 +83,9 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 	 * @param id the abstract ID to copy
 	 */
 	public AbstractID(AbstractID id) {
+		if (id == null) {
+			throw new IllegalArgumentException("Id must not be null.");
+		}
 		this.lowerPart = id.lowerPart;
 		this.upperPart = id.upperPart;
 	}
@@ -91,7 +97,19 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, j
 		this.lowerPart = generateRandomLong();
 		this.upperPart = generateRandomLong();
 	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public long getLowerPart() {
+		return lowerPart;
+	}
+	
+	public long getUpperPart() {
+		return upperPart;
+	}
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Generates a uniformly distributed random positive long.
 	 *

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java
new file mode 100644
index 0000000..c4a4211
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/JobException.java
@@ -0,0 +1,32 @@
+/**
+ * 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;
+
+public class JobException extends Exception {
+
+	private static final long serialVersionUID = 1275864691743020176L;
+
+	public JobException(String msg) {
+		super(msg);
+	}
+
+	public JobException(String message, Throwable cause) {
+		super(message, cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/JobSubmissionException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/JobSubmissionException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/JobSubmissionException.java
deleted file mode 100644
index 4960d80..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/JobSubmissionException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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;
-
-/**
- * A job submission exception is thrown if an error occurs while submitting
- * a job from the client to the job manager.
- * 
- */
-public class JobSubmissionException extends Exception {
-
-	/**
-	 * Generated serial UID
-	 */
-	private static final long serialVersionUID = 1275864691743020176L;
-
-	/**
-	 * Constructs a new job submission exception with the given error message.
-	 * 
-	 * @param msg
-	 *        the error message to be transported through this exception
-	 */
-	public JobSubmissionException(String msg) {
-		super(msg);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/client/AbstractJobResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/AbstractJobResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/AbstractJobResult.java
index 2ca8b68..416453f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/AbstractJobResult.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/AbstractJobResult.java
@@ -37,18 +37,13 @@ public abstract class AbstractJobResult implements IOReadableWritable {
 
 	/**
 	 * The possible return codes for a job operation.
-	 * 
 	 */
 	public enum ReturnCode {
 
-		/**
-		 * The success return code.
-		 */
+		/** The success return code. */
 		SUCCESS,
 
-		/**
-		 * The error return code.
-		 */
+		/** The error return code. */
 		ERROR
 	};
 
@@ -60,7 +55,7 @@ public abstract class AbstractJobResult implements IOReadableWritable {
 	/**
 	 * An optional description which can provide further information in case of an error.
 	 */
-	private String description = null;
+	private String description = "";
 
 	/**
 	 * Constructs a new abstract job result object and sets the description.
@@ -70,7 +65,7 @@ public abstract class AbstractJobResult implements IOReadableWritable {
 	 * @param description
 	 *        the optional error description
 	 */
-	public AbstractJobResult(final ReturnCode returnCode, final String description) {
+	public AbstractJobResult(ReturnCode returnCode, String description) {
 		this.returnCode = returnCode;
 		this.description = description;
 	}
@@ -79,8 +74,7 @@ public abstract class AbstractJobResult implements IOReadableWritable {
 	 * Construct a new abstract job result object. This constructor is required
 	 * for the deserialization process.
 	 */
-	public AbstractJobResult() {
-	}
+	public AbstractJobResult() {}
 
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
index 79da72f..028713d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobClient.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.client;
 
 import java.io.IOException;
@@ -45,29 +44,19 @@ import org.apache.flink.util.StringUtils;
 
 /**
  * The job client is able to submit, control, and abort jobs.
- * <p>
- * This class is thread-safe.
  */
 public class JobClient {
 
-	/**
-	 * The logging object used for debugging.
-	 */
+	/** The logging object used for debugging. */
 	private static final Logger LOG = LoggerFactory.getLogger(JobClient.class);
 
-	/**
-	 * The job management server stub.
-	 */
+	/** The job management server stub.*/
 	private final JobManagementProtocol jobSubmitClient;
 
-	/**
-	 * The accumulator protocol stub to request accumulators from JobManager
-	 */
+	/** The accumulator protocol stub to request accumulators from JobManager */
 	private AccumulatorProtocol accumulatorProtocolProxy;
 
-	/**
-	 * The job graph assigned with this job client.
-	 */
+	/** The job graph assigned with this job client. */
 	private final JobGraph jobGraph;
 
 	/**
@@ -121,7 +110,7 @@ public class JobClient {
 
 	/**
 	 * Constructs a new job client object and instantiates a local
-	 * RPC proxy for the {@link JobSubmissionProtocol}.
+	 * RPC proxy for the JobSubmissionProtocol
 	 * 
 	 * @param jobGraph
 	 *        the job graph to run
@@ -134,7 +123,7 @@ public class JobClient {
 
 	/**
 	 * Constructs a new job client object and instantiates a local
-	 * RPC proxy for the {@link JobSubmissionProtocol}.
+	 * RPC proxy for the JobSubmissionProtocol
 	 * 
 	 * @param jobGraph
 	 *        the job graph to run
@@ -160,7 +149,7 @@ public class JobClient {
 
 	/**
 	 * Constructs a new job client object and instantiates a local
-	 * RPC proxy for the {@link JobSubmissionProtocol}.
+	 * RPC proxy for the JobSubmissionProtocol
 	 * 
 	 * @param jobGraph
 	 *        the job graph to run
@@ -335,7 +324,7 @@ public class JobClient {
 				if (event instanceof JobEvent) {
 					final JobEvent jobEvent = (JobEvent) event;
 					final JobStatus jobStatus = jobEvent.getCurrentJobStatus();
-					if (jobStatus == JobStatus.SCHEDULED) {
+					if (jobStatus == JobStatus.RUNNING) {
 						startTimestamp = jobEvent.getTimestamp();
 					}
 					if (jobStatus == JobStatus.FINISHED) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionResult.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionResult.java b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionResult.java
index c755dc6..53319d3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionResult.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/client/JobSubmissionResult.java
@@ -16,14 +16,8 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.client;
 
-import java.io.IOException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
 /**
  * A <code>JobSubmissionResult</code> is used to report the results
  * of a job submission. It contains a return code and a description.
@@ -53,17 +47,4 @@ public class JobSubmissionResult extends AbstractJobResult {
 	public JobSubmissionResult() {
 		super();
 	}
-
-
-	@Override
-	public void read(final DataInputView in) throws IOException {
-		super.read(in);
-	}
-
-
-	@Override
-	public void write(final DataOutputView out) throws IOException {
-		super.write(out);
-	}
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
index f89e999..b4a38f2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/ChannelDeploymentDescriptor.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.deployment;
 
 import java.io.IOException;
@@ -24,43 +23,30 @@ import java.io.IOException;
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.executiongraph.ExecutionEdge2;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 
 /**
  * A channel deployment descriptor contains all the information necessary to deploy either an input or an output channel
  * as part of a task on a task manager.
- * <p>
- * This class is not thread-safe in general.
- * 
  */
 public final class ChannelDeploymentDescriptor implements IOReadableWritable {
 
-	/**
-	 * The ID of the output channel.
-	 */
+	/** The ID of the output channel. */
 	private final ChannelID outputChannelID;
 
-	/**
-	 * The ID of the input channel.
-	 */
+	/** The ID of the input channel. */
 	private final ChannelID inputChannelID;
 
 	/**
 	 * Constructs a new channel deployment descriptor.
 	 * 
-	 * @param outputChannelID
-	 *        the ID of the output channel
-	 * @param inputChannelID
-	 *        the ID of the input channel
+	 * @param outputChannelID The ID of the output channel
+	 * @param inputChannelID The ID of the input channel
 	 */
-	public ChannelDeploymentDescriptor(final ChannelID outputChannelID, final ChannelID inputChannelID) {
-
-		if (outputChannelID == null) {
-			throw new IllegalArgumentException("Argument outputChannelID must not be null");
-		}
-
-		if (inputChannelID == null) {
-			throw new IllegalArgumentException("Argument inputChannelID must not be null");
+	public ChannelDeploymentDescriptor(ChannelID outputChannelID, ChannelID inputChannelID) {
+		if (outputChannelID == null || inputChannelID == null) {
+			throw new IllegalArgumentException("Channel IDs must not be null");
 		}
 
 		this.outputChannelID = outputChannelID;
@@ -71,23 +57,20 @@ public final class ChannelDeploymentDescriptor implements IOReadableWritable {
 	 * Default constructor for serialization/deserialization.
 	 */
 	public ChannelDeploymentDescriptor() {
-
 		this.outputChannelID = new ChannelID();
 		this.inputChannelID = new ChannelID();
 	}
 
 
 	@Override
-	public void write(final DataOutputView out) throws IOException {
-
+	public void write(DataOutputView out) throws IOException {
 		this.outputChannelID.write(out);
 		this.inputChannelID.write(out);
 	}
 
 
 	@Override
-	public void read(final DataInputView in) throws IOException {
-
+	public void read(DataInputView in) throws IOException {
 		this.outputChannelID.read(in);
 		this.inputChannelID.read(in);
 	}
@@ -98,7 +81,6 @@ public final class ChannelDeploymentDescriptor implements IOReadableWritable {
 	 * @return the output channel ID attached to this deployment descriptor
 	 */
 	public ChannelID getOutputChannelID() {
-
 		return this.outputChannelID;
 	}
 
@@ -108,7 +90,12 @@ public final class ChannelDeploymentDescriptor implements IOReadableWritable {
 	 * @return the input channel ID attached to this deployment descriptor
 	 */
 	public ChannelID getInputChannelID() {
-
 		return this.inputChannelID;
 	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static ChannelDeploymentDescriptor fromExecutionEdge(ExecutionEdge2 edge) {
+		return new ChannelDeploymentDescriptor(edge.getOutputChannelId(), edge.getInputChannelId());
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
index dfec497..e4a447f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/deployment/GateDeploymentDescriptor.java
@@ -16,74 +16,36 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.deployment;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Iterator;
 import java.util.List;
 
 import org.apache.flink.core.io.IOReadableWritable;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.runtime.io.network.channels.ChannelType;
-import org.apache.flink.runtime.io.network.gates.GateID;
-import org.apache.flink.runtime.util.EnumUtils;
+import org.apache.flink.runtime.executiongraph.ExecutionEdge2;
 
 /**
- * A gate deployment descriptor contains all the information necessary to deploy either an input or an output gate as
- * part of a task on a task manager.
- * <p>
- * This class is not thread-safe in general.
- * 
+ * A gate deployment descriptor contains the deployment descriptors for the channels associated with that gate.
  */
 public final class GateDeploymentDescriptor implements IOReadableWritable {
 
-	/**
-	 * The ID of the gate.
-	 */
-	private final GateID gateID;
-
-	/**
-	 * The channel type of the gate.
-	 */
-	private ChannelType channelType;
-
-	/**
-	 * The list of channel deployment descriptors attached to this gate.
-	 */
+	/** The list of channel deployment descriptors attached to this gate. */
 	private final List<ChannelDeploymentDescriptor> channels;
 
 	/**
 	 * Constructs a new gate deployment descriptor
 	 * 
-	 * @param gateID
-	 *        the ID of the gate
-	 * @param channelType
-	 *        the channel type of the gate
-	 * @param compressionLevel
-	 *        the compression level of the gate
 	 * @param channels
 	 *        the list of channel deployment descriptors attached to this gate
 	 */
-	public GateDeploymentDescriptor(final GateID gateID, final ChannelType channelType,
-			List<ChannelDeploymentDescriptor> channels) {
-
-		if (gateID == null) {
-			throw new IllegalArgumentException("Argument gateID must no be null");
-		}
-
-		if (channelType == null) {
-			throw new IllegalArgumentException("Argument channelType must no be null");
-		}
-
+	public GateDeploymentDescriptor(List<ChannelDeploymentDescriptor> channels) {
 		if (channels == null) {
-			throw new IllegalArgumentException("Argument channels must no be null");
+			throw new NullPointerException();
 		}
 
-		this.gateID = gateID;
-		this.channelType = channelType;
 		this.channels = channels;
 	}
 
@@ -91,71 +53,49 @@ public final class GateDeploymentDescriptor implements IOReadableWritable {
 	 * Default constructor for serialization/deserialization.
 	 */
 	public GateDeploymentDescriptor() {
-
-		this.gateID = new GateID();
-		this.channelType = null;
 		this.channels = new ArrayList<ChannelDeploymentDescriptor>();
 	}
 
-
+	
+	public List<ChannelDeploymentDescriptor> getChannels() {
+		return channels;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
 	@Override
 	public void write(final DataOutputView out) throws IOException {
-
-		this.gateID.write(out);
-		EnumUtils.writeEnum(out, channelType);
 		out.writeInt(this.channels.size());
-		final Iterator<ChannelDeploymentDescriptor> it = this.channels.iterator();
-		while (it.hasNext()) {
-			it.next().write(out);
+		for (ChannelDeploymentDescriptor cdd : this.channels) {
+			cdd.write(out);
 		}
 	}
 
-
 	@Override
 	public void read(final DataInputView in) throws IOException {
-
-		this.gateID.read(in);
-		this.channelType = EnumUtils.readEnum(in, ChannelType.class);
 		final int nocdd = in.readInt();
 		for (int i = 0; i < nocdd; ++i) {
-			final ChannelDeploymentDescriptor cdd = new ChannelDeploymentDescriptor();
+			ChannelDeploymentDescriptor cdd = new ChannelDeploymentDescriptor();
 			cdd.read(in);
 			this.channels.add(cdd);
 		}
 	}
-
-	/**
-	 * Returns the ID of the gate.
-	 * 
-	 * @return the ID of the gate
-	 */
-	public GateID getGateID() {
-
-		return this.gateID;
-	}
-
-	/**
-	 * Returns the channel type of the gate.
-	 * 
-	 * @return the channel type of the gate
-	 */
-	public ChannelType getChannelType() {
-
-		return this.channelType;
-	}
-
-	/**
-	 * Returns the number of channel deployment descriptors attached to this gate descriptor.
-	 * 
-	 * @return the number of channel deployment descriptors
-	 */
-	public int getNumberOfChannelDescriptors() {
-
-		return this.channels.size();
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public static GateDeploymentDescriptor fromEdges(List<ExecutionEdge2> edges) {
+		List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(edges.size());
+		for (ExecutionEdge2 edge : edges) {
+			channels.add(ChannelDeploymentDescriptor.fromExecutionEdge(edge));
+		}
+		return new GateDeploymentDescriptor(channels);
 	}
-
-	public ChannelDeploymentDescriptor getChannelDescriptor(final int index) {
-
-		return this.channels.get(index);
+	
+	public static GateDeploymentDescriptor fromEdges(ExecutionEdge2[] edges) {
+		List<ChannelDeploymentDescriptor> channels = new ArrayList<ChannelDeploymentDescriptor>(edges.length);
+		for (ExecutionEdge2 edge : edges) {
+			channels.add(ChannelDeploymentDescriptor.fromExecutionEdge(edge));
+		}
+		return new GateDeploymentDescriptor(channels);
 	}
 }


[25/63] [abbrv] git commit: Redesign Scheduler part 2

Posted by se...@apache.org.
Redesign Scheduler part 2


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/e6aadfcc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/e6aadfcc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/e6aadfcc

Branch: refs/heads/master
Commit: e6aadfccdaf02b9df65d10ac835cab7fd26e274e
Parents: aa7550a
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Jul 25 03:09:49 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../flink/client/CliFrontendListCancelTest.java |   7 -
 .../org/apache/flink/runtime/AbstractID.java    |  49 +-
 .../runtime/execution/ExecutionListener.java    |  11 -
 .../runtime/executiongraph/ExecutionGraph.java  |  11 +-
 .../executiongraph/ExecutionVertex2.java        |  63 +-
 .../VertexAssignmentListener.java               |  40 --
 .../flink/runtime/filecache/FileCache.java      |   2 +-
 .../flink/runtime/instance/AllocatedSlot.java   |  84 ++-
 .../flink/runtime/instance/AllocationID.java    |  32 -
 .../apache/flink/runtime/instance/Hardware.java |   2 +-
 .../apache/flink/runtime/instance/Instance.java | 202 ++++--
 .../instance/InstanceConnectionInfo.java        |   6 +-
 .../flink/runtime/instance/InstanceID.java      |   6 +-
 .../runtime/instance/InstanceListener.java      |  10 +
 .../flink/runtime/jobgraph/JobVertexID.java     |   6 +-
 .../flink/runtime/jobmanager/JobManager.java    | 102 +--
 .../jobmanager/scheduler/DefaultScheduler.java  | 528 ++++++++--------
 .../scheduler/InstanceFillDegreeComparator.java |  31 -
 .../jobmanager/scheduler/ScheduledUnit.java     |  68 +-
 .../scheduler/SchedulingStrategy.java           |  33 -
 .../runtime/jobmanager/scheduler/SetQueue.java  | 134 ++++
 .../jobmanager/scheduler/SharedSlot.java        |  99 +++
 .../scheduler/SlotAllocationFuture.java         |  94 +++
 .../scheduler/SlotAllocationFutureAction.java   |  34 +
 .../scheduler/SlotAvailablilityListener.java    |  29 +
 .../jobmanager/scheduler/SlotSharingGroup.java  |  70 ++
 .../scheduler/SlotSharingGroupAssignment.java   | 270 ++++++++
 .../runtime/jobmanager/scheduler/SubSlot.java   |  69 ++
 .../jobmanager/web/SetupInfoServlet.java        |   2 +-
 .../protocols/ExtendedManagementProtocol.java   |  10 -
 .../protocols/TaskOperationProtocol.java        |   5 -
 .../taskmanager/ExecutorThreadFactory.java      |  41 --
 .../flink/runtime/taskmanager/TaskManager.java  |   1 +
 .../runtime/ExecutorThreadFactory.java          |  41 --
 .../runtime/util/ExecutorThreadFactory.java     |  38 ++
 .../flink/runtime/util/NativeCodeLoader.java    | 129 ----
 .../apache/flink/runtime/AbstractIDTest.java    |  75 ++-
 .../flink/runtime/instance/HardwareTest.java    |  25 +-
 .../instance/InstanceConnectionInfoTest.java    |  25 +-
 .../flink/runtime/instance/InstanceTest.java    | 174 +++++
 .../instance/LocalInstanceManagerTest.java      |   4 -
 .../scheduler/DefaultSchedulerTest.java         | 180 ------
 .../scheduler/SchedulerIsolatedTasksTest.java   | 385 +++++++++++
 .../scheduler/SchedulerSlotSharingTest.java     | 633 +++++++++++++++++++
 .../scheduler/SchedulerTestUtils.java           | 145 +++++
 .../jobmanager/scheduler/SetQueueTest.java      | 110 ++++
 .../jobmanager/scheduler/SharedSlotsTest.java   | 118 ++++
 .../scheduler/SlotAllocationFutureTest.java     | 177 ++++++
 48 files changed, 3328 insertions(+), 1082 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
index fa4d9a3..f335745 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.client;
 
 import static org.junit.Assert.assertTrue;
@@ -28,7 +27,6 @@ import java.util.List;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.flink.client.CliFrontend;
-import org.apache.flink.core.io.StringRecord;
 import org.apache.flink.runtime.client.JobCancelResult;
 import org.apache.flink.runtime.client.JobProgressResult;
 import org.apache.flink.runtime.client.JobSubmissionResult;
@@ -197,11 +195,6 @@ public class CliFrontendListCancelTest {
 		}
 
 		@Override
-		public void killInstance(StringRecord instanceName) throws IOException {
-			throw new UnsupportedOperationException();
-		}
-
-		@Override
 		public void logBufferUtilization(JobID jobID) throws IOException {
 			throw new UnsupportedOperationException();
 		}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
index 73cd8fc..458907c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
@@ -32,7 +32,7 @@ import io.netty.buffer.ByteBuf;
 /**
  * A statistically unique identification number.
  */
-public class AbstractID implements IOReadableWritable {
+public class AbstractID implements IOReadableWritable, Comparable<AbstractID> {
 
 	/** The size of a long in bytes */
 	private static final int SIZE_OF_LONG = 8;
@@ -139,24 +139,8 @@ public class AbstractID implements IOReadableWritable {
 		this.lowerPart = src.lowerPart;
 		this.upperPart = src.upperPart;
 	}
-
-	@Override
-	public boolean equals(Object obj) {
-		if (obj != null && obj instanceof AbstractID) {
-			AbstractID src = (AbstractID) obj;
-			return src.lowerPart == this.lowerPart && src.upperPart == this.upperPart;
-		} else {
-			return false;
-		}
-	}
-
-	@Override
-	public int hashCode() {
-		return ((int)  this.lowerPart) ^
-				((int) (this.lowerPart >>> 32)) ^
-				((int)  this.upperPart) ^
-				((int) (this.upperPart >>> 32));
-	}
+	
+	// --------------------------------------------------------------------------------------------
 
 	@Override
 	public void read(DataInputView in) throws IOException {
@@ -180,6 +164,26 @@ public class AbstractID implements IOReadableWritable {
 		buf.writeLong(this.upperPart);
 	}
 
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj != null && obj instanceof AbstractID) {
+			AbstractID src = (AbstractID) obj;
+			return src.lowerPart == this.lowerPart && src.upperPart == this.upperPart;
+		} else {
+			return false;
+		}
+	}
+
+	@Override
+	public int hashCode() {
+		return ((int)  this.lowerPart) ^
+				((int) (this.lowerPart >>> 32)) ^
+				((int)  this.upperPart) ^
+				((int) (this.upperPart >>> 32));
+	}
+	
 	@Override
 	public String toString() {
 		final byte[] ba = new byte[SIZE];
@@ -187,4 +191,11 @@ public class AbstractID implements IOReadableWritable {
 		longToByteArray(this.upperPart, ba, SIZE_OF_LONG);
 		return StringUtils.byteToHexString(ba);
 	}
+	
+	@Override
+	public int compareTo(AbstractID o) {
+		int diff1 = Long.compare(this.upperPart, o.upperPart);
+		int diff2 = Long.compare(this.lowerPart, o.lowerPart);
+		return diff1 == 0 ? diff2 : diff1;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
index 305f47d..ded630f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/ExecutionListener.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.execution;
 
 import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
@@ -25,20 +24,10 @@ import org.apache.flink.runtime.jobgraph.JobID;
 /**
  * This interface must be implemented by classes which should be able to receive notifications about
  * changes of a task's execution state.
- * 
  */
 public interface ExecutionListener {
 
 	/**
-	 * Returns the priority of the execution listener object. If multiple execution listener objects are registered for
-	 * a given vertex, the priority determines in which order they will be called. Priorities are expressed as
-	 * non-negative integer values. The lower the integer value, the higher the call priority.
-	 * 
-	 * @return the priority of this execution listener
-	 */
-	int getPriority();
-
-	/**
 	 * Called when the execution state of the associated task has changed. It is important to point out that multiple
 	 * execution listeners can be invoked as a reaction to a state change, according to their priority. As a result, the
 	 * value of <code>newExecutionState</code> may be out-dated by the time a particular execution listener is called.

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index cea0271..bc95250 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.executiongraph;
 
 import java.util.ArrayList;
@@ -43,8 +42,6 @@ import org.apache.flink.core.io.InputSplit;
 import org.apache.flink.runtime.execution.ExecutionListener;
 import org.apache.flink.runtime.execution.ExecutionState;
 import org.apache.flink.runtime.execution.librarycache.LibraryCacheManager;
-import org.apache.flink.runtime.instance.AllocatedResource;
-import org.apache.flink.runtime.instance.DummyInstance;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.io.network.channels.ChannelType;
 import org.apache.flink.runtime.io.network.gates.GateID;
@@ -57,7 +54,7 @@ import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobInputVertex;
 import org.apache.flink.runtime.jobgraph.JobOutputVertex;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.taskmanager.ExecutorThreadFactory;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.util.StringUtils;
 
 /**
@@ -1291,12 +1288,6 @@ public class ExecutionGraph implements ExecutionListener {
 		return this.stages.iterator();
 	}
 
-
-	@Override
-	public int getPriority() {
-		return 1;
-	}
-
 	/**
 	 * Performs an asynchronous update operation to this execution graph.
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
index ab33ca0..09a6f5b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex2.java
@@ -15,11 +15,70 @@
 
 package org.apache.flink.runtime.executiongraph;
 
+import org.apache.flink.runtime.instance.Instance;
+import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
 public class ExecutionVertex2 {
 
+	private final JobVertexID jobVertexId;
+	
+			
+			
+	public ExecutionVertex2() {
+		this(new JobVertexID());
+	}
+	
+	public ExecutionVertex2(JobVertexID jobVertexId) {
+		this.jobVertexId = jobVertexId;
+	}
+	
+	
+	
+	public JobID getJobId() {
+		return new JobID();
+	}
+	
+	
+	public JobVertexID getJobvertexId() {
+		return this.jobVertexId;
+	}
+	
+	public String getTaskName() {
+		return "task";
+	}
+	
+	public int getTotalNumberOfParallelSubtasks() {
+		return 1;
+	}
+	
+	public int getParallelSubtaskIndex() {
+		return 0;
+	}
+
+	
+	// --------------------------------------------------------------------------------------------
+	//  Scheduling
+	// --------------------------------------------------------------------------------------------
+	
+	public Iterable<Instance> getPreferredLocations() {
+		return null;
+	}
+	
 	
+	// --------------------------------------------------------------------------------------------
+	//  Utilities
+	// --------------------------------------------------------------------------------------------
 	
-	public void handleException(Throwable t) {
-		t.printStackTrace();
+	/**
+	 * Creates a simple name representation in the style 'taskname (x/y)', where
+	 * 'taskname' is the name as returned by {@link #getTaskName()}, 'x' is the parallel
+	 * subtask index as returned by {@link #getParallelSubtaskIndex()}{@code + 1}, and 'y' is the total
+	 * number of tasks, as returned by {@link #getTotalNumberOfParallelSubtasks()}.
+	 * 
+	 * @return A simple name representation.
+	 */
+	public String getSimpleName() {
+		return getTaskName() + " (" + (getParallelSubtaskIndex()+1) + '/' + getTotalNumberOfParallelSubtasks() + ')';
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/VertexAssignmentListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/VertexAssignmentListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/VertexAssignmentListener.java
deleted file mode 100644
index f09b10f..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/VertexAssignmentListener.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * 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 org.apache.flink.runtime.instance.AllocatedResource;
-
-/**
- * Classes implementing the {@link VertexAssignmentListener} interface can register for notifications about changes in
- * the assignment of an {@link ExecutionVertex} to an {@link AllocatedResource}.
- * 
- */
-public interface VertexAssignmentListener {
-
-	/**
-	 * Called when an {@link ExecutionVertex} has been assigned to an {@link AllocatedResource}.
-	 * 
-	 * @param id
-	 *        the ID of the vertex which has been reassigned
-	 * @param newAllocatedResource
-	 *        the allocated resource the vertex is now assigned to
-	 */
-	void vertexAssignmentChanged(ExecutionVertexID id, AllocatedResource newAllocatedResource);
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
index 35562de..63b4d54 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/filecache/FileCache.java
@@ -42,7 +42,7 @@ import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.fs.local.LocalFileSystem;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.taskmanager.runtime.ExecutorThreadFactory;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.util.IOUtils;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
index 6289d45..cb7e658 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocatedSlot.java
@@ -18,20 +18,24 @@
 
 package org.apache.flink.runtime.instance;
 
-import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
 
 import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobmanager.scheduler.ResourceId;
 
 /**
  * An allocated slot is the unit in which resources are allocated on instances.
  */
 public class AllocatedSlot {
+	
+	private static final AtomicIntegerFieldUpdater<AllocatedSlot> STATUS_UPDATER = 
+			AtomicIntegerFieldUpdater.newUpdater(AllocatedSlot.class, "status");
+	 
+	private static final int ALLOCATED_AND_ALIVE = 0;		// tasks may be added and might be running
+	private static final int CANCELLED = 1;					// no more tasks may run
+	private static final int RELEASED = 2;					// has been given back to the instance
 
-	/** The ID which identifies the resources occupied by this slot. */
-	private final ResourceId resourceId;
-
+	
 	/** The ID of the job this slice belongs to. */
 	private final JobID jobID;
 	
@@ -41,12 +45,15 @@ public class AllocatedSlot {
 	/** The number of the slot on which the task is deployed */
 	private final int slotNumber;
 	
-	/** Flag that marks the schedule as active */
-	private final AtomicBoolean active = new AtomicBoolean(true);
+	private volatile int status = ALLOCATED_AND_ALIVE;
+	
 
 
-	public AllocatedSlot(JobID jobID, ResourceId resourceId, Instance instance, int slotNumber) {
-		this.resourceId = resourceId;
+	public AllocatedSlot(JobID jobID, Instance instance, int slotNumber) {
+		if (jobID == null || instance == null || slotNumber < 0) {
+			throw new IllegalArgumentException();
+		}
+		
 		this.jobID = jobID;
 		this.instance = instance;
 		this.slotNumber = slotNumber;
@@ -63,10 +70,6 @@ public class AllocatedSlot {
 		return this.jobID;
 	}
 	
-	public ResourceId getResourceId() {
-		return resourceId;
-	}
-	
 	public Instance getInstance() {
 		return instance;
 	}
@@ -87,7 +90,60 @@ public class AllocatedSlot {
 		return true;
 	}
 	
-	public void cancelResource() {
+	// --------------------------------------------------------------------------------------------
+	//  Status and life cycle
+	// --------------------------------------------------------------------------------------------
+	
+	public boolean isAlive() {
+		return status == ALLOCATED_AND_ALIVE;
+	}
+	
+	public boolean isCanceled() {
+		return status != ALLOCATED_AND_ALIVE;
+	}
+	
+	public boolean isReleased() {
+		return status == RELEASED;
+	}
+	
+	
+	public void cancel() {
+		if (STATUS_UPDATER.compareAndSet(this, ALLOCATED_AND_ALIVE, CANCELLED)) {
+			// kill all tasks currently running in this slot
+		}
+	}
+	
+	public void releaseSlot() {
+		// cancel everything, if there is something. since this is atomically status based,
+		// it will not happen twice if another attempt happened before or concurrently
+		cancel();
 		
+		this.instance.returnAllocatedSlot(this);
+	}
+	
+	protected boolean markReleased() {
+		return STATUS_UPDATER.compareAndSet(this, CANCELLED, RELEASED);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Utilities
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return instance.getId() + " (" + slotNumber + ") - " + getStateName(status);
+	}
+	
+	private static final String getStateName(int state) {
+		switch (state) {
+		case ALLOCATED_AND_ALIVE:
+			return "ALLOCATED/ALIVE";
+		case CANCELLED:
+			return "CANCELLED";
+		case RELEASED:
+			return "RELEASED";
+		default:
+			return "(unknown)";
+		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocationID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocationID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocationID.java
deleted file mode 100644
index c315dc3..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/AllocationID.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/**
- * 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.instance;
-
-import org.apache.flink.runtime.AbstractID;
-
-/**
- * An allocation ID unambiguously identifies the allocated resources
- * within an {@link Instance}. The ID is necessary if an {@link InstanceManager} decides to partition
- * {@link Instance}s
- * without the knowledge of Nephele's scheduler.
- * 
- */
-public class AllocationID extends AbstractID {
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java
index 92ccfb2..aa61927 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Hardware.java
@@ -76,7 +76,7 @@ public class Hardware {
 				return -1;
 				
 			default:
-				LOG.error("Unrecognized OS");
+				LOG.error("Unrecognized OS: " + OperatingSystem.getCurrentOperatingSystem());
 				return -1;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
index 3d39c8f..a168b2c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/Instance.java
@@ -21,13 +21,15 @@ package org.apache.flink.runtime.instance;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayDeque;
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Queue;
 import java.util.Set;
 
 import org.apache.flink.runtime.ipc.RPC;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobmanager.scheduler.ResourceId;
+import org.apache.flink.runtime.jobmanager.scheduler.SlotAvailablilityListener;
 import org.apache.flink.runtime.net.NetUtils;
 import org.apache.flink.runtime.protocols.TaskOperationProtocol;
 import org.eclipse.jetty.util.log.Log;
@@ -52,14 +54,19 @@ public class Instance {
 	/** The number of task slots available on the node */
 	private final int numberOfSlots;
 
-	
+	/** A list of available slot positons */
 	private final Queue<Integer> availableSlots;
 	
 	/** Allocated slots on this instance */
 	private final Set<AllocatedSlot> allocatedSlots = new HashSet<AllocatedSlot>();
 
+	
+	/** A listener to be notified upon new slot availability */
+	private SlotAvailablilityListener slotListener;
+	
+	
 	/** The RPC proxy to send calls to the task manager represented by this instance */
-	private volatile TaskOperationProtocol taskManager ;
+	private volatile TaskOperationProtocol taskManager;
 
 	/**
 	 * Time when last heat beat has been received from the task manager running on this instance.
@@ -68,6 +75,8 @@ public class Instance {
 	
 	private volatile boolean isDead;
 
+	// --------------------------------------------------------------------------------------------
+	
 	/**
 	 * Constructs an abstract instance object.
 	 * 
@@ -82,43 +91,36 @@ public class Instance {
 		this.resources = resources;
 		this.numberOfSlots = numberOfSlots;
 		
-		this.availableSlots = new ArrayDeque<Integer>();
+		this.availableSlots = new ArrayDeque<Integer>(numberOfSlots);
 		for (int i = 0; i < numberOfSlots; i++) {
 			this.availableSlots.add(i);
 		}
 	}
 
-	public TaskOperationProtocol getTaskManagerProxy() throws IOException {
-		TaskOperationProtocol tm = this.taskManager;
-		
-		if (tm == null) {
-			synchronized (this) {
-				if (this.taskManager == null) {
-					this.taskManager = RPC.getProxy(TaskOperationProtocol.class,
-						new InetSocketAddress(getInstanceConnectionInfo().address(),
-							getInstanceConnectionInfo().ipcPort()), NetUtils.getSocketFactory());
-				}
-				tm = this.taskManager;
-			}
-		}
-		
-		return tm;
+	// --------------------------------------------------------------------------------------------
+	// Properties
+	// --------------------------------------------------------------------------------------------
+	
+	public InstanceID getId() {
+		return instanceId;
 	}
-
-	/**  Destroys and removes the RPC stub object for this instance's task manager. */
-	private void destroyTaskManagerProxy() {
-		synchronized (this) {
-			if (this.taskManager != null) {
-				try {
-					RPC.stopProxy(this.taskManager);
-				} catch (Throwable t) {
-					Log.debug("Error shutting down RPC proxy.", t);
-				}
-			}
-		}
+	
+	public HardwareDescription getResources() {
+		return this.resources;
+	}
+	
+	public int getTotalNumberOfSlots() {
+		return numberOfSlots;
+	}
+	
+	/**
+	 * Returns the instance's connection information object.
+	 * 
+	 * @return the instance's connection information object
+	 */
+	public InstanceConnectionInfo getInstanceConnectionInfo() {
+		return this.instanceConnectionInfo;
 	}
-
-
 	
 	// --------------------------------------------------------------------------------------------
 	// Life and Death
@@ -136,38 +138,52 @@ public class Instance {
 		isDead = true;
 		
 		synchronized (instanceLock) {
-			this.allocatedSlots.clear();
+			
+			// no more notifications for the slot releasing
+			this.slotListener = null;
+			
 			for (AllocatedSlot slot : allocatedSlots) {
-				slot.cancelResource();
+				slot.releaseSlot();
 			}
+			allocatedSlots.clear();
+			availableSlots.clear();
 		}
 		
 		destroyTaskManagerProxy();
 	}
 	
 	// --------------------------------------------------------------------------------------------
-	// Properties
+	//  Connection to the TaskManager
 	// --------------------------------------------------------------------------------------------
 	
-	public InstanceID getId() {
-		return instanceId;
-	}
-	
-	public HardwareDescription getResources() {
-		return this.resources;
-	}
-	
-	public int getTotalNumberOfSlots() {
-		return numberOfSlots;
+	public TaskOperationProtocol getTaskManagerProxy() throws IOException {
+		TaskOperationProtocol tm = this.taskManager;
+		
+		if (tm == null) {
+			synchronized (this) {
+				if (this.taskManager == null) {
+					this.taskManager = RPC.getProxy(TaskOperationProtocol.class,
+						new InetSocketAddress(getInstanceConnectionInfo().address(),
+							getInstanceConnectionInfo().ipcPort()), NetUtils.getSocketFactory());
+				}
+				tm = this.taskManager;
+			}
+		}
+		
+		return tm;
 	}
-	
-	/**
-	 * Returns the instance's connection information object.
-	 * 
-	 * @return the instance's connection information object
-	 */
-	public InstanceConnectionInfo getInstanceConnectionInfo() {
-		return this.instanceConnectionInfo;
+
+	/**  Destroys and removes the RPC stub object for this instance's task manager. */
+	private void destroyTaskManagerProxy() {
+		synchronized (this) {
+			if (this.taskManager != null) {
+				try {
+					RPC.stopProxy(this.taskManager);
+				} catch (Throwable t) {
+					Log.debug("Error shutting down RPC proxy.", t);
+				}
+			}
+		}
 	}
 	
 	// --------------------------------------------------------------------------------------------
@@ -206,7 +222,11 @@ public class Instance {
 	// Resource allocation
 	// --------------------------------------------------------------------------------------------
 	
-	public AllocatedSlot allocateSlot(JobID jobID, ResourceId resourceId) throws InstanceDiedException {
+	public AllocatedSlot allocateSlot(JobID jobID) throws InstanceDiedException {
+		if (jobID == null) {
+			throw new IllegalArgumentException();
+		}
+		
 		synchronized (instanceLock) {
 			if (isDead) {
 				throw new InstanceDiedException(this);
@@ -216,27 +236,95 @@ public class Instance {
 			if (nextSlot == null) {
 				return null;
 			} else {
-				AllocatedSlot slot = new AllocatedSlot(jobID, resourceId, this, nextSlot);
+				AllocatedSlot slot = new AllocatedSlot(jobID, this, nextSlot);
 				allocatedSlots.add(slot);
 				return slot;
 			}
 		}
 	}
 	
+	public boolean returnAllocatedSlot(AllocatedSlot slot) {
+		// the slot needs to be in the returned to instance state
+		if (slot == null || slot.getInstance() != this) {
+			throw new IllegalArgumentException("Slot is null or belongs to the wrong instance.");
+		}
+		if (slot.isAlive()) {
+			throw new IllegalArgumentException("Slot is still alive");
+		}
+		
+		if (slot.markReleased()) { 
+			synchronized (instanceLock) {
+				if (isDead) {
+					return false;
+				}
+			
+				if (this.allocatedSlots.remove(slot)) {
+					this.availableSlots.add(slot.getSlotNumber());
+					
+					if (this.slotListener != null) {
+						this.slotListener.newSlotAvailable(this);
+					}
+					
+					return true;
+				} else {
+					throw new IllegalArgumentException("Slot was not allocated from the instance.");
+				}
+			}
+		} else {
+			return false;
+		}
+	}
+	
+	public void cancelAndReleaseAllSlots() {
+		synchronized (instanceLock) {
+			// we need to do this copy because of concurrent modification exceptions
+			List<AllocatedSlot> copy = new ArrayList<AllocatedSlot>(this.allocatedSlots);
+			
+			for (AllocatedSlot slot : copy) {
+				slot.releaseSlot();
+			}
+			allocatedSlots.clear();
+		}
+	}
+	
 	public int getNumberOfAvailableSlots() {
 		return this.availableSlots.size();
 	}
 	
+	public int getNumberOfAllocatedSlots() {
+		return this.allocatedSlots.size();
+	}
+	
 	public boolean hasResourcesAvailable() {
 		return !isDead && getNumberOfAvailableSlots() > 0;
 	}
 	
 	// --------------------------------------------------------------------------------------------
+	// Listeners
+	// --------------------------------------------------------------------------------------------
+	
+	public void setSlotAvailabilityListener(SlotAvailablilityListener slotListener) {
+		synchronized (instanceLock) {
+			if (this.slotListener != null) {
+				throw new IllegalStateException("Instance has already a slot listener.");
+			} else {
+				this.slotListener = slotListener;
+			}
+		}
+	}
+	
+	public void removeSlotListener() {
+		synchronized (instanceLock) {
+			this.slotListener = null;
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
 	// Standard Utilities
 	// --------------------------------------------------------------------------------------------
 	
 	@Override
 	public String toString() {
-		return "Instance (" + this.instanceConnectionInfo + "), resources: " + this.resources + ", numberOfSlots=" + numberOfSlots;
+		return instanceId + " @" + this.instanceConnectionInfo + ' ' + numberOfSlots + " slots";
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
index 4cbb2a9..ec63c00 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceConnectionInfo.java
@@ -67,18 +67,18 @@ public class InstanceConnectionInfo implements IOReadableWritable, Comparable<In
 	 *        the port instance's task manager expects to receive transfer envelopes on
 	 */
 	public InstanceConnectionInfo(InetAddress inetAddress, int ipcPort, int dataPort) {
-
 		if (inetAddress == null) {
 			throw new IllegalArgumentException("Argument inetAddress must not be null");
 		}
-
 		if (ipcPort <= 0) {
 			throw new IllegalArgumentException("Argument ipcPort must be greater than zero");
 		}
-
 		if (dataPort <= 0) {
 			throw new IllegalArgumentException("Argument dataPort must be greater than zero");
 		}
+		if (ipcPort == dataPort) {
+			throw new IllegalArgumentException("IPC and data port must be different");
+		}
 
 		this.ipcPort = ipcPort;
 		this.dataPort = dataPort;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
index 388bba2..9c922af 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
@@ -16,15 +16,11 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.instance;
 
 import org.apache.flink.runtime.AbstractID;
 
 /**
  * Class for statistically unique instance IDs.
- * 
  */
-public class InstanceID extends AbstractID {
-
-}
+public class InstanceID extends AbstractID {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
index 76e63b8..d42d404 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceListener.java
@@ -24,7 +24,17 @@ package org.apache.flink.runtime.instance;
  */
 public interface InstanceListener {
 
+	/**
+	 * Called when a new instance becomes available.
+	 * 
+	 * @param instance The instance that became available.
+	 */
 	void newInstanceAvailable(Instance instance);
 	
+	/**
+	 * Called when an instance died.
+	 * 
+	 * @param instance The instance that died.
+	 */
 	void instanceDied(Instance instance);
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
index ba76e02..6a56c19 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
@@ -16,15 +16,11 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobgraph;
 
 import org.apache.flink.runtime.AbstractID;
 
 /**
  * A class for statistically unique job vertex IDs.
- * 
  */
-public class JobVertexID extends AbstractID {
-
-}
+public class JobVertexID extends AbstractID {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index 1f5fc96..3b76b78 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -70,11 +70,11 @@ import org.apache.flink.runtime.executiongraph.GraphConversionException;
 import org.apache.flink.runtime.executiongraph.InternalJobStatus;
 import org.apache.flink.runtime.executiongraph.JobStatusListener;
 import org.apache.flink.runtime.instance.DefaultInstanceManager;
-import org.apache.flink.runtime.instance.DummyInstance;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
 import org.apache.flink.runtime.instance.InstanceManager;
+import org.apache.flink.runtime.instance.LocalInstanceManager;
 import org.apache.flink.runtime.io.network.ConnectionInfoLookupResponse;
 import org.apache.flink.runtime.io.network.RemoteReceiver;
 import org.apache.flink.runtime.io.network.channels.ChannelID;
@@ -87,8 +87,6 @@ import org.apache.flink.runtime.jobmanager.accumulators.AccumulatorManager;
 import org.apache.flink.runtime.jobmanager.archive.ArchiveListener;
 import org.apache.flink.runtime.jobmanager.archive.MemoryArchivist;
 import org.apache.flink.runtime.jobmanager.scheduler.DefaultScheduler;
-import org.apache.flink.runtime.jobmanager.scheduler.SchedulingException;
-import org.apache.flink.runtime.jobmanager.splitassigner.InputSplitManager;
 import org.apache.flink.runtime.jobmanager.splitassigner.InputSplitWrapper;
 import org.apache.flink.runtime.jobmanager.web.WebInfoServer;
 import org.apache.flink.runtime.managementgraph.ManagementGraph;
@@ -101,14 +99,13 @@ import org.apache.flink.runtime.protocols.ExtendedManagementProtocol;
 import org.apache.flink.runtime.protocols.InputSplitProviderProtocol;
 import org.apache.flink.runtime.protocols.JobManagerProtocol;
 import org.apache.flink.runtime.taskmanager.AbstractTaskResult;
-import org.apache.flink.runtime.taskmanager.ExecutorThreadFactory;
 import org.apache.flink.runtime.taskmanager.TaskCancelResult;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
 import org.apache.flink.runtime.taskmanager.TaskKillResult;
 import org.apache.flink.runtime.taskmanager.TaskSubmissionResult;
-import org.apache.flink.runtime.taskmanager.transferenvelope.RegisterTaskManagerResult;
 import org.apache.flink.runtime.types.IntegerRecord;
 import org.apache.flink.runtime.util.EnvironmentInformation;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
 import org.apache.flink.runtime.util.SerializableArrayList;
 import org.apache.flink.util.StringUtils;
 import org.apache.log4j.ConsoleAppender;
@@ -130,32 +127,32 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 
 	private static final Log LOG = LogFactory.getLog(JobManager.class);
 
+	private final static int FAILURE_RETURN_CODE = 1;
+	
+	
+	private final ExecutorService executorService = Executors.newCachedThreadPool(ExecutorThreadFactory.INSTANCE);
+	
 	private final Server jobManagerServer;
 
-	private final JobManagerProfiler profiler;
-
 	private final EventCollector eventCollector;
 	
 	private final ArchiveListener archive;
 
-	private final InputSplitManager inputSplitManager;
-
+	private final InstanceManager instanceManager;
+	
 	private final DefaultScheduler scheduler;
 	
-	private AccumulatorManager accumulatorManager;
-
-	private InstanceManager instanceManager;
+	private final AccumulatorManager accumulatorManager;
 
+	
 	private final int recommendedClientPollingInterval;
 
-	private final ExecutorService executorService = Executors.newCachedThreadPool(ExecutorThreadFactory.INSTANCE);
-
-	private final static int FAILURE_RETURN_CODE = 1;
-
+	
 	private final AtomicBoolean isShutdownInProgress = new AtomicBoolean(false);
 
 	private volatile boolean isShutDown;
 	
+	
 	private WebInfoServer server;
 	
 	
@@ -199,8 +196,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		// accumulator results.
 		this.accumulatorManager = new AccumulatorManager(Math.min(1, archived_items));
 
-		// Load the input split manager
-		this.inputSplitManager = new InputSplitManager();
 
 		// Determine own RPC address
 		final InetSocketAddress rpcServerAddress = new InetSocketAddress(ipcAddress, ipcPort);
@@ -219,7 +214,8 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 
 		// Try to load the instance manager for the given execution mode
 		if (executionMode == ExecutionMode.LOCAL) {
-			this.instanceManager = new Lo
+			final int numTaskManagers = GlobalConfiguration.getInteger(ConfigConstants.LOCAL_INSTANCE_MANAGER_NUMBER_TASK_MANAGER, 1);
+			this.instanceManager = new LocalInstanceManager(numTaskManagers);
 		}
 		else if (executionMode == ExecutionMode.CLUSTER) {
 			this.instanceManager = new DefaultInstanceManager();
@@ -237,19 +233,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		if (this.scheduler == null) {
 			throw new Exception("Unable to load scheduler " + schedulerClassName);
 		}
-
-		// Load profiler if it should be used
-		if (GlobalConfiguration.getBoolean(ProfilingUtils.ENABLE_PROFILING_KEY, false)) {
-			final String profilerClassName = GlobalConfiguration.getString(ProfilingUtils.JOBMANAGER_CLASSNAME_KEY,
-				"org.apache.flink.runtime.profiling.impl.JobManagerProfilerImpl");
-			this.profiler = ProfilingUtils.loadJobManagerProfiler(profilerClassName, ipcAddress);
-			if (this.profiler == null) {
-				throw new Exception("Cannot load profiler");
-			}
-		} else {
-			this.profiler = null;
-			LOG.debug("Profiler disabled");
-		}
 	}
 
 	public void shutdown() {
@@ -263,11 +246,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 			this.instanceManager.shutdown();
 		}
 
-		// Stop profiling if enabled
-		if (this.profiler != null) {
-			this.profiler.shutdown();
-		}
-
 		// Stop RPC server
 		if (this.jobManagerServer != null) {
 			this.jobManagerServer.stop();
@@ -458,12 +436,6 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 				LOG.debug("The dependency chain for instance sharing is acyclic");
 			}
 	
-			// Check if the job will be executed with profiling enabled
-			boolean jobRunsWithProfiling = false;
-			if (this.profiler != null && job.getJobConfiguration().getBoolean(ProfilingUtils.PROFILE_JOB_KEY, true)) {
-				jobRunsWithProfiling = true;
-			}
-	
 			// Try to create initial execution graph from job graph
 			LOG.info("Creating initial execution graph from job graph " + job.getName());
 			ExecutionGraph eg;
@@ -485,22 +457,9 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 	
 			// Register job with the progress collector
 			if (this.eventCollector != null) {
-				this.eventCollector.registerJob(eg, jobRunsWithProfiling, System.currentTimeMillis());
+				this.eventCollector.registerJob(eg, false, System.currentTimeMillis());
 			}
 	
-			// Check if profiling should be enabled for this job
-			if (jobRunsWithProfiling) {
-				this.profiler.registerProfilingJob(eg);
-	
-				if (this.eventCollector != null) {
-					this.profiler.registerForProfilingData(eg.getJobID(), this.eventCollector);
-				}
-	
-			}
-	
-			// Register job with the dynamic input split assigner
-			this.inputSplitManager.registerJob(eg);
-	
 			// Register for updates on the job status
 			eg.registerJobStatusListener(this);
 	
@@ -902,41 +861,12 @@ public class JobManager implements DeploymentManager, ExtendedManagementProtocol
 		eg.executeCommand(runnable);
 	}
 
-
-	@Override
-	public void killInstance(final StringRecord instanceName) throws IOException {
-
-		final Instance instance = this.instanceManager.getInstanceByName(instanceName.toString());
-		if (instance == null) {
-			LOG.error("Cannot find instance with name " + instanceName + " to kill it");
-			return;
-		}
-
-		LOG.info("Killing task manager on instance " + instance);
-
-		final Runnable runnable = new Runnable() {
-
-			@Override
-			public void run() {
-				try {
-					instance.killTaskManager();
-				} catch (IOException ioe) {
-					LOG.error(ioe);
-				}
-			}
-		};
-
-		// Hand it over to the executor service
-		this.executorService.execute(runnable);
-	}
-
 	/**
 	 * Tests whether the job manager has been shut down completely.
 	 * 
 	 * @return <code>true</code> if the job manager has been shut down completely, <code>false</code> otherwise
 	 */
 	public boolean isShutDown() {
-
 		return this.isShutDown;
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
index e038d7d..3cae48b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
@@ -18,170 +18,293 @@
 
 package org.apache.flink.runtime.jobmanager.scheduler;
 
-import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.ArrayDeque;
 import java.util.HashSet;
+import java.util.Iterator;
 import java.util.Queue;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.instance.InstanceDiedException;
 import org.apache.flink.runtime.instance.InstanceListener;
-import org.apache.flink.runtime.jobgraph.JobID;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
 
 /**
  * The scheduler is responsible for distributing the ready-to-run tasks and assigning them to instances and
  * slots.
- * <p>
- * The scheduler's bookkeeping on the available instances is lazy: It is not modified once an
- * instance is dead, but it will lazily remove the instance from its pool as soon as it tries
- * to allocate a resource on that instance and it fails with an {@link InstanceDiedException}.
  */
-public class DefaultScheduler implements InstanceListener {
+public class DefaultScheduler implements InstanceListener, SlotAvailablilityListener {
 
 	protected static final Logger LOG = LoggerFactory.getLogger(DefaultScheduler.class);
-
 	
-	private final Object lock = new Object();
+	
+	private final Object globalLock = new Object();
+	
 	
 	/** All instances that the scheduler can deploy to */
 	private final Set<Instance> allInstances = new HashSet<Instance>();
 	
 	/** All instances that still have available resources */
-	private final Queue<Instance> instancesWithAvailableResources = new LifoSetQueue<Instance>();
-
-	
-	private final ConcurrentHashMap<ResourceId, AllocatedSlot> allocatedSlots = new ConcurrentHashMap<ResourceId, AllocatedSlot>();
-	
-//	/** A cache that remembers the last resource IDs it has seen, to co-locate future
-//	 *  deployments of tasks with the same resource ID to the same instance.
-//	 */
-//	private final Cache<ResourceId, Instance> ghostCache;
+	private final Queue<Instance> instancesWithAvailableResources = new SetQueue<Instance>();
 	
 	/** All tasks pending to be scheduled */
-	private final Queue<ScheduledUnit> taskQueue = new ArrayDeque<ScheduledUnit>();
-
+	private final Queue<QueuedTask> taskQueue = new ArrayDeque<QueuedTask>();
 	
-	/** The thread that runs the scheduling loop, picking up tasks to be scheduled and scheduling them. */
-	private final Thread schedulerThread;
 	
+	private int unconstrainedAssignments = 0;
 	
-	/** Atomic flag to safely control the shutdown */
-	private final AtomicBoolean shutdown = new AtomicBoolean(false);
+	private int localizedAssignments = 0;
 	
-	/** Flag indicating whether the scheduler should reject a unit if it cannot find a resource
-	 * for it at the time of scheduling */
-	private final boolean rejectIfNoResourceAvailable;
+	private int nonLocalizedAssignments = 0;
 	
-
 	
 	public DefaultScheduler() {
-		this(true);
-	}
-	
-	public DefaultScheduler(boolean rejectIfNoResourceAvailable) {
-		this.rejectIfNoResourceAvailable = rejectIfNoResourceAvailable;
-		
-//		this.ghostCache = CacheBuilder.newBuilder()
-//				.initialCapacity(64)	// easy start
-//				.maximumSize(1024)		// retain some history
-//				.weakValues()			// do not prevent dead instances from being collected
-//				.build();
-		
-		// set up (but do not start) the scheduling thread
-		Runnable loopRunner = new Runnable() {
-			@Override
-			public void run() {
-				runSchedulerLoop();
-			}
-		};
-		this.schedulerThread = new Thread(loopRunner, "Scheduling Thread");
-	}
-	
-	public void start() {
-		if (shutdown.get()) {
-			throw new IllegalStateException("Scheduler has been shut down.");
-		}
-		
-		try {
-			this.schedulerThread.start();
-		}
-		catch (IllegalThreadStateException e) {
-			throw new IllegalStateException("The scheduler has already been started.");
-		}
 	}
 	
 	/**
 	 * Shuts the scheduler down. After shut down no more tasks can be added to the scheduler.
 	 */
 	public void shutdown() {
-		if (this.shutdown.compareAndSet(false, true)) {
-			// clear the task queue and add the termination signal, to let
-			// the scheduling loop know that things are done
-			this.taskQueue.clear();
-			this.taskQueue.add(TERMINATION_SIGNAL);
-			
-			// interrupt the scheduling thread, in case it was waiting for resources to
-			// show up to deploy a task
-			this.schedulerThread.interrupt();
+		synchronized (globalLock) {
+			for (Instance i : allInstances) {
+				i.removeSlotListener();
+				i.cancelAndReleaseAllSlots();
+			}
+			allInstances.clear();
+			instancesWithAvailableResources.clear();
+			taskQueue.clear();
 		}
 	}
-	
-	public void setUncaughtExceptionHandler(UncaughtExceptionHandler handler) {
-		if (this.schedulerThread.getState() != Thread.State.NEW) {
-			throw new IllegalStateException("Can only add exception handler before starting the scheduler.");
+
+	/**
+	 * 
+	 * NOTE: In the presence of multi-threaded operations, this number may be inexact.
+	 * 
+	 * @return The number of empty slots, for tasks.
+	 */
+	public int getNumberOfAvailableSlots() {
+		int count = 0;
+		
+		synchronized (globalLock) {
+			for (Instance instance : instancesWithAvailableResources) {
+				count += instance.getNumberOfAvailableSlots();
+			}
 		}
-		this.schedulerThread.setUncaughtExceptionHandler(handler);
+		
+		return count;
 	}
-
 	
 	// --------------------------------------------------------------------------------------------
 	//  Scheduling
 	// --------------------------------------------------------------------------------------------
 	
+	public AllocatedSlot scheduleImmediately(ScheduledUnit task) throws NoResourceAvailableException {
+		Object ret = scheduleTask(task, false);
+		if (ret instanceof AllocatedSlot) {
+			return (AllocatedSlot) ret;
+		}
+		else {
+			throw new RuntimeException();
+		}
+	}
+	
+	public SlotAllocationFuture scheduleQueued(ScheduledUnit task) throws NoResourceAvailableException {
+		Object ret = scheduleTask(task, true);
+		if (ret instanceof AllocatedSlot) {
+			return new SlotAllocationFuture((AllocatedSlot) ret);
+		}
+		if (ret instanceof SlotAllocationFuture) {
+			return (SlotAllocationFuture) ret;
+		}
+		else {
+			throw new RuntimeException();
+		}
+	}
+	
 	/**
-	 * @param task
-	 * @param queueIfNoResource If true, this call will queue the request if no resource is immediately
-	 *                          available. If false, it will throw a {@link NoResourceAvailableException}
-	 *                          if no resource is immediately available.
+	 * Returns either an {@link AllocatedSlot}, or an {@link SlotAllocationFuture}.
 	 */
-	public void scheduleTask(ScheduledUnit task, boolean queueIfNoResource) {
+	private Object scheduleTask(ScheduledUnit task, boolean queueIfNoResource) throws NoResourceAvailableException {
 		if (task == null) {
 			throw new IllegalArgumentException();
 		}
 		
-		// if there is already a slot for that resource
-		AllocatedSlot existing = this.allocatedSlots.get(task.getSharedResourceId());
-		if (existing != null) {
-			// try to attach to the existing slot
-			if (existing.runTask(task.getTaskVertex())) {
-				// all good, we are done
-				return;
+		if (LOG.isDebugEnabled()) {
+			LOG.debug("Scheduling task " + task);
+		}
+	
+		synchronized (globalLock) {
+			// 1) If the task has a strict co-schedule hint, obey it, if it has been assigned.
+//			CoLocationHint hint = task.getCoScheduleHint();
+//			if (hint != null) {
+//				
+//				// try to add to the slot, or make it wait on the hint and schedule the hint itself
+//				if () {
+//					return slot;
+//				}
+//			}
+		
+			// 2) See if we can place the task somewhere together with another existing task.
+			//    This is defined by the slot sharing groups
+			SlotSharingGroup sharingUnit = task.getSlotSharingGroup();
+			if (sharingUnit != null) {
+				// see if we can add the task to the current sharing group.
+				SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
+				AllocatedSlot slot = assignment.getSlotForTask(task.getJobVertexId(), task.getTaskVertex());
+				if (slot != null) {
+					return slot;
+				}
+			}
+		
+			// 3) We could not schedule it to an existing slot, so we need to get a new one or queue the task
+			
+			// we need potentially to loop multiple times, because there may be false positives
+			// in the set-with-available-instances
+			while (true) {
+				Instance instanceToUse = getFreeInstanceForTask(task.getTaskVertex());
+			
+				if (instanceToUse != null) {
+					try {
+						AllocatedSlot slot = instanceToUse.allocateSlot(task.getTaskVertex().getJobId());
+						
+						// if the instance has further available slots, re-add it to the set of available resources.
+						if (instanceToUse.hasResourcesAvailable()) {
+							this.instancesWithAvailableResources.add(instanceToUse);
+						}
+						
+						if (slot != null) {
+							
+							// if the task is in a shared group, assign the slot to that group
+							// and get a sub slot in turn
+							if (sharingUnit != null) {
+								slot = sharingUnit.getTaskAssignment().addSlotWithTask(slot, task.getJobVertexId());
+							}
+							
+							// try to run the task 
+							if (slot.runTask(task.getTaskVertex())) {
+								return slot;
+							} else {
+								// did not assign, so we recycle the resource
+								slot.releaseSlot();
+							}
+						}
+					}
+					catch (InstanceDiedException e) {
+						// the instance died it has not yet been propagated to this scheduler
+						// remove the instance from the set of available instances
+						this.allInstances.remove(instanceToUse);
+						this.instancesWithAvailableResources.remove(instanceToUse);
+					}
+				}
+				else {
+					// no resource available now, so queue the request
+					if (queueIfNoResource) {
+						SlotAllocationFuture future = new SlotAllocationFuture();
+						this.taskQueue.add(new QueuedTask(task, future));
+						return future;
+					}
+					else {
+						throw new NoResourceAvailableException(task);
+					}
+				}
 			}
-			// else: the slot was deallocated, we need to proceed as if there was none
 		}
+	}
 		
-		// check if there is a slot that has an available sub-slot for that group-vertex
-		// TODO
+	/**
+	 * Gets a suitable instance to schedule the vertex execution to.
+	 * <p>
+	 * NOTE: This method does is not thread-safe, it needs to be synchronized by the caller.
+	 * 
+	 * @param vertex The task to run. 
+	 * @return The instance to run the vertex on, it {@code null}, if no instance is available.
+	 */
+	protected Instance getFreeInstanceForTask(ExecutionVertex2 vertex) {
+		if (this.instancesWithAvailableResources.isEmpty()) {
+			return null;
+		}
 		
+		Iterable<Instance> locationsIterable = vertex.getPreferredLocations();
+		Iterator<Instance> locations = locationsIterable == null ? null : locationsIterable.iterator();
 		
+		if (locations != null && locations.hasNext()) {
+			
+			while (locations.hasNext()) {
+				Instance location = locations.next();
+				
+				if (location != null && this.instancesWithAvailableResources.remove(location)) {
+					
+					localizedAssignments++;
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Local assignment: " + vertex.getSimpleName() + " --> " + location);
+					}
+					
+					return location;
+				}
+			}
+			
+			Instance instance = this.instancesWithAvailableResources.poll();
+			nonLocalizedAssignments++;
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Non-local assignment: " + vertex.getSimpleName() + " --> " + instance);
+			}
+			return instance;
+		}
+		else {
+			Instance instance = this.instancesWithAvailableResources.poll();
+			unconstrainedAssignments++;
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Unconstrained assignment: " + vertex.getSimpleName() + " --> " + instance);
+			}
+			
+			return instance;
+		}
+	}
+	
+	@Override
+	public void newSlotAvailable(Instance instance) {
+		
+		// global lock before instance lock, so that the order of acquiring locks is always 1) global, 2) instance
+		synchronized (globalLock) {
+			QueuedTask queued = taskQueue.peek();
+			
+			// the slot was properly released, we can allocate a new one from that instance
+			
+			if (queued != null) {
+				ScheduledUnit task = queued.getTask();
+				
+				try {
+					AllocatedSlot newSlot = instance.allocateSlot(task.getTaskVertex().getJobId());
+					if (newSlot != null && newSlot.runTask(task.getTaskVertex())) {
+						
+						// success, remove from the task queue and notify the future
+						taskQueue.poll();
+						if (queued.getFuture() != null) {
+							queued.getFuture().setSlot(newSlot);
+						}
+					}
+				}
+				catch (InstanceDiedException e) {
+					this.allInstances.remove(instance);
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Instance " + instance + " was marked dead asynchronously.");
+					}
+				}
+			}
+			else {
+				this.instancesWithAvailableResources.add(instance);
+			}
+		}
 	}
 
 	// --------------------------------------------------------------------------------------------
 	//  Instance Availability
 	// --------------------------------------------------------------------------------------------
 	
-	
 	@Override
 	public void newInstanceAvailable(Instance instance) {
 		if (instance == null) {
@@ -195,15 +318,28 @@ public class DefaultScheduler implements InstanceListener {
 		}
 		
 		// synchronize globally for instance changes
-		synchronized (this.lock) {
+		synchronized (this.globalLock) {
+			
 			// check we do not already use this instance
 			if (!this.allInstances.add(instance)) {
 				throw new IllegalArgumentException("The instance is already contained.");
 			}
 			
+			try {
+				instance.setSlotAvailabilityListener(this);
+			}
+			catch (IllegalStateException e) {
+				this.allInstances.remove(instance);
+				LOG.error("Scheduler could not attach to the instance as a listener.");
+			}
+			
+			
 			// add it to the available resources and let potential waiters know
 			this.instancesWithAvailableResources.add(instance);
-			this.lock.notifyAll();
+			
+			for (int i = 0; i < instance.getNumberOfAvailableSlots(); i++) {
+				newSlotAvailable(instance);
+			}
 		}
 	}
 	
@@ -216,173 +352,57 @@ public class DefaultScheduler implements InstanceListener {
 		instance.markDead();
 		
 		// we only remove the instance from the pools, we do not care about the 
-		synchronized (this.lock) {
+		synchronized (this.globalLock) {
 			// the instance must not be available anywhere any more
 			this.allInstances.remove(instance);
 			this.instancesWithAvailableResources.remove(instance);
 		}
 	}
 	
+	// --------------------------------------------------------------------------------------------
+	//  Status reporting
+	// --------------------------------------------------------------------------------------------
+
 	public int getNumberOfAvailableInstances() {
-		synchronized (lock) {
-			return allInstances.size();
-		}
+		return allInstances.size();
 	}
 	
-	// --------------------------------------------------------------------------------------------
-	//  Scheduling
-	// --------------------------------------------------------------------------------------------
+	public int getNumberOfInstancesWithAvailableSlots() {
+		return instancesWithAvailableResources.size();
+	}
 	
-//	/**
-//	 * Schedules the given unit to an available resource. This call blocks if no resource
-//	 * is currently available
-//	 * 
-//	 * @param unit The unit to be scheduled.
-//	 */
-//	protected void scheduleQueuedUnit(ScheduledUnit unit) {
-//		if (unit == null) {
-//			throw new IllegalArgumentException("Unit to schedule must not be null.");
-//		}
-//		
-//		// see if the resource Id has already an assigned resource
-//		AllocatedSlot resource = this.allocatedSlots.get(unit.getSharedResourceId());
-//		
-//		if (resource == null) {
-//			// not yet allocated. find a slot to schedule to
-//			try {
-//				resource = getResourceToScheduleUnit(unit, this.rejectIfNoResourceAvailable);
-//				if (resource == null) {
-//					throw new RuntimeException("Error: The resource to schedule to is null.");
-//				}
-//			}
-//			catch (Exception e) {
-//				// we cannot go on, the task needs to know what to do now.
-//				unit.getTaskVertex().handleException(e);
-//				return;
-//			}
-//		}
-//		
-//		resource.runTask(unit.getTaskVertex());
-//	}
+	public int getNumberOfUnconstrainedAssignments() {
+		return unconstrainedAssignments;
+	}
 	
-	/**
-	 * Acquires a resource to schedule the given unit to. This call may block if no
-	 * resource is currently available, or throw an exception, based on the given flag.
-	 * 
-	 * @param unit The unit to find a resource for.
-	 * @return The resource to schedule the execution of the given unit on.
-	 * 
-	 * @throws NoResourceAvailableException If the {@code exceptionOnNoAvailability} flag is true and the scheduler
-	 *                                      has currently no resources available.
-	 */
-	protected AllocatedSlot getNewSlotForTask(ScheduledUnit unit, boolean queueIfNoResource) 
-		throws NoResourceAvailableException
-	{
-		synchronized (this.lock) {
-			Instance instanceToUse = this.instancesWithAvailableResources.poll();
-			
-			// if there is nothing, throw an exception or wait, depending on what is configured
-			if (instanceToUse != null) {
-				try {
-					AllocatedSlot slot = instanceToUse.allocateSlot(unit.getJobId(), unit.getSharedResourceId());
-					
-					// if the instance has further available slots, re-add it to the set of available resources.
-					if (instanceToUse.hasResourcesAvailable()) {
-						this.instancesWithAvailableResources.add(instanceToUse);
-					}
-					
-					if (slot != null) {
-						AllocatedSlot previous = this.allocatedSlots.putIfAbsent(unit.getSharedResourceId(), slot);
-						if (previous != null) {
-							// concurrently, someone allocated a slot for that ID
-							// release the new one
-							slot.cancelResource();
-							slot = previous;
-						}
-					}
-					// else fall through the loop
-				}
-				catch (InstanceDiedException e) {
-					// the instance died it has not yet been propagated to this scheduler
-					// remove the instance from the set of available instances
-					this.allInstances.remove(instanceToUse);
-				}
-			}
-				
-			
-			if (queueIfNoResource) {
-				this.taskQueue.add(unit);
-			}
-			else {
-				throw new NoResourceAvailableException(unit);
-			}
-				// at this point, we have an instance. request a slot from the instance
-				
-				
-				// if the instance has further available slots, re-add it to the set of available
-				// resources.
-				// if it does not, but asynchronously a slot became available, we may attempt to add the
-				// instance twice, which does not matter because of the set semantics of the "instancesWithAvailableResources"
-				if (instanceToUse.hasResourcesAvailable()) {
-					this.instancesWithAvailableResources.add(instanceToUse);
-				}
-				
-				if (slot != null) {
-					AllocatedSlot previous = this.allocatedSlots.putIfAbsent(unit.getSharedResourceId(), slot);
-					if (previous != null) {
-						// concurrently, someone allocated a slot for that ID
-						// release the new one
-						slot.cancelResource();
-						slot = previous;
-					}
-				}
-				// else fall through the loop
-			}
-		}
-		
-		return slot;
+	public int getNumberOfLocalizedAssignments() {
+		return localizedAssignments;
 	}
 	
-	protected void runSchedulerLoop() {
-		// while the scheduler is alive
-		while (!shutdown.get()) {
-			
-			// get the next unit
-			ScheduledUnit next = null;
-			try {
-				next = this.taskQueue.take();
-			}
-			catch (InterruptedException e) {
-				if (shutdown.get()) {
-					return;
-				} else {
-					LOG.error("Scheduling loop was interrupted.");
-				}
-			}
-			
-			// if we see this special unit, it means we are done
-			if (next == TERMINATION_SIGNAL) {
-				return;
-			}
-			
-			// deploy the next scheduling unit
-			try {
-				scheduleNextUnit(next);
-			}
-			catch (Throwable t) {
-				// ignore the errors in the presence of a shutdown
-				if (!shutdown.get()) {
-					if (t instanceof Error) {
-						throw (Error) t;
-					} else if (t instanceof RuntimeException) {
-						throw (RuntimeException) t;
-					} else {
-						throw new RuntimeException("Critical error in scheduler thread.", t);
-					}
-				}
-			}
-		}
+	public int getNumberOfNonLocalizedAssignments() {
+		return nonLocalizedAssignments;
 	}
 	
-	private static final ScheduledUnit TERMINATION_SIGNAL = new ScheduledUnit();
+	// --------------------------------------------------------------------------------------------
+	
+	private static final class QueuedTask {
+		
+		private final ScheduledUnit task;
+		
+		private final SlotAllocationFuture future;
+		
+		
+		public QueuedTask(ScheduledUnit task, SlotAllocationFuture future) {
+			this.task = task;
+			this.future = future;
+		}
+
+		public ScheduledUnit getTask() {
+			return task;
+		}
+
+		public SlotAllocationFuture getFuture() {
+			return future;
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java
deleted file mode 100644
index 47aadf9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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.jobmanager.scheduler;
-
-import java.util.Comparator;
-
-import org.apache.flink.runtime.instance.Instance;
-
-public class InstanceFillDegreeComparator implements Comparator<Instance> {
-
-	@Override
-	public int compare(Instance o1, Instance o2) {
-		float fraction1 = o1.getNumberOfAvailableSlots() / (float) o1.getTotalNumberOfSlots();
-		float fraction2 = o2.getNumberOfAvailableSlots() / (float) o2.getTotalNumberOfSlots();
-		
-		return fraction1 < fraction2 ? -1 : 1;
-	}
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
index 338529f..2b0de6f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
@@ -1,71 +1,75 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.jobmanager.scheduler;
 
-import java.util.concurrent.atomic.AtomicBoolean;
-
 import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
-import org.apache.flink.runtime.jobgraph.JobID;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 public class ScheduledUnit {
 	
-	private final JobID jobId;
-	
 	private final ExecutionVertex2 taskVertex;
 	
-	private final ResourceId resourceId;
-	
-	private final AtomicBoolean scheduled = new AtomicBoolean(false);
+	private final SlotSharingGroup sharingGroup;
 	
+	// --------------------------------------------------------------------------------------------
 	
-	public ScheduledUnit(JobID jobId, ExecutionVertex2 taskVertex) {
-		this(jobId, taskVertex, new ResourceId());
+	public ScheduledUnit(ExecutionVertex2 taskVertex) {
+		if (taskVertex == null) {
+			throw new NullPointerException();
+		}
+		
+		this.taskVertex = taskVertex;
+		this.sharingGroup = null;
 	}
 	
-	public ScheduledUnit(JobID jobId, ExecutionVertex2 taskVertex, ResourceId resourceId) {
-		if (jobId == null || taskVertex == null || resourceId == null) {
+	public ScheduledUnit(ExecutionVertex2 taskVertex, SlotSharingGroup sharingUnit) {
+		if (taskVertex == null) {
 			throw new NullPointerException();
 		}
 		
-		this.jobId = jobId;
 		this.taskVertex = taskVertex;
-		this.resourceId = resourceId;
+		this.sharingGroup = sharingUnit;
 	}
 	
 	ScheduledUnit() {
-		this.jobId = null;
 		this.taskVertex = null;
-		this.resourceId = null;
+		this.sharingGroup = null;
 	}
 
+	// --------------------------------------------------------------------------------------------
 	
-	public JobID getJobId() {
-		return jobId;
+	public JobVertexID getJobVertexId() {
+		return this.taskVertex.getJobvertexId();
 	}
 	
 	public ExecutionVertex2 getTaskVertex() {
 		return taskVertex;
 	}
 	
-	public ResourceId getSharedResourceId() {
-		return resourceId;
+	public SlotSharingGroup getSlotSharingGroup() {
+		return sharingGroup;
 	}
+
+	// --------------------------------------------------------------------------------------------
 	
 	@Override
 	public String toString() {
-		return "(job=" + jobId + ", resourceId=" + resourceId + ", vertex=" + taskVertex + ')';
+		return "{vertex=" + taskVertex.getSimpleName() + ", sharingUnit=" + sharingGroup + '}';
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java
deleted file mode 100644
index 3bca46b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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.jobmanager.scheduler;
-
-/**
- * The scheduling strategy describes how scheduler distributes tasks across resources.
- */
-public enum SchedulingStrategy {
-	
-	/**
-	 * This strategy tries to keep all machines utilized roughly the same.
-	 */
-	SPREAD_OUT_TASKS,
-	
-	/**
-	 * This strategy will put as many tasks on one each machine as possible, before putting
-	 * tasks on the next machine.
-	 */
-	CLUSTER_TASKS
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueue.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueue.java
new file mode 100644
index 0000000..56b1146
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SetQueue.java
@@ -0,0 +1,134 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import java.util.AbstractQueue;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedHashSet;
+import java.util.Queue;
+
+/**
+ * A queue that maintains set characteristics, i.e., elements
+ * that are already in the queue may not be added another time.
+ *
+ * @param <E> The type of the elements in the queue.
+ */
+public class SetQueue<E> extends AbstractQueue<E> implements Queue<E> {
+
+	private final LinkedHashSet<E> set = new LinkedHashSet<E>();
+	
+	@Override
+	public boolean offer(E e) {
+		if (e == null) {
+			throw new NullPointerException();
+		}
+		
+		// may, or may not, add the element.
+		set.add(e);
+		
+		// we always return true, because the queue did not reject the element
+		// due to capacity constraints
+		return true;
+	}
+
+	@Override
+	public E poll() {
+		Iterator<E> iter = set.iterator();
+		if (iter.hasNext()) {
+			E next = iter.next();
+			iter.remove();
+			return next;
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public E peek() {
+		Iterator<E> iter = set.iterator();
+		if (iter.hasNext()) {
+			return iter.next();
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public Iterator<E> iterator() {
+		return set.iterator();
+	}
+
+	@Override
+	public int size() {
+		return set.size();
+	}
+	
+	@Override
+	public void clear() {
+		set.clear();
+	}
+	
+	@Override
+	public boolean remove(Object o) {
+		return set.remove(o);
+	}
+	
+	@Override
+	public boolean contains(Object o) {
+		return set.contains(o);
+	}
+	
+	@Override
+	public boolean removeAll(Collection<?> c) {
+		return set.removeAll(c);
+	}
+	
+	@Override
+	public boolean containsAll(Collection<?> c) {
+		return set.containsAll(c);
+	}
+	
+	@Override
+	public boolean retainAll(Collection<?> c) {
+		return set.retainAll(c);
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public int hashCode() {
+		return set.hashCode();
+	}
+	
+	@Override
+	public boolean equals(Object obj) {
+		if (obj != null && obj.getClass() == SetQueue.class) {
+			return set.equals(((SetQueue<?>) obj).set);
+		}
+		else {
+			return false;
+		}
+	}
+	
+	@Override
+	public String toString() {
+		return set.toString();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
new file mode 100644
index 0000000..36d8a8b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
@@ -0,0 +1,99 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+
+public class SharedSlot {
+
+	private final AllocatedSlot allocatedSlot;
+	
+	private final SlotSharingGroupAssignment assignmentGroup;
+	
+	private final Set<SubSlot> subSlots;
+	
+	private int subSlotNumber;
+	
+	private volatile boolean disposed;
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public SharedSlot(AllocatedSlot allocatedSlot, SlotSharingGroupAssignment assignmentGroup) {
+		if (allocatedSlot == null || assignmentGroup == null) {
+			throw new NullPointerException();
+		}
+		
+		this.allocatedSlot = allocatedSlot;
+		this.assignmentGroup = assignmentGroup;
+		this.subSlots = new HashSet<SubSlot>();
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public AllocatedSlot getAllocatedSlot() {
+		return this.allocatedSlot;
+	}
+	
+	public boolean isDisposed() {
+		return disposed;
+	}
+	
+	public int getNumberOfAllocatedSubSlots() {
+		synchronized (this.subSlots) {
+			return this.subSlots.size();
+		}
+	}
+	
+	public SubSlot allocateSubSlot(JobVertexID jid) {
+		synchronized (this.subSlots) {
+			if (isDisposed()) {
+				return null;
+			} else {
+				SubSlot ss = new SubSlot(this, subSlotNumber++, jid);
+				this.subSlots.add(ss);
+				return ss;
+			}
+		}
+	}
+	
+	void returnAllocatedSlot(SubSlot slot) {
+		boolean release;
+		
+		synchronized (this.subSlots) {
+			if (!this.subSlots.remove(slot)) {
+				throw new IllegalArgumentException("Wrong shared slot for subslot.");
+			}
+			
+			release = assignmentGroup.sharedSlotAvailableForJid(this, slot.getJobVertexId(), this.subSlots.isEmpty());
+			
+			if (release) {
+				disposed = true;
+			}
+		}
+		
+		// do this call outside the lock, because releasing the allocated slot may go into further scheduler calls
+		if (release) {
+			this.allocatedSlot.releaseSlot();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/e6aadfcc/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java
new file mode 100644
index 0000000..679cccc
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotAllocationFuture.java
@@ -0,0 +1,94 @@
+/**
+ * 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.jobmanager.scheduler;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
+
+public class SlotAllocationFuture {
+	
+	private final Object monitor = new Object();
+	
+	private volatile AllocatedSlot slot;
+	
+	private volatile SlotAllocationFutureAction action;
+	
+	// --------------------------------------------------------------------------------------------
+
+	public SlotAllocationFuture() {}
+	
+	public SlotAllocationFuture(AllocatedSlot slot) {
+		this.slot = slot;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	public AllocatedSlot waitTillAllocated() throws InterruptedException {
+		return waitTillAllocated(0);
+	}
+	
+	public AllocatedSlot waitTillAllocated(long timeout) throws InterruptedException {
+		synchronized (monitor) {
+			while (slot == null) {
+				monitor.wait(timeout);
+			}
+			
+			return slot;
+		}
+	}
+	
+	public void setFutureAction(SlotAllocationFutureAction action) {
+		synchronized (monitor) {
+			if (this.action != null) {
+				throw new IllegalStateException("Future already has an action registered.");
+			}
+			
+			this.action = action;
+			
+			if (this.slot != null) {
+				action.slotAllocated(this.slot);
+			}
+		}
+	}
+	
+	public void setSlot(AllocatedSlot slot) {
+		if (slot == null) {
+			throw new NullPointerException();
+		}
+		
+		synchronized (monitor) {
+			if (this.slot != null) {
+				throw new IllegalStateException("The future has already been assigned a slot.");
+			}
+			
+			this.slot = slot;
+			monitor.notifyAll();
+			
+			if (action != null) {
+				action.slotAllocated(slot);
+			}
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return slot == null ? "PENDING" : "DONE";
+	}
+}


[30/63] [abbrv] Redesign Scheduler from pre-assignment to more flexible schedule-on-demand model

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
index 701f802..d23d35f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/DefaultScheduler.java
@@ -16,757 +16,963 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.jobmanager.scheduler;
 
-import java.util.ArrayList;
-import java.util.Collection;
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
 import java.util.Map;
+import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.Deque;
-import java.util.ArrayDeque;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.flink.runtime.execution.ExecutionState;
-import org.apache.flink.runtime.executiongraph.ExecutionEdge;
-import org.apache.flink.runtime.executiongraph.ExecutionGate;
-import org.apache.flink.runtime.executiongraph.ExecutionGraph;
-import org.apache.flink.runtime.executiongraph.ExecutionGraphIterator;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionGroupVertexIterator;
-import org.apache.flink.runtime.executiongraph.ExecutionPipeline;
-import org.apache.flink.runtime.executiongraph.ExecutionStage;
-import org.apache.flink.runtime.executiongraph.ExecutionStageListener;
-import org.apache.flink.runtime.executiongraph.ExecutionVertex;
-import org.apache.flink.runtime.executiongraph.ExecutionVertexID;
-import org.apache.flink.runtime.executiongraph.InternalJobStatus;
-import org.apache.flink.runtime.executiongraph.JobStatusListener;
-import org.apache.flink.runtime.instance.AllocatedResource;
-import org.apache.flink.runtime.instance.AllocationID;
-import org.apache.flink.runtime.instance.DummyInstance;
+
+import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
-import org.apache.flink.runtime.instance.InstanceException;
+import org.apache.flink.runtime.instance.InstanceDiedException;
 import org.apache.flink.runtime.instance.InstanceListener;
-import org.apache.flink.runtime.instance.InstanceManager;
 import org.apache.flink.runtime.jobgraph.JobID;
-import org.apache.flink.runtime.jobmanager.DeploymentManager;
-import org.apache.flink.util.StringUtils;
 
 /**
- * The default scheduler for Nephele. While Nephele's
- * {@link org.apache.flink.runtime.jobmanager.JobManager} is responsible for requesting the required instances for the
- * job at the {@link org.apache.flink.runtime.instance.InstanceManager}, the scheduler is in charge of assigning the
- * individual tasks to the instances.
- * 
+ * The scheduler is responsible for distributing the ready-to-run tasks and assigning them to instances and
+ * slots.
+ * <p>
+ * The scheduler's bookkeeping on the available instances is lazy: It is not modified once an
+ * instance is dead, but it will lazily remove the instance from its pool as soon as it tries
+ * to allocate a resource on that instance and it fails with an {@link InstanceDiedException}.
  */
-public class DefaultScheduler implements InstanceListener, JobStatusListener, ExecutionStageListener {
+public class DefaultScheduler implements InstanceListener {
 
-	/**
-	 * The LOG object to report events within the scheduler.
-	 */
 	protected static final Logger LOG = LoggerFactory.getLogger(DefaultScheduler.class);
 
-	/**
-	 * The instance manager assigned to this scheduler.
-	 */
-	private final InstanceManager instanceManager;
-
-	/**
-	 * The deployment manager assigned to this scheduler.
-	 */
-	private final DeploymentManager deploymentManager;
-
-	/**
-	 * Stores the vertices to be restarted once they have switched to the <code>CANCELED</code> state.
-	 */
-	private final Map<ExecutionVertexID, ExecutionVertex> verticesToBeRestarted = new ConcurrentHashMap<ExecutionVertexID, ExecutionVertex>();
-
-	/**
-	 * The job queue where all submitted jobs go to.
-	 */
-	private Deque<ExecutionGraph> jobQueue = new ArrayDeque<ExecutionGraph>();
-
-	/**
-	 * Constructs a new abstract scheduler.
-	 * 
-	 * @param deploymentManager
-	 *        the deployment manager assigned to this scheduler
-	 * @param instanceManager
-	 *        the instance manager to be used with this scheduler
-	 */
-	public DefaultScheduler(final DeploymentManager deploymentManager, final InstanceManager instanceManager) {
-
-		this.deploymentManager = deploymentManager;
-		this.instanceManager = instanceManager;
-		this.instanceManager.setInstanceListener(this);
+	
+	private final Object lock = new Object();
+	
+	/** All instances that the scheduler can deploy to */
+	private final Set<Instance> allInstances = new HashSet<Instance>();
+	
+	/** All instances that still have available resources */
+	private final Queue<Instance> instancesWithAvailableResources = new LifoSetQueue<Instance>();
+
+	
+	private final ConcurrentHashMap<ResourceId, AllocatedSlot> allocatedSlots = new ConcurrentHashMap<ResourceId, AllocatedSlot>();
+	
+//	/** A cache that remembers the last resource IDs it has seen, to co-locate future
+//	 *  deployments of tasks with the same resource ID to the same instance.
+//	 */
+//	private final Cache<ResourceId, Instance> ghostCache;
+	
+	
+	/** All tasks pending to be scheduled */
+	private final LinkedBlockingQueue<ScheduledUnit> taskQueue = new LinkedBlockingQueue<ScheduledUnit>();
+
+	
+	/** The thread that runs the scheduling loop, picking up tasks to be scheduled and scheduling them. */
+	private final Thread schedulerThread;
+	
+	
+	/** Atomic flag to safely control the shutdown */
+	private final AtomicBoolean shutdown = new AtomicBoolean(false);
+	
+	/** Flag indicating whether the scheduler should reject a unit if it cannot find a resource
+	 * for it at the time of scheduling */
+	private final boolean rejectIfNoResourceAvailable;
+	
+
+	
+	public DefaultScheduler() {
+		this(true);
 	}
-
-	/**
-	 * Removes the job represented by the given {@link ExecutionGraph} from the scheduler.
-	 *
-	 * @param executionGraphToRemove
-	 *        the job to be removed
-	 */
-	void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) {
-
-		boolean removedFromQueue = false;
-
-		synchronized (this.jobQueue) {
-
-			final Iterator<ExecutionGraph> it = this.jobQueue.iterator();
-			while (it.hasNext()) {
-
-				final ExecutionGraph executionGraph = it.next();
-				if (executionGraph.getJobID().equals(executionGraphToRemove.getJobID())) {
-					removedFromQueue = true;
-					it.remove();
-					break;
-				}
+	
+	public DefaultScheduler(boolean rejectIfNoResourceAvailable) {
+		this.rejectIfNoResourceAvailable = rejectIfNoResourceAvailable;
+		
+		
+//		this.ghostCache = CacheBuilder.newBuilder()
+//				.initialCapacity(64)	// easy start
+//				.maximumSize(1024)		// retain some history
+//				.weakValues()			// do not prevent dead instances from being collected
+//				.build();
+		
+		// set up (but do not start) the scheduling thread
+		Runnable loopRunner = new Runnable() {
+			@Override
+			public void run() {
+				runSchedulerLoop();
 			}
-		}
-
-		if (!removedFromQueue) {
-			LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " ("
-					+ executionGraphToRemove.getJobID() + ") to remove");
-		}
+		};
+		this.schedulerThread = new Thread(loopRunner, "Scheduling Thread");
 	}
-
-	/**
-	 * Adds a job represented by an {@link ExecutionGraph} object to the scheduler. The job is then executed according
-	 * to the strategies of the concrete scheduler implementation.
-	 *
-	 * @param executionGraph
-	 *        the job to be added to the scheduler
-	 * @throws SchedulingException
-	 *         thrown if an error occurs and the scheduler does not accept the new job
-	 */
-	public void scheduleJob(final ExecutionGraph executionGraph) throws SchedulingException {
-
-		final int requiredSlots = executionGraph.getRequiredSlots();
-		final int availableSlots = this.getInstanceManager().getNumberOfSlots();
-
-		if(requiredSlots > availableSlots){
-			throw new SchedulingException(String.format(
-					"Not enough available task slots to run job %s (%s). Required: %d Available: %d . "
-					+ "Either reduce the parallelism of your program, wait for other programs to finish, or increase "
-					+ "the number of task slots in the cluster by adding more machines or increasing the number of slots "
-					+ "per machine in conf/flink-conf.yaml .", 
-					executionGraph.getJobName(), executionGraph.getJobID(), requiredSlots, availableSlots));
-		}
-
-		// Subscribe to job status notifications
-		executionGraph.registerJobStatusListener(this);
-
-		// Register execution listener for each vertex
-		final ExecutionGraphIterator it2 = new ExecutionGraphIterator(executionGraph, true);
-		while (it2.hasNext()) {
-
-			final ExecutionVertex vertex = it2.next();
-			vertex.registerExecutionListener(new DefaultExecutionListener(this, vertex));
-		}
-
-		// Register the scheduler as an execution stage listener
-		executionGraph.registerExecutionStageListener(this);
-
-		// Add job to the job queue (important to add job to queue before requesting instances)
-		synchronized (this.jobQueue) {
-			this.jobQueue.add(executionGraph);
+	
+	public void start() {
+		if (shutdown.get()) {
+			throw new IllegalStateException("Scheduler has been shut down.");
 		}
-
-		// Request resources for the first stage of the job
-
-		final ExecutionStage executionStage = executionGraph.getCurrentExecutionStage();
+		
 		try {
-			requestInstances(executionStage);
-		} catch (InstanceException e) {
-			final String exceptionMessage = StringUtils.stringifyException(e);
-			LOG.error(exceptionMessage);
-			this.jobQueue.remove(executionGraph);
-			throw new SchedulingException(exceptionMessage);
+			this.schedulerThread.start();
 		}
-	}
-
-	/**
-	 * Returns the execution graph which is associated with the given job ID.
-	 *
-	 * @param jobID
-	 *        the job ID to search the execution graph for
-	 * @return the execution graph which belongs to the given job ID or <code>null</code if no such execution graph
-	 *         exists
-	 */
-	public ExecutionGraph getExecutionGraphByID(final JobID jobID) {
-
-		synchronized (this.jobQueue) {
-
-			final Iterator<ExecutionGraph> it = this.jobQueue.iterator();
-			while (it.hasNext()) {
-
-				final ExecutionGraph executionGraph = it.next();
-				if (executionGraph.getJobID().equals(jobID)) {
-					return executionGraph;
-				}
-			}
+		catch (IllegalThreadStateException e) {
+			throw new IllegalStateException("The scheduler has already been started.");
 		}
-
-		return null;
 	}
-
+	
 	/**
-	 * Shuts the scheduler down. After shut down no jobs can be added to the scheduler.
+	 * Shuts the scheduler down. After shut down no more tasks can be added to the scheduler.
 	 */
 	public void shutdown() {
-
-		synchronized (this.jobQueue) {
-			this.jobQueue.clear();
+		if (this.shutdown.compareAndSet(false, true)) {
+			// clear the task queue and add the termination signal, to let
+			// the scheduling loop know that things are done
+			this.taskQueue.clear();
+			this.taskQueue.add(TERMINATION_SIGNAL);
+			
+			// interrupt the scheduling thread, in case it was waiting for resources to
+			// show up to deploy a task
+			this.schedulerThread.interrupt();
 		}
-
 	}
-
-	public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
-									final String optionalMessage) {
-
-		if (newJobStatus == InternalJobStatus.FAILED || newJobStatus == InternalJobStatus.FINISHED
-				|| newJobStatus == InternalJobStatus.CANCELED) {
-			removeJobFromSchedule(executionGraph);
-		}
-	}
-
-	public void nextExecutionStageEntered(final JobID jobID, final ExecutionStage executionStage) {
-
-		// Request new instances if necessary
-		try {
-			requestInstances(executionStage);
-		} catch (InstanceException e) {
-			// TODO: Handle error correctly
-			LOG.error(StringUtils.stringifyException(e));
+	
+	public void setUncaughtExceptionHandler(UncaughtExceptionHandler handler) {
+		if (this.schedulerThread.getState() != Thread.State.NEW) {
+			throw new IllegalStateException("Can only add exception handler before starting the scheduler.");
 		}
-
-		// Deploy the assigned vertices
-		deployAssignedInputVertices(executionStage.getExecutionGraph());
+		this.schedulerThread.setUncaughtExceptionHandler(handler);
 	}
 
-
-	/**
-	 * Returns the {@link org.apache.flink.runtime.instance.InstanceManager} object which is used by the current scheduler.
-	 * 
-	 * @return the {@link org.apache.flink.runtime.instance.InstanceManager} object which is used by the current scheduler
-	 */
-	public InstanceManager getInstanceManager() {
-		return this.instanceManager;
+//	/**
+//	 * Removes the job represented by the given {@link ExecutionGraph} from the scheduler.
+//	 *
+//	 * @param executionGraphToRemove
+//	 *        the job to be removed
+//	 */
+//	void removeJobFromSchedule(final ExecutionGraph executionGraphToRemove) {
+//
+//		boolean removedFromQueue = false;
+//
+//		synchronized (this.jobQueue) {
+//
+//			final Iterator<ExecutionGraph> it = this.jobQueue.iterator();
+//			while (it.hasNext()) {
+//
+//				final ExecutionGraph executionGraph = it.next();
+//				if (executionGraph.getJobID().equals(executionGraphToRemove.getJobID())) {
+//					removedFromQueue = true;
+//					it.remove();
+//					break;
+//				}
+//			}
+//		}
+//
+//		if (!removedFromQueue) {
+//			LOG.error("Cannot find job " + executionGraphToRemove.getJobName() + " ("
+//					+ executionGraphToRemove.getJobID() + ") to remove");
+//		}
+//	}
+//
+//	/**
+//	 * Adds a job represented by an {@link ExecutionGraph} object to the scheduler. The job is then executed according
+//	 * to the strategies of the concrete scheduler implementation.
+//	 *
+//	 * @param executionGraph
+//	 *        the job to be added to the scheduler
+//	 * @throws SchedulingException
+//	 *         thrown if an error occurs and the scheduler does not accept the new job
+//	 */
+//	public void scheduleJob(final ExecutionGraph executionGraph) throws SchedulingException {
+//
+//		final int requiredSlots = executionGraph.getRequiredSlots();
+//		final int availableSlots = this.getInstanceManager().getNumberOfSlots();
+//
+//		if(requiredSlots > availableSlots){
+//			throw new SchedulingException(String.format(
+//					"Not enough available task slots to run job %s (%s). Required: %d Available: %d . "
+//					+ "Either reduce the parallelism of your program, wait for other programs to finish, or increase "
+//					+ "the number of task slots in the cluster by adding more machines or increasing the number of slots "
+//					+ "per machine in conf/flink-conf.yaml .", 
+//					executionGraph.getJobName(), executionGraph.getJobID(), requiredSlots, availableSlots));
+//		}
+//
+//		// Subscribe to job status notifications
+//		executionGraph.registerJobStatusListener(this);
+//
+//		// Register execution listener for each vertex
+//		final ExecutionGraphIterator it2 = new ExecutionGraphIterator(executionGraph, true);
+//		while (it2.hasNext()) {
+//
+//			final ExecutionVertex vertex = it2.next();
+//			vertex.registerExecutionListener(new DefaultExecutionListener(this, vertex));
+//		}
+//
+//		// Register the scheduler as an execution stage listener
+//		executionGraph.registerExecutionStageListener(this);
+//
+//		// Add job to the job queue (important to add job to queue before requesting instances)
+//		synchronized (this.jobQueue) {
+//			this.jobQueue.add(executionGraph);
+//		}
+//
+//		// Request resources for the first stage of the job
+//
+//		final ExecutionStage executionStage = executionGraph.getCurrentExecutionStage();
+//		try {
+//			requestInstances(executionStage);
+//		} catch (InstanceException e) {
+//			final String exceptionMessage = StringUtils.stringifyException(e);
+//			LOG.error(exceptionMessage);
+//			this.jobQueue.remove(executionGraph);
+//			throw new SchedulingException(exceptionMessage);
+//		}
+//	}
+//
+//	/**
+//	 * Returns the execution graph which is associated with the given job ID.
+//	 *
+//	 * @param jobID
+//	 *        the job ID to search the execution graph for
+//	 * @return the execution graph which belongs to the given job ID or <code>null</code if no such execution graph
+//	 *         exists
+//	 */
+//	public ExecutionGraph getExecutionGraphByID(final JobID jobID) {
+//
+//		synchronized (this.jobQueue) {
+//
+//			final Iterator<ExecutionGraph> it = this.jobQueue.iterator();
+//			while (it.hasNext()) {
+//
+//				final ExecutionGraph executionGraph = it.next();
+//				if (executionGraph.getJobID().equals(jobID)) {
+//					return executionGraph;
+//				}
+//			}
+//		}
+//
+//		return null;
+//	}
+//
+//
+//
+//	public void jobStatusHasChanged(final ExecutionGraph executionGraph, final InternalJobStatus newJobStatus,
+//									final String optionalMessage) {
+//
+//		if (newJobStatus == InternalJobStatus.FAILED || newJobStatus == InternalJobStatus.FINISHED
+//				|| newJobStatus == InternalJobStatus.CANCELED) {
+//			removeJobFromSchedule(executionGraph);
+//		}
+//	}
+//
+//	public void nextExecutionStageEntered(final JobID jobID, final ExecutionStage executionStage) {
+//
+//		// Request new instances if necessary
+//		try {
+//			requestInstances(executionStage);
+//		} catch (InstanceException e) {
+//			// TODO: Handle error correctly
+//			LOG.error(StringUtils.stringifyException(e));
+//		}
+//
+//		// Deploy the assigned vertices
+//		deployAssignedInputVertices(executionStage.getExecutionGraph());
+//	}
+//
+//
+//	/**
+//	 * Returns the {@link org.apache.flink.runtime.instance.InstanceManager} object which is used by the current scheduler.
+//	 * 
+//	 * @return the {@link org.apache.flink.runtime.instance.InstanceManager} object which is used by the current scheduler
+//	 */
+//	public InstanceManager getInstanceManager() {
+//		return this.instanceManager;
+//	}
+//
+//
+//	/**
+//	 * Collects the instances required to run the job from the given {@link ExecutionStage} and requests them at the
+//	 * loaded instance manager.
+//	 * 
+//	 * @param executionStage
+//	 *        the execution stage to collect the required instances from
+//	 * @throws InstanceException
+//	 *         thrown if the given execution graph is already processing its final stage
+//	 */
+//	protected void requestInstances(final ExecutionStage executionStage) throws InstanceException {
+//
+//		final ExecutionGraph executionGraph = executionStage.getExecutionGraph();
+//
+//		synchronized (executionStage) {
+//
+//			final int requiredSlots = executionStage.getRequiredSlots();
+//
+//			LOG.info("Requesting " + requiredSlots + " slots for job " + executionGraph.getJobID());
+//
+//			this.instanceManager.requestInstance(executionGraph.getJobID(), executionGraph.getJobConfiguration(),
+//				requiredSlots);
+//
+//			// Switch vertex state to assigning
+//			final ExecutionGraphIterator it2 = new ExecutionGraphIterator(executionGraph, executionGraph
+//				.getIndexOfCurrentExecutionStage(), true, true);
+//			while (it2.hasNext()) {
+//
+//				it2.next().compareAndUpdateExecutionState(ExecutionState.CREATED, ExecutionState.SCHEDULED);
+//			}
+//		}
+//	}
+//
+//	void findVerticesToBeDeployed(final ExecutionVertex vertex,
+//			final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed,
+//			final Set<ExecutionVertex> alreadyVisited) {
+//
+//		if (!alreadyVisited.add(vertex)) {
+//			return;
+//		}
+//
+//		if (vertex.compareAndUpdateExecutionState(ExecutionState.ASSIGNED, ExecutionState.READY)) {
+//			final Instance instance = vertex.getAllocatedResource().getInstance();
+//
+//			if (instance instanceof DummyInstance) {
+//				LOG.error("Inconsistency: Vertex " + vertex + " is about to be deployed on a DummyInstance");
+//			}
+//
+//			List<ExecutionVertex> verticesForInstance = verticesToBeDeployed.get(instance);
+//			if (verticesForInstance == null) {
+//				verticesForInstance = new ArrayList<ExecutionVertex>();
+//				verticesToBeDeployed.put(instance, verticesForInstance);
+//			}
+//
+//			verticesForInstance.add(vertex);
+//		}
+//
+//		final int numberOfOutputGates = vertex.getNumberOfOutputGates();
+//		for (int i = 0; i < numberOfOutputGates; ++i) {
+//
+//			final ExecutionGate outputGate = vertex.getOutputGate(i);
+//			boolean deployTarget;
+//
+//			switch (outputGate.getChannelType()) {
+//			case NETWORK:
+//				deployTarget = false;
+//				break;
+//			case IN_MEMORY:
+//				deployTarget = true;
+//				break;
+//			default:
+//				throw new IllegalStateException("Unknown channel type");
+//			}
+//
+//			if (deployTarget) {
+//
+//				final int numberOfOutputChannels = outputGate.getNumberOfEdges();
+//				for (int j = 0; j < numberOfOutputChannels; ++j) {
+//					final ExecutionEdge outputChannel = outputGate.getEdge(j);
+//					final ExecutionVertex connectedVertex = outputChannel.getInputGate().getVertex();
+//					findVerticesToBeDeployed(connectedVertex, verticesToBeDeployed, alreadyVisited);
+//				}
+//			}
+//		}
+//	}
+//
+//	/**
+//	 * Collects all execution vertices with the state ASSIGNED starting from the given start vertex and
+//	 * deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
+//	 * 
+//	 * @param startVertex
+//	 *        the execution vertex to start the deployment from
+//	 */
+//	public void deployAssignedVertices(final ExecutionVertex startVertex) {
+//
+//		final JobID jobID = startVertex.getExecutionGraph().getJobID();
+//
+//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
+//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
+//
+//		findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited);
+//
+//		if (!verticesToBeDeployed.isEmpty()) {
+//
+//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
+//				.entrySet()
+//				.iterator();
+//
+//			while (it2.hasNext()) {
+//
+//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
+//				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
+//			}
+//		}
+//	}
+//
+//	/**
+//	 * Collects all execution vertices with the state ASSIGNED from the given pipeline and deploys them on the assigned
+//	 * {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
+//	 * 
+//	 * @param pipeline
+//	 *        the execution pipeline to be deployed
+//	 */
+//	public void deployAssignedPipeline(final ExecutionPipeline pipeline) {
+//
+//		final JobID jobID = null;
+//
+//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
+//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
+//
+//		final Iterator<ExecutionVertex> it = pipeline.iterator();
+//		while (it.hasNext()) {
+//			findVerticesToBeDeployed(it.next(), verticesToBeDeployed, alreadyVisited);
+//		}
+//
+//		if (!verticesToBeDeployed.isEmpty()) {
+//
+//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
+//				.entrySet()
+//				.iterator();
+//
+//			while (it2.hasNext()) {
+//
+//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
+//				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
+//			}
+//		}
+//	}
+//
+//	/**
+//	 * Collects all execution vertices with the state ASSIGNED starting from the given collection of start vertices and
+//	 * deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
+//	 * 
+//	 * @param startVertices
+//	 *        the collection of execution vertices to start the deployment from
+//	 */
+//	public void deployAssignedVertices(final Collection<ExecutionVertex> startVertices) {
+//
+//		JobID jobID = null;
+//
+//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
+//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
+//
+//		for (final ExecutionVertex startVertex : startVertices) {
+//
+//			if (jobID == null) {
+//				jobID = startVertex.getExecutionGraph().getJobID();
+//			}
+//
+//			findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited);
+//		}
+//
+//		if (!verticesToBeDeployed.isEmpty()) {
+//
+//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
+//				.entrySet()
+//				.iterator();
+//
+//			while (it2.hasNext()) {
+//
+//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
+//				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
+//			}
+//		}
+//	}
+//
+//	/**
+//	 * Collects all execution vertices with the state ASSIGNED starting from the input vertices of the current execution
+//	 * stage and deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
+//	 * 
+//	 * @param executionGraph
+//	 *        the execution graph to collect the vertices from
+//	 */
+//	public void deployAssignedInputVertices(final ExecutionGraph executionGraph) {
+//
+//		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
+//		final ExecutionStage executionStage = executionGraph.getCurrentExecutionStage();
+//
+//		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
+//
+//		for (int i = 0; i < executionStage.getNumberOfStageMembers(); ++i) {
+//
+//			final ExecutionGroupVertex startVertex = executionStage.getStageMember(i);
+//			if (!startVertex.isInputVertex()) {
+//				continue;
+//			}
+//
+//			for (int j = 0; j < startVertex.getCurrentNumberOfGroupMembers(); ++j) {
+//				final ExecutionVertex vertex = startVertex.getGroupMember(j);
+//				findVerticesToBeDeployed(vertex, verticesToBeDeployed, alreadyVisited);
+//			}
+//		}
+//
+//		if (!verticesToBeDeployed.isEmpty()) {
+//
+//			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
+//				.entrySet()
+//				.iterator();
+//
+//			while (it2.hasNext()) {
+//
+//				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
+//				this.deploymentManager.deploy(executionGraph.getJobID(), entry.getKey(), entry.getValue());
+//			}
+//		}
+//	}
+//
+//
+//	@Override
+//	public void resourcesAllocated(final JobID jobID, final List<AllocatedResource> allocatedResources) {
+//
+//		if (allocatedResources == null) {
+//			LOG.error("Resource to lock is null!");
+//			return;
+//		}
+//
+//		for (final AllocatedResource allocatedResource : allocatedResources) {
+//			if (allocatedResource.getInstance() instanceof DummyInstance) {
+//				LOG.debug("Available instance is of type DummyInstance!");
+//				return;
+//			}
+//		}
+//
+//		final ExecutionGraph eg = getExecutionGraphByID(jobID);
+//
+//		if (eg == null) {
+//			/*
+//			 * The job have have been canceled in the meantime, in this case
+//			 * we release the instance immediately.
+//			 */
+//			try {
+//				for (final AllocatedResource allocatedResource : allocatedResources) {
+//					getInstanceManager().releaseAllocatedResource(allocatedResource);
+//				}
+//			} catch (InstanceException e) {
+//				LOG.error(e);
+//			}
+//			return;
+//		}
+//
+//		final Runnable command = new Runnable() {
+//
+//			/**
+//			 * {@inheritDoc}
+//			 */
+//			@Override
+//			public void run() {
+//
+//				final ExecutionStage stage = eg.getCurrentExecutionStage();
+//
+//				synchronized (stage) {
+//
+//					for (final AllocatedResource allocatedResource : allocatedResources) {
+//
+//						AllocatedResource resourceToBeReplaced = null;
+//						// Important: only look for instances to be replaced in the current stage
+//						final Iterator<ExecutionGroupVertex> groupIterator = new ExecutionGroupVertexIterator(eg, true,
+//							stage.getStageNumber());
+//						while (groupIterator.hasNext()) {
+//
+//							final ExecutionGroupVertex groupVertex = groupIterator.next();
+//							for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); ++i) {
+//
+//								final ExecutionVertex vertex = groupVertex.getGroupMember(i);
+//
+//								if (vertex.getExecutionState() == ExecutionState.SCHEDULED
+//									&& vertex.getAllocatedResource() != null) {
+//										resourceToBeReplaced = vertex.getAllocatedResource();
+//										break;
+//								}
+//							}
+//
+//							if (resourceToBeReplaced != null) {
+//								break;
+//							}
+//						}
+//
+//						// For some reason, we don't need this instance
+//						if (resourceToBeReplaced == null) {
+//							LOG.error("Instance " + allocatedResource.getInstance() + " is not required for job"
+//								+ eg.getJobID());
+//							try {
+//								getInstanceManager().releaseAllocatedResource(allocatedResource);
+//							} catch (InstanceException e) {
+//								LOG.error(e);
+//							}
+//							return;
+//						}
+//
+//						// Replace the selected instance
+//						final Iterator<ExecutionVertex> it = resourceToBeReplaced.assignedVertices();
+//						while (it.hasNext()) {
+//							final ExecutionVertex vertex = it.next();
+//							vertex.setAllocatedResource(allocatedResource);
+//							vertex.updateExecutionState(ExecutionState.ASSIGNED);
+//						}
+//					}
+//				}
+//
+//				// Deploy the assigned vertices
+//				deployAssignedInputVertices(eg);
+//
+//			}
+//
+//		};
+//
+//		eg.executeCommand(command);
+//	}
+//
+//	/**
+//	 * Checks if the given {@link AllocatedResource} is still required for the
+//	 * execution of the given execution graph. If the resource is no longer
+//	 * assigned to a vertex that is either currently running or about to run
+//	 * the given resource is returned to the instance manager for deallocation.
+//	 * 
+//	 * @param executionGraph
+//	 *        the execution graph the provided resource has been used for so far
+//	 * @param allocatedResource
+//	 *        the allocated resource to check the assignment for
+//	 */
+//	public void checkAndReleaseAllocatedResource(final ExecutionGraph executionGraph,
+//			final AllocatedResource allocatedResource) {
+//
+//		if (allocatedResource == null) {
+//			LOG.error("Resource to lock is null!");
+//			return;
+//		}
+//
+//		if (allocatedResource.getInstance() instanceof DummyInstance) {
+//			LOG.debug("Available instance is of type DummyInstance!");
+//			return;
+//		}
+//
+//		boolean resourceCanBeReleased = true;
+//		final Iterator<ExecutionVertex> it = allocatedResource.assignedVertices();
+//		while (it.hasNext()) {
+//			final ExecutionVertex vertex = it.next();
+//			final ExecutionState state = vertex.getExecutionState();
+//
+//			if (state != ExecutionState.CREATED && state != ExecutionState.FINISHED
+//				&& state != ExecutionState.FAILED && state != ExecutionState.CANCELED) {
+//
+//				resourceCanBeReleased = false;
+//				break;
+//			}
+//		}
+//
+//		if (resourceCanBeReleased) {
+//
+//			LOG.info("Releasing instance " + allocatedResource.getInstance());
+//			try {
+//				getInstanceManager().releaseAllocatedResource(allocatedResource);
+//			} catch (InstanceException e) {
+//				LOG.error(StringUtils.stringifyException(e));
+//			}
+//		}
+//	}
+//
+//	DeploymentManager getDeploymentManager() {
+//		return this.deploymentManager;
+//	}
+//
+//
+//
+//	@Override
+//	public void allocatedResourcesDied(final JobID jobID, final List<AllocatedResource> allocatedResources) {
+//
+//		final ExecutionGraph eg = getExecutionGraphByID(jobID);
+//
+//		if (eg == null) {
+//			LOG.error("Cannot find execution graph for job with ID " + jobID);
+//			return;
+//		}
+//
+//		final Runnable command = new Runnable() {
+//
+//			/**
+//			 * {@inheritDoc}
+//			 */
+//			@Override
+//			public void run() {
+//
+//				synchronized (eg) {
+//
+//					for (final AllocatedResource allocatedResource : allocatedResources) {
+//
+//						LOG.info("Resource " + allocatedResource.getInstance().getName() + " for Job " + jobID
+//							+ " died.");
+//
+//						final ExecutionGraph executionGraph = getExecutionGraphByID(jobID);
+//
+//						if (executionGraph == null) {
+//							LOG.error("Cannot find execution graph for job " + jobID);
+//							return;
+//						}
+//
+//						Iterator<ExecutionVertex> vertexIter = allocatedResource.assignedVertices();
+//
+//						// Assign vertices back to a dummy resource.
+//						final DummyInstance dummyInstance = DummyInstance.createDummyInstance();
+//						final AllocatedResource dummyResource = new AllocatedResource(dummyInstance,
+//								new AllocationID());
+//
+//						while (vertexIter.hasNext()) {
+//							final ExecutionVertex vertex = vertexIter.next();
+//							vertex.setAllocatedResource(dummyResource);
+//						}
+//
+//						final String failureMessage = allocatedResource.getInstance().getName() + " died";
+//
+//						vertexIter = allocatedResource.assignedVertices();
+//
+//						while (vertexIter.hasNext()) {
+//							final ExecutionVertex vertex = vertexIter.next();
+//							final ExecutionState state = vertex.getExecutionState();
+//
+//							switch (state) {
+//							case ASSIGNED:
+//							case READY:
+//							case STARTING:
+//							case RUNNING:
+//							case FINISHING:
+//
+//							vertex.updateExecutionState(ExecutionState.FAILED, failureMessage);
+//
+//							break;
+//						default:
+//							}
+//					}
+//
+//					// TODO: Fix this
+//					/*
+//					 * try {
+//					 * requestInstances(this.executionVertex.getGroupVertex().getExecutionStage());
+//					 * } catch (InstanceException e) {
+//					 * e.printStackTrace();
+//					 * // TODO: Cancel the entire job in this case
+//					 * }
+//					 */
+//				}
+//			}
+//
+//			final InternalJobStatus js = eg.getJobStatus();
+//			if (js != InternalJobStatus.FAILING && js != InternalJobStatus.FAILED) {
+//
+//				// TODO: Fix this
+//				// deployAssignedVertices(eg);
+//
+//				final ExecutionStage stage = eg.getCurrentExecutionStage();
+//
+//				try {
+//					requestInstances(stage);
+//				} catch (InstanceException e) {
+//					e.printStackTrace();
+//					// TODO: Cancel the entire job in this case
+//				}
+//			}
+//		}
+//		};
+//
+//		eg.executeCommand(command);
+//	}
+	
+	// --------------------------------------------------------------------------------------------
+	//  Canceling
+	// --------------------------------------------------------------------------------------------
+	
+	public void removeAllTasksForJob(JobID job) {
+		
 	}
 
-
-	/**
-	 * Collects the instances required to run the job from the given {@link ExecutionStage} and requests them at the
-	 * loaded instance manager.
-	 * 
-	 * @param executionStage
-	 *        the execution stage to collect the required instances from
-	 * @throws InstanceException
-	 *         thrown if the given execution graph is already processing its final stage
-	 */
-	protected void requestInstances(final ExecutionStage executionStage) throws InstanceException {
-
-		final ExecutionGraph executionGraph = executionStage.getExecutionGraph();
-
-		synchronized (executionStage) {
-
-			final int requiredSlots = executionStage.getRequiredSlots();
-
-			LOG.info("Requesting " + requiredSlots + " slots for job " + executionGraph.getJobID());
-
-			this.instanceManager.requestInstance(executionGraph.getJobID(), executionGraph.getJobConfiguration(),
-				requiredSlots);
-
-			// Switch vertex state to assigning
-			final ExecutionGraphIterator it2 = new ExecutionGraphIterator(executionGraph, executionGraph
-				.getIndexOfCurrentExecutionStage(), true, true);
-			while (it2.hasNext()) {
-
-				it2.next().compareAndUpdateExecutionState(ExecutionState.CREATED, ExecutionState.SCHEDULED);
-			}
+	// --------------------------------------------------------------------------------------------
+	//  Instance Availability
+	// --------------------------------------------------------------------------------------------
+	
+	
+	@Override
+	public void newInstanceAvailable(Instance instance) {
+		if (instance == null) {
+			throw new IllegalArgumentException();
 		}
-	}
-
-	void findVerticesToBeDeployed(final ExecutionVertex vertex,
-			final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed,
-			final Set<ExecutionVertex> alreadyVisited) {
-
-		if (!alreadyVisited.add(vertex)) {
-			return;
+		if (instance.getNumberOfAvailableSlots() <= 0) {
+			throw new IllegalArgumentException("The given instance has no resources.");
 		}
-
-		if (vertex.compareAndUpdateExecutionState(ExecutionState.ASSIGNED, ExecutionState.READY)) {
-			final Instance instance = vertex.getAllocatedResource().getInstance();
-
-			if (instance instanceof DummyInstance) {
-				LOG.error("Inconsistency: Vertex " + vertex + " is about to be deployed on a DummyInstance");
-			}
-
-			List<ExecutionVertex> verticesForInstance = verticesToBeDeployed.get(instance);
-			if (verticesForInstance == null) {
-				verticesForInstance = new ArrayList<ExecutionVertex>();
-				verticesToBeDeployed.put(instance, verticesForInstance);
-			}
-
-			verticesForInstance.add(vertex);
+		if (!instance.isAlive()) {
+			throw new IllegalArgumentException("The instance is not alive.");
 		}
-
-		final int numberOfOutputGates = vertex.getNumberOfOutputGates();
-		for (int i = 0; i < numberOfOutputGates; ++i) {
-
-			final ExecutionGate outputGate = vertex.getOutputGate(i);
-			boolean deployTarget;
-
-			switch (outputGate.getChannelType()) {
-			case NETWORK:
-				deployTarget = false;
-				break;
-			case IN_MEMORY:
-				deployTarget = true;
-				break;
-			default:
-				throw new IllegalStateException("Unknown channel type");
-			}
-
-			if (deployTarget) {
-
-				final int numberOfOutputChannels = outputGate.getNumberOfEdges();
-				for (int j = 0; j < numberOfOutputChannels; ++j) {
-					final ExecutionEdge outputChannel = outputGate.getEdge(j);
-					final ExecutionVertex connectedVertex = outputChannel.getInputGate().getVertex();
-					findVerticesToBeDeployed(connectedVertex, verticesToBeDeployed, alreadyVisited);
-				}
+		
+		// synchronize globally for instance changes
+		synchronized (this.lock) {
+			// check we do not already use this instance
+			if (!this.allInstances.add(instance)) {
+				throw new IllegalArgumentException("The instance is already contained.");
 			}
+			
+			// add it to the available resources and let potential waiters know
+			this.instancesWithAvailableResources.add(instance);
+			this.lock.notifyAll();
 		}
 	}
-
-	/**
-	 * Collects all execution vertices with the state ASSIGNED starting from the given start vertex and
-	 * deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
-	 * 
-	 * @param startVertex
-	 *        the execution vertex to start the deployment from
-	 */
-	public void deployAssignedVertices(final ExecutionVertex startVertex) {
-
-		final JobID jobID = startVertex.getExecutionGraph().getJobID();
-
-		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-
-		findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited);
-
-		if (!verticesToBeDeployed.isEmpty()) {
-
-			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
-			}
+	
+	@Override
+	public void instanceDied(Instance instance) {
+		if (instance == null) {
+			throw new IllegalArgumentException();
 		}
-	}
-
-	/**
-	 * Collects all execution vertices with the state ASSIGNED from the given pipeline and deploys them on the assigned
-	 * {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
-	 * 
-	 * @param pipeline
-	 *        the execution pipeline to be deployed
-	 */
-	public void deployAssignedPipeline(final ExecutionPipeline pipeline) {
-
-		final JobID jobID = null;
-
-		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-
-		final Iterator<ExecutionVertex> it = pipeline.iterator();
-		while (it.hasNext()) {
-			findVerticesToBeDeployed(it.next(), verticesToBeDeployed, alreadyVisited);
+		
+		instance.markDead();
+		
+		// we only remove the instance from the pools, we do not care about the 
+		synchronized (this.lock) {
+			// the instance must not be available anywhere any more
+			this.allInstances.remove(instance);
+			this.instancesWithAvailableResources.remove(instance);
 		}
-
-		if (!verticesToBeDeployed.isEmpty()) {
-
-			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
-			}
+	}
+	
+	public int getNumberOfAvailableInstances() {
+		synchronized (lock) {
+			return allInstances.size();
 		}
 	}
-
+	
+	// --------------------------------------------------------------------------------------------
+	//  Scheduling
+	// --------------------------------------------------------------------------------------------
+	
 	/**
-	 * Collects all execution vertices with the state ASSIGNED starting from the given collection of start vertices and
-	 * deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
+	 * Schedules the given unit to an available resource. This call blocks if no resource
+	 * is currently available
 	 * 
-	 * @param startVertices
-	 *        the collection of execution vertices to start the deployment from
+	 * @param unit The unit to be scheduled.
 	 */
-	public void deployAssignedVertices(final Collection<ExecutionVertex> startVertices) {
-
-		JobID jobID = null;
-
-		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-
-		for (final ExecutionVertex startVertex : startVertices) {
-
-			if (jobID == null) {
-				jobID = startVertex.getExecutionGraph().getJobID();
-			}
-
-			findVerticesToBeDeployed(startVertex, verticesToBeDeployed, alreadyVisited);
+	protected void scheduleNextUnit(ScheduledUnit unit) {
+		if (unit == null) {
+			throw new IllegalArgumentException("Unit to schedule must not be null.");
 		}
-
-		if (!verticesToBeDeployed.isEmpty()) {
-
-			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-				this.deploymentManager.deploy(jobID, entry.getKey(), entry.getValue());
+		
+		// see if the resource Id has already an assigned resource
+		AllocatedSlot resource = this.allocatedSlots.get(unit.getSharedResourceId());
+		
+		if (resource == null) {
+			// not yet allocated. find a slot to schedule to
+			try {
+				resource = getResourceToScheduleUnit(unit, this.rejectIfNoResourceAvailable);
+				if (resource == null) {
+					throw new RuntimeException("Error: The resource to schedule to is null.");
+				}
+			}
+			catch (Exception e) {
+				// we cannot go on, the task needs to know what to do now.
+				unit.getTaskVertex().handleException(e);
+				return;
 			}
 		}
+		
+		resource.runTask(unit.getTaskVertex());
 	}
-
+	
 	/**
-	 * Collects all execution vertices with the state ASSIGNED starting from the input vertices of the current execution
-	 * stage and deploys them on the assigned {@link org.apache.flink.runtime.instance.AllocatedResource} objects.
+	 * Acquires a resource to schedule the given unit to. This call may block if no
+	 * resource is currently available, or throw an exception, based on the given flag.
 	 * 
-	 * @param executionGraph
-	 *        the execution graph to collect the vertices from
+	 * @param unit The unit to find a resource for.
+	 * @param exceptionOnNoAvailability If true, this call should not block is no resource is available,
+	 *                                  but throw a {@link NoResourceAvailableException}.
+	 * @return The resource to schedule the execution of the given unit on.
+	 * 
+	 * @throws NoResourceAvailableException If the {@code exceptionOnNoAvailability} flag is true and the scheduler
+	 *                                      has currently no resources available.
 	 */
-	public void deployAssignedInputVertices(final ExecutionGraph executionGraph) {
-
-		final Map<Instance, List<ExecutionVertex>> verticesToBeDeployed = new HashMap<Instance, List<ExecutionVertex>>();
-		final ExecutionStage executionStage = executionGraph.getCurrentExecutionStage();
-
-		final Set<ExecutionVertex> alreadyVisited = new HashSet<ExecutionVertex>();
-
-		for (int i = 0; i < executionStage.getNumberOfStageMembers(); ++i) {
-
-			final ExecutionGroupVertex startVertex = executionStage.getStageMember(i);
-			if (!startVertex.isInputVertex()) {
-				continue;
-			}
-
-			for (int j = 0; j < startVertex.getCurrentNumberOfGroupMembers(); ++j) {
-				final ExecutionVertex vertex = startVertex.getGroupMember(j);
-				findVerticesToBeDeployed(vertex, verticesToBeDeployed, alreadyVisited);
-			}
-		}
-
-		if (!verticesToBeDeployed.isEmpty()) {
-
-			final Iterator<Map.Entry<Instance, List<ExecutionVertex>>> it2 = verticesToBeDeployed
-				.entrySet()
-				.iterator();
-
-			while (it2.hasNext()) {
-
-				final Map.Entry<Instance, List<ExecutionVertex>> entry = it2.next();
-				this.deploymentManager.deploy(executionGraph.getJobID(), entry.getKey(), entry.getValue());
-			}
-		}
-	}
-
-
-	@Override
-	public void resourcesAllocated(final JobID jobID, final List<AllocatedResource> allocatedResources) {
-
-		if (allocatedResources == null) {
-			LOG.error("Resource to lock is null!");
-			return;
-		}
-
-		for (final AllocatedResource allocatedResource : allocatedResources) {
-			if (allocatedResource.getInstance() instanceof DummyInstance) {
-				LOG.debug("Available instance is of type DummyInstance!");
-				return;
-			}
-		}
-
-		final ExecutionGraph eg = getExecutionGraphByID(jobID);
-
-		if (eg == null) {
-			/*
-			 * The job have have been canceled in the meantime, in this case
-			 * we release the instance immediately.
-			 */
-			try {
-				for (final AllocatedResource allocatedResource : allocatedResources) {
-					getInstanceManager().releaseAllocatedResource(allocatedResource);
-				}
-			} catch (InstanceException e) {
-				LOG.error("InstanceException while releasing allocated ressources.", e);
-			}
-			return;
-		}
-
-		final Runnable command = new Runnable() {
-
-			/**
-			 * {@inheritDoc}
-			 */
-			@Override
-			public void run() {
-
-				final ExecutionStage stage = eg.getCurrentExecutionStage();
-
-				synchronized (stage) {
-
-					for (final AllocatedResource allocatedResource : allocatedResources) {
-
-						AllocatedResource resourceToBeReplaced = null;
-						// Important: only look for instances to be replaced in the current stage
-						final Iterator<ExecutionGroupVertex> groupIterator = new ExecutionGroupVertexIterator(eg, true,
-							stage.getStageNumber());
-						while (groupIterator.hasNext()) {
-
-							final ExecutionGroupVertex groupVertex = groupIterator.next();
-							for (int i = 0; i < groupVertex.getCurrentNumberOfGroupMembers(); ++i) {
-
-								final ExecutionVertex vertex = groupVertex.getGroupMember(i);
-
-								if (vertex.getExecutionState() == ExecutionState.SCHEDULED
-									&& vertex.getAllocatedResource() != null) {
-										resourceToBeReplaced = vertex.getAllocatedResource();
-										break;
-								}
-							}
-
-							if (resourceToBeReplaced != null) {
-								break;
+	protected AllocatedSlot getResourceToScheduleUnit(ScheduledUnit unit, boolean exceptionOnNoAvailability) 
+		throws NoResourceAvailableException
+	{
+		AllocatedSlot slot = null;
+		
+		while (true) {
+			synchronized (this.lock) {
+				Instance instanceToUse = this.instancesWithAvailableResources.poll();
+				
+				// if there is nothing, throw an exception or wait, depending on what is configured
+				if (instanceToUse == null) {
+					if (exceptionOnNoAvailability) {
+						throw new NoResourceAvailableException(unit);
+					}
+					else {
+						try {
+							do {
+								this.lock.wait(2000);
 							}
+							while (!shutdown.get() && 
+									(instanceToUse = this.instancesWithAvailableResources.poll()) == null);
 						}
-
-						// For some reason, we don't need this instance
-						if (resourceToBeReplaced == null) {
-							LOG.error("Instance " + allocatedResource.getInstance() + " is not required for job"
-								+ eg.getJobID());
-							try {
-								getInstanceManager().releaseAllocatedResource(allocatedResource);
-							} catch (InstanceException e) {
-								LOG.error("InstanceException while releasing allocated ressources.", e);
-							}
-							return;
-						}
-
-						// Replace the selected instance
-						final Iterator<ExecutionVertex> it = resourceToBeReplaced.assignedVertices();
-						while (it.hasNext()) {
-							final ExecutionVertex vertex = it.next();
-							vertex.setAllocatedResource(allocatedResource);
-							vertex.updateExecutionState(ExecutionState.ASSIGNED);
+						catch (InterruptedException e) {
+							throw new NoResourceAvailableException("The scheduler was interrupted.");
 						}
 					}
 				}
-
-				// Deploy the assigned vertices
-				deployAssignedInputVertices(eg);
-
-			}
-
-		};
-
-		eg.executeCommand(command);
-	}
-
-	/**
-	 * Checks if the given {@link AllocatedResource} is still required for the
-	 * execution of the given execution graph. If the resource is no longer
-	 * assigned to a vertex that is either currently running or about to run
-	 * the given resource is returned to the instance manager for deallocation.
-	 * 
-	 * @param executionGraph
-	 *        the execution graph the provided resource has been used for so far
-	 * @param allocatedResource
-	 *        the allocated resource to check the assignment for
-	 */
-	public void checkAndReleaseAllocatedResource(final ExecutionGraph executionGraph,
-			final AllocatedResource allocatedResource) {
-
-		if (allocatedResource == null) {
-			LOG.error("Resource to lock is null!");
-			return;
-		}
-
-		if (allocatedResource.getInstance() instanceof DummyInstance) {
-			LOG.debug("Available instance is of type DummyInstance!");
-			return;
-		}
-
-		boolean resourceCanBeReleased = true;
-		final Iterator<ExecutionVertex> it = allocatedResource.assignedVertices();
-		while (it.hasNext()) {
-			final ExecutionVertex vertex = it.next();
-			final ExecutionState state = vertex.getExecutionState();
-
-			if (state != ExecutionState.CREATED && state != ExecutionState.FINISHED
-				&& state != ExecutionState.FAILED && state != ExecutionState.CANCELED) {
-
-				resourceCanBeReleased = false;
-				break;
+				
+				// at this point, we have an instance. request a slot from the instance
+				try {
+					slot = instanceToUse.allocateSlot(unit.getJobId(), unit.getSharedResourceId());
+				}
+				catch (InstanceDiedException e) {
+					// the instance died it has not yet been propagated to this scheduler
+					// remove the instance from the set of available instances
+					this.allInstances.remove(instanceToUse);
+				}
+				
+				// if the instance has further available slots, re-add it to the set of available
+				// resources.
+				// if it does not, but asynchronously 
+				if (instanceToUse.hasResourcesAvailable()) {
+					this.instancesWithAvailableResources.add(instanceToUse);
+				}
+				
+				if (slot != null) {
+					return slot;
+				}
+				// else fall through the loop
 			}
 		}
-
-		if (resourceCanBeReleased) {
-
-			LOG.info("Releasing instance " + allocatedResource.getInstance());
+	}
+	
+	protected void runSchedulerLoop() {
+		// while the scheduler is alive
+		while (!shutdown.get()) {
+			
+			// get the next unit
+			ScheduledUnit next = null;
 			try {
-				getInstanceManager().releaseAllocatedResource(allocatedResource);
-			} catch (InstanceException e) {
-				LOG.error(StringUtils.stringifyException(e));
+				next = this.taskQueue.take();
 			}
-		}
-	}
-
-	DeploymentManager getDeploymentManager() {
-		return this.deploymentManager;
-	}
-
-	protected void replayCheckpointsFromPreviousStage(final ExecutionGraph executionGraph) {
-
-		final int currentStageIndex = executionGraph.getIndexOfCurrentExecutionStage();
-		final ExecutionStage previousStage = executionGraph.getStage(currentStageIndex - 1);
-
-		final List<ExecutionVertex> verticesToBeReplayed = new ArrayList<ExecutionVertex>();
-
-		for (int i = 0; i < previousStage.getNumberOfOutputExecutionVertices(); ++i) {
-
-			final ExecutionVertex vertex = previousStage.getOutputExecutionVertex(i);
-			vertex.updateExecutionState(ExecutionState.ASSIGNED);
-			verticesToBeReplayed.add(vertex);
-		}
-
-		deployAssignedVertices(verticesToBeReplayed);
-	}
-
-	/**
-	 * Returns a map of vertices to be restarted once they have switched to their <code>CANCELED</code> state.
-	 * 
-	 * @return the map of vertices to be restarted
-	 */
-	Map<ExecutionVertexID, ExecutionVertex> getVerticesToBeRestarted() {
-
-		return this.verticesToBeRestarted;
-	}
-
-
-	@Override
-	public void allocatedResourcesDied(final JobID jobID, final List<AllocatedResource> allocatedResources) {
-
-		final ExecutionGraph eg = getExecutionGraphByID(jobID);
-
-		if (eg == null) {
-			LOG.error("Cannot find execution graph for job with ID " + jobID);
-			return;
-		}
-
-		final Runnable command = new Runnable() {
-
-			/**
-			 * {@inheritDoc}
-			 */
-			@Override
-			public void run() {
-
-				synchronized (eg) {
-
-					for (final AllocatedResource allocatedResource : allocatedResources) {
-
-						LOG.info("Resource " + allocatedResource.getInstance().getName() + " for Job " + jobID
-							+ " died.");
-
-						final ExecutionGraph executionGraph = getExecutionGraphByID(jobID);
-
-						if (executionGraph == null) {
-							LOG.error("Cannot find execution graph for job " + jobID);
-							return;
-						}
-
-						Iterator<ExecutionVertex> vertexIter = allocatedResource.assignedVertices();
-
-						// Assign vertices back to a dummy resource.
-						final DummyInstance dummyInstance = DummyInstance.createDummyInstance();
-						final AllocatedResource dummyResource = new AllocatedResource(dummyInstance,
-								new AllocationID());
-
-						while (vertexIter.hasNext()) {
-							final ExecutionVertex vertex = vertexIter.next();
-							vertex.setAllocatedResource(dummyResource);
-						}
-
-						final String failureMessage = allocatedResource.getInstance().getName() + " died";
-
-						vertexIter = allocatedResource.assignedVertices();
-
-						while (vertexIter.hasNext()) {
-							final ExecutionVertex vertex = vertexIter.next();
-							final ExecutionState state = vertex.getExecutionState();
-
-							switch (state) {
-							case ASSIGNED:
-							case READY:
-							case STARTING:
-							case RUNNING:
-							case FINISHING:
-
-							vertex.updateExecutionState(ExecutionState.FAILED, failureMessage);
-
-							break;
-						default:
-							}
-					}
-
-					// TODO: Fix this
-					/*
-					 * try {
-					 * requestInstances(this.executionVertex.getGroupVertex().getExecutionStage());
-					 * } catch (InstanceException e) {
-					 * e.printStackTrace();
-					 * // TODO: Cancel the entire job in this case
-					 * }
-					 */
+			catch (InterruptedException e) {
+				if (shutdown.get()) {
+					return;
+				} else {
+					LOG.error("Scheduling loop was interrupted.");
 				}
 			}
-
-			final InternalJobStatus js = eg.getJobStatus();
-			if (js != InternalJobStatus.FAILING && js != InternalJobStatus.FAILED) {
-
-				// TODO: Fix this
-				// deployAssignedVertices(eg);
-
-				final ExecutionStage stage = eg.getCurrentExecutionStage();
-
-				try {
-					requestInstances(stage);
-				} catch (InstanceException e) {
-					e.printStackTrace();
-					// TODO: Cancel the entire job in this case
+			
+			// if we see this special unit, it means we are done
+			if (next == TERMINATION_SIGNAL) {
+				return;
+			}
+			
+			// deploy the next scheduling unit
+			try {
+				scheduleNextUnit(next);
+			}
+			catch (Throwable t) {
+				// ignore the errors in the presence of a shutdown
+				if (!shutdown.get()) {
+					if (t instanceof Error) {
+						throw (Error) t;
+					} else if (t instanceof RuntimeException) {
+						throw (RuntimeException) t;
+					} else {
+						throw new RuntimeException("Critical error in scheduler thread.", t);
+					}
 				}
 			}
 		}
-		};
-
-		eg.executeCommand(command);
 	}
+	
+	private static final ScheduledUnit TERMINATION_SIGNAL = new ScheduledUnit();
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java
new file mode 100644
index 0000000..47aadf9
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/InstanceFillDegreeComparator.java
@@ -0,0 +1,31 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.scheduler;
+
+import java.util.Comparator;
+
+import org.apache.flink.runtime.instance.Instance;
+
+public class InstanceFillDegreeComparator implements Comparator<Instance> {
+
+	@Override
+	public int compare(Instance o1, Instance o2) {
+		float fraction1 = o1.getNumberOfAvailableSlots() / (float) o1.getTotalNumberOfSlots();
+		float fraction2 = o2.getNumberOfAvailableSlots() / (float) o2.getTotalNumberOfSlots();
+		
+		return fraction1 < fraction2 ? -1 : 1;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java
new file mode 100644
index 0000000..d8bb852
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/LifoSetQueue.java
@@ -0,0 +1,110 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.scheduler;
+
+import java.util.AbstractQueue;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import java.util.Queue;
+
+/**
+ * A queue that returns elements in LIFO order and simultaneously maintains set characteristics, i.e., elements
+ * that are already in the queue may not be added another time.
+ *
+ * @param <E> The type of the elements in the queue.
+ */
+public class LifoSetQueue<E> extends AbstractQueue<E> implements Queue<E> {
+
+	private final ArrayList<E> lifo = new ArrayList<E>();
+	
+	private final HashSet<E> set = new HashSet<E>();
+	
+	@Override
+	public boolean offer(E e) {
+		if (e == null) {
+			throw new NullPointerException();
+		}
+		
+		if (set.add(e)) {
+			lifo.add(e);
+		}
+		
+		return true;
+	}
+
+	@Override
+	public E poll() {
+		int size = lifo.size();
+		if (size > 0) {
+			E element = lifo.remove(size-1);
+			set.remove(element);
+			return element;
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public E peek() {
+		int size = lifo.size();
+		if (size > 0) {
+			return lifo.get(size-1);
+		} else {
+			return null;
+		}
+	}
+
+	@Override
+	public Iterator<E> iterator() {
+		return new Iterator<E>() {
+			
+			private int currentPos = lifo.size() - 1;
+			private int posToRemove = -1;
+
+			@Override
+			public boolean hasNext() {
+				return currentPos >= 0;
+			}
+
+			@Override
+			public E next() {
+				if (!hasNext()) {
+					throw new NoSuchElementException();
+				} else {
+					posToRemove = currentPos;
+					return lifo.get(currentPos--);
+				}
+			}
+
+			@Override
+			public void remove() {
+				if (posToRemove == -1) {
+					throw new NoSuchElementException();
+				} else {
+					E element = lifo.remove(posToRemove);
+					set.remove(element);
+				}
+			}
+		};
+	}
+
+	@Override
+	public int size() {
+		return lifo.size();
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
new file mode 100644
index 0000000..2b76d26
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/NoResourceAvailableException.java
@@ -0,0 +1,33 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.scheduler;
+
+public class NoResourceAvailableException extends Exception {
+
+	private static final long serialVersionUID = -2249953165298717803L;
+
+	public NoResourceAvailableException() {
+		super();
+	}
+	
+	public NoResourceAvailableException(ScheduledUnit unit) {
+		super("No resource available to schedule unit " + unit);
+	}
+
+	public NoResourceAvailableException(String message) {
+		super(message);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java
new file mode 100644
index 0000000..58833f4
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java
@@ -0,0 +1,20 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.scheduler;
+
+import org.apache.flink.runtime.AbstractID;
+
+public class ResourceId extends AbstractID {}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
new file mode 100644
index 0000000..8caf64a
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduledUnit.java
@@ -0,0 +1,67 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.scheduler;
+
+import org.apache.flink.runtime.executiongraph.ExecutionVertex2;
+import org.apache.flink.runtime.jobgraph.JobID;
+
+public class ScheduledUnit {
+	
+	private final JobID jobId;
+	
+	private final ExecutionVertex2 taskVertex;
+	
+	private final ResourceId resourceId;
+	
+	
+	public ScheduledUnit(JobID jobId, ExecutionVertex2 taskVertex) {
+		this(jobId, taskVertex, new ResourceId());
+	}
+	
+	public ScheduledUnit(JobID jobId, ExecutionVertex2 taskVertex, ResourceId resourceId) {
+		if (jobId == null || taskVertex == null || resourceId == null) {
+			throw new NullPointerException();
+		}
+		
+		this.jobId = jobId;
+		this.taskVertex = taskVertex;
+		this.resourceId = resourceId;
+	}
+	
+	ScheduledUnit() {
+		this.jobId = null;
+		this.taskVertex = null;
+		this.resourceId = null;
+	}
+
+	
+	public JobID getJobId() {
+		return jobId;
+	}
+	
+	public ExecutionVertex2 getTaskVertex() {
+		return taskVertex;
+	}
+	
+	public ResourceId getSharedResourceId() {
+		return resourceId;
+	}
+	
+	@Override
+	public String toString() {
+		return "(job=" + jobId + ", resourceId=" + resourceId + ", vertex=" + taskVertex + ')';
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingException.java
deleted file mode 100644
index 7187cfb..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * 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.jobmanager.scheduler;
-
-/**
- * Scheduling exceptions are thrown to indicate problems or errors
- * related to Nephele's scheduler.
- * 
- */
-public class SchedulingException extends Exception {
-
-	/**
-	 * Generated serial version UID.
-	 */
-	private static final long serialVersionUID = 1L;
-
-	/**
-	 * Constructs a new scheduling exception object.
-	 * 
-	 * @param msg
-	 *        the error message of the exception
-	 */
-	public SchedulingException(String msg) {
-		super(msg);
-	}
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java
new file mode 100644
index 0000000..3bca46b
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulingStrategy.java
@@ -0,0 +1,33 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.scheduler;
+
+/**
+ * The scheduling strategy describes how scheduler distributes tasks across resources.
+ */
+public enum SchedulingStrategy {
+	
+	/**
+	 * This strategy tries to keep all machines utilized roughly the same.
+	 */
+	SPREAD_OUT_TASKS,
+	
+	/**
+	 * This strategy will put as many tasks on one each machine as possible, before putting
+	 * tasks on the next machine.
+	 */
+	CLUSTER_TASKS
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/2d6199ff/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
index 85ede42..9c46ac5 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/JobManagerProtocol.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.protocols;
 
 import java.io.IOException;
@@ -24,41 +23,34 @@ import java.io.IOException;
 import org.apache.flink.core.protocols.VersionedProtocol;
 import org.apache.flink.runtime.instance.HardwareDescription;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
+import org.apache.flink.runtime.instance.InstanceID;
 import org.apache.flink.runtime.taskmanager.TaskExecutionState;
-import org.apache.flink.runtime.taskmanager.transferenvelope.RegisterTaskManagerResult;
-import org.apache.flink.runtime.types.IntegerRecord;
 
 /**
  * The job manager protocol is implemented by the job manager and offers functionality
  * to task managers which allows them to register themselves, send heart beat messages
  * or to report the results of a task execution.
- * 
  */
 public interface JobManagerProtocol extends VersionedProtocol {
 
 	/**
 	 * Sends a heart beat to the job manager.
 	 * 
-	 * @param instanceConnectionInfo
-	 *        the information the job manager requires to connect to the instance's task manager
-	 * @throws IOException
-	 *         thrown if an error occurs during this remote procedure call
+	 * @param taskManagerId The ID identifying the task manager.
+	 * @throws IOException Thrown if an error occurs during this remote procedure call.
 	 */
-	void sendHeartbeat(InstanceConnectionInfo instanceConnectionInfo)
-			throws IOException;
+	boolean sendHeartbeat(InstanceID taskManagerId) throws IOException;
 
 	/**
 	 * Registers a task manager at the JobManager.
 	 *
 	 * @param instanceConnectionInfo the information the job manager requires to connect to the instance's task manager
 	 * @param hardwareDescription a hardware description with details on the instance's compute resources.
-	 * @throws IOException
+	 * @param numberOfSlots The number of task slots that the TaskManager provides.
 	 *
-	 * @return whether the task manager was successfully registered
+	 * @return The ID under which the TaskManager is registered. Null, if the JobManager does not register the TaskManager.
 	 */
-	RegisterTaskManagerResult registerTaskManager(InstanceConnectionInfo instanceConnectionInfo,
-						HardwareDescription hardwareDescription, IntegerRecord numberOfSlots)
-			throws IOException;
+	InstanceID registerTaskManager(InstanceConnectionInfo instanceConnectionInfo, HardwareDescription hardwareDescription, int numberOfSlots) throws IOException;
 
 	/**
 	 * Reports an update of a task's execution state to the job manager.


[14/63] [abbrv] Refactor job graph construction to incremental attachment based

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/b32e77a2/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java
new file mode 100644
index 0000000..a7cbeaf
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionEdge2.java
@@ -0,0 +1,74 @@
+/**
+ * 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 org.apache.flink.runtime.io.network.channels.ChannelID;
+
+public class ExecutionEdge2 {
+
+	private final IntermediateResultPartition source;
+	
+	private final ExecutionVertex2 target;
+	
+	private final int inputNum;
+
+	private final ChannelID inputChannelId;
+	
+	private final ChannelID outputChannelId;
+	
+	
+	public ExecutionEdge2(IntermediateResultPartition source, ExecutionVertex2 target, int inputNum) {
+		this.source = source;
+		this.target = target;
+		this.inputNum = inputNum;
+		
+		this.inputChannelId = new ChannelID();
+		this.outputChannelId = new ChannelID();
+	}
+	
+	public ExecutionEdge2(IntermediateResultPartition source, ExecutionVertex2 target, int inputNum, ChannelID inputChannelId, ChannelID outputChannelId) {
+		this.source = source;
+		this.target = target;
+		this.inputNum = inputNum;
+		
+		this.inputChannelId = inputChannelId;
+		this.outputChannelId = outputChannelId;
+	}
+	
+	
+	public IntermediateResultPartition getSource() {
+		return source;
+	}
+	
+	public ExecutionVertex2 getTarget() {
+		return target;
+	}
+	
+	public int getInputNum() {
+		return inputNum;
+	}
+	
+	public ChannelID getInputChannelId() {
+		return inputChannelId;
+	}
+	
+	public ChannelID getOutputChannelId() {
+		return outputChannelId;
+	}
+}


[19/63] [abbrv] git commit: Make IDs immutable and serializable.

Posted by se...@apache.org.
Make IDs immutable and serializable.


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/9035b6d0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/9035b6d0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/9035b6d0

Branch: refs/heads/master
Commit: 9035b6d0cfbe02a3b5d6d6726d3bd41e50586fdb
Parents: cb7039e
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Jun 23 19:44:18 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:02:48 2014 +0200

----------------------------------------------------------------------
 .../org/apache/flink/runtime/AbstractID.java    | 14 ++-------
 .../executiongraph/ExecutionVertexID.java       | 32 +++++++++++++-------
 .../flink/runtime/instance/InstanceID.java      |  4 ++-
 .../runtime/io/network/channels/ChannelID.java  |  6 ++--
 .../flink/runtime/io/network/gates/GateID.java  |  4 +--
 .../apache/flink/runtime/jobgraph/JobID.java    |  7 ++---
 .../flink/runtime/jobgraph/JobVertexID.java     |  4 ++-
 .../jobmanager/scheduler/ResourceId.java        | 29 ++++++++++--------
 .../managementgraph/ManagementEdgeID.java       | 20 ++++--------
 .../managementgraph/ManagementGateID.java       |  6 +---
 .../ManagementGroupVertexID.java                | 17 ++---------
 .../managementgraph/ManagementVertexID.java     | 13 +++++---
 .../apache/flink/runtime/AbstractIDTest.java    | 19 ++----------
 13 files changed, 75 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
index 458907c..c187961 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/AbstractID.java
@@ -32,7 +32,9 @@ import io.netty.buffer.ByteBuf;
 /**
  * A statistically unique identification number.
  */
-public class AbstractID implements IOReadableWritable, Comparable<AbstractID> {
+public class AbstractID implements IOReadableWritable, Comparable<AbstractID>, java.io.Serializable {
+
+	private static final long serialVersionUID = 1L;
 
 	/** The size of a long in bytes */
 	private static final int SIZE_OF_LONG = 8;
@@ -129,16 +131,6 @@ public class AbstractID implements IOReadableWritable, Comparable<AbstractID> {
 			ba[offset + SIZE_OF_LONG - 1 - i] = (byte) ((l & (0xffL << shift)) >>> shift);
 		}
 	}
-
-	/**
-	 * Sets an ID from another ID by copying its internal byte representation.
-	 *
-	 * @param src source ID
-	 */
-	public void setID(AbstractID src) {
-		this.lowerPart = src.lowerPart;
-		this.upperPart = src.upperPart;
-	}
 	
 	// --------------------------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java
index 7088ec1..87e2120 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertexID.java
@@ -28,6 +28,24 @@ import org.apache.flink.runtime.managementgraph.ManagementVertexID;
  */
 public class ExecutionVertexID extends AbstractID {
 
+	private static final long serialVersionUID = 1L;
+	
+	/**
+	 * Creates a new random execution vertex id.
+	 */
+	public ExecutionVertexID() {
+		super();
+	}
+	
+	/**
+	 * Creates a new execution vertex id, equal to the given id.
+	 * 
+	 * @param from The id to copy.
+	 */
+	public ExecutionVertexID(AbstractID from) {
+		super(from);
+	}
+	
 	/**
 	 * Converts the execution vertex ID into a
 	 * management vertex ID. The new management vertex ID
@@ -38,11 +56,7 @@ public class ExecutionVertexID extends AbstractID {
 	 * @return the new management vertex ID
 	 */
 	public ManagementVertexID toManagementVertexID() {
-
-		final ManagementVertexID newID = new ManagementVertexID();
-		newID.setID(this);
-
-		return newID;
+		return new ManagementVertexID(this);
 	}
 
 	/**
@@ -54,11 +68,7 @@ public class ExecutionVertexID extends AbstractID {
 	 *        the management vertex ID to be converted
 	 * @return the resulting execution vertex ID
 	 */
-	public static ExecutionVertexID fromManagementVertexID(final ManagementVertexID vertexID) {
-
-		final ExecutionVertexID newID = new ExecutionVertexID();
-		newID.setID(vertexID);
-
-		return newID;
+	public static ExecutionVertexID fromManagementVertexID(ManagementVertexID vertexID) {
+		return new ExecutionVertexID(vertexID);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
index 9c922af..cc0d819 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/instance/InstanceID.java
@@ -23,4 +23,6 @@ import org.apache.flink.runtime.AbstractID;
 /**
  * Class for statistically unique instance IDs.
  */
-public class InstanceID extends AbstractID {}
+public class InstanceID extends AbstractID {
+	private static final long serialVersionUID = 1L;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/channels/ChannelID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/channels/ChannelID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/channels/ChannelID.java
index 9c9575d..d1c09d2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/channels/ChannelID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/channels/ChannelID.java
@@ -25,6 +25,8 @@ import org.apache.flink.runtime.AbstractID;
 
 public class ChannelID extends AbstractID {
 
+	private static final long serialVersionUID = 1L;
+	
 	public ChannelID() {
 		super();
 	}
@@ -33,8 +35,8 @@ public class ChannelID extends AbstractID {
 		super(lowerPart, upperPart);
 	}
 
-	public ChannelID(byte[] bytes) {
-		super(bytes);
+	public ChannelID(AbstractID id) {
+		super(id);
 	}
 
 	public static ChannelID fromByteBuffer(ByteBuffer buf) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/GateID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/GateID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/GateID.java
index e106ad9..954f527 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/GateID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/gates/GateID.java
@@ -16,15 +16,13 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.network.gates;
 
 import org.apache.flink.runtime.AbstractID;
 
 /**
  * A class for statistically unique gate IDs.
- * 
  */
 public final class GateID extends AbstractID {
-
+	private static final long serialVersionUID = 1L;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java
index 27eebc1..be2fcc1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobID.java
@@ -27,6 +27,8 @@ import java.nio.ByteBuffer;
 
 public final class JobID extends AbstractID {
 
+	private static final long serialVersionUID = 1L;
+	
 	public JobID() {
 		super();
 	}
@@ -40,10 +42,7 @@ public final class JobID extends AbstractID {
 	}
 
 	public static JobID generate() {
-		long lowerPart = AbstractID.generateRandomLong();
-		long upperPart = AbstractID.generateRandomLong();
-
-		return new JobID(lowerPart, upperPart);
+		return new JobID();
 	}
 
 	public static JobID fromByteArray(byte[] bytes) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
index 6a56c19..cb63b06 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/JobVertexID.java
@@ -23,4 +23,6 @@ import org.apache.flink.runtime.AbstractID;
 /**
  * A class for statistically unique job vertex IDs.
  */
-public class JobVertexID extends AbstractID {}
+public class JobVertexID extends AbstractID {
+	private static final long serialVersionUID = 1L;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java
index 58833f4..ab98b68 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/ResourceId.java
@@ -1,20 +1,25 @@
-/***********************************************************************************************************************
- *
- * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
- *
- * Licensed 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
+/**
+ * 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.
- *
- **********************************************************************************************************************/
+ * 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.jobmanager.scheduler;
 
 import org.apache.flink.runtime.AbstractID;
 
-public class ResourceId extends AbstractID {}
+public class ResourceId extends AbstractID {
+	private static final long serialVersionUID = 1L;
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java
index 955f064..16ba974 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementEdgeID.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.managementgraph;
 
 import org.apache.flink.runtime.AbstractID;
@@ -24,17 +23,15 @@ import org.apache.flink.runtime.io.network.channels.ChannelID;
 
 /**
  * A management edge ID uniquely identifies a {@link ManagementEdge}.
- * <p>
- * This class is not thread-safe.
- * 
  */
 public class ManagementEdgeID extends AbstractID {
-
+	
+	private static final long serialVersionUID = 1L;
+	
 	/**
 	 * Initializes ManagementEdgeID.
 	 */
-	ManagementEdgeID() {
-	}
+	public ManagementEdgeID() {}
 
 	/**
 	 * A ManagementEdgeID is derived from the #{@link ChannelID} of the corresponding
@@ -44,8 +41,7 @@ public class ManagementEdgeID extends AbstractID {
 	 *        ID of the corresponding output channel
 	 */
 	public ManagementEdgeID(ChannelID source) {
-		super();
-		this.setID(source);
+		super(source);
 	}
 
 	/**
@@ -54,10 +50,6 @@ public class ManagementEdgeID extends AbstractID {
 	 * @return the corresponding channelID.
 	 */
 	public ChannelID toChannelID() {
-
-		final ChannelID channelID = new ChannelID();
-		channelID.setID(this);
-
-		return channelID;
+		return new ChannelID(this);
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java
index 08a46d1..3e890d4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGateID.java
@@ -16,17 +16,13 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.managementgraph;
 
 import org.apache.flink.runtime.AbstractID;
 
 /**
  * A management gate ID uniquely identifies a {@link ManagementGate}.
- * <p>
- * This class is not thread-safe.
- * 
  */
 public final class ManagementGateID extends AbstractID {
-
+	private static final long serialVersionUID = 1L;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java
index 8038080..66465dd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementGroupVertexID.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.managementgraph;
 
 import javax.xml.bind.DatatypeConverter;
@@ -25,16 +24,13 @@ import org.apache.flink.runtime.AbstractID;
 
 /**
  * A management group vertex ID uniquely identifies a {@link ManagementGroupVertex}.
- * <p>
- * This class is not thread-safe.
- * 
  */
 public final class ManagementGroupVertexID extends AbstractID {
 	
-	
+	private static final long serialVersionUID = 1L;
+
 	/**
 	 * Constructs a new ManagementGroupVertexID
-	 * 
 	 */
 	public ManagementGroupVertexID() {
 		super();
@@ -50,16 +46,7 @@ public final class ManagementGroupVertexID extends AbstractID {
 		super(bytes);
 	}
 	
-	/**
-	 * Constructs a new job ID and initializes it with the given bytes.
-	 * 
-	 * @param bytes
-	 *        the bytes to initialize the new job ID with
-	 * @return the new job ID
-	 */
 	public static ManagementGroupVertexID fromHexString(final String hexString) {
-
 		return new ManagementGroupVertexID(DatatypeConverter.parseHexBinary(hexString));
 	}
-	
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
index d5a7ba0..92955e8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/managementgraph/ManagementVertexID.java
@@ -16,16 +16,21 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.managementgraph;
 
 import org.apache.flink.runtime.AbstractID;
 
 /**
  * A management vertex ID uniquely identifies a {@link ManagementVertex}.
- * <p>
- * This class is not thread-safe.
- * 
  */
 public final class ManagementVertexID extends AbstractID {
+	private static final long serialVersionUID = 1L;
+	
+	public ManagementVertexID() {
+		super();
+	}
+	
+	public ManagementVertexID(AbstractID toCopy) {
+		super(toCopy);
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9035b6d0/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
index e55012b..1e2a1ab 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/AbstractIDTest.java
@@ -24,7 +24,6 @@ import static org.junit.Assert.fail;
 
 import org.junit.Test;
 
-import org.apache.flink.runtime.io.network.channels.ChannelID;
 import org.apache.flink.runtime.testutils.CommonTestUtils;
 
 /**
@@ -32,27 +31,13 @@ import org.apache.flink.runtime.testutils.CommonTestUtils;
  */
 public class AbstractIDTest {
 	/**
-	 * Tests the setID method of an abstract ID.
-	 */
-	@Test
-	public void testSetID() {
-
-		final ChannelID id1 = new ChannelID();
-		final ChannelID id2 = new ChannelID();
-		id1.setID(id2);
-
-		assertEquals(id1.hashCode(), id2.hashCode());
-		assertEquals(id1, id2);
-	}
-
-	/**
 	 * Tests the serialization/deserialization of an abstract ID.
 	 */
 	@Test
 	public void testSerialization() {
-		final ChannelID origID = new ChannelID();
+		final AbstractID origID = new AbstractID();
 		try {
-			final ChannelID copyID = (ChannelID) CommonTestUtils.createCopyWritable(origID);
+			final AbstractID copyID = (AbstractID) CommonTestUtils.createCopyWritable(origID);
 
 			assertEquals(origID.hashCode(), copyID.hashCode());
 			assertEquals(origID, copyID);


[55/63] [abbrv] git commit: Make Co-Location constraints resilient aginst out of order scheduling and depply integrate them with slot sharing Fix miscellaneous checkstyle errors/warnings

Posted by se...@apache.org.
Make Co-Location constraints resilient aginst out of order scheduling and depply integrate them with slot sharing
Fix miscellaneous checkstyle errors/warnings


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/cdee8750
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/cdee8750
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/cdee8750

Branch: refs/heads/master
Commit: cdee87501762c092c216adf35fceeea339e0c4c4
Parents: 8e7216a
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 05:22:24 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:01 2014 +0200

----------------------------------------------------------------------
 .../flink/client/CliFrontendListCancelTest.java |   7 +-
 .../plantranslate/NepheleJobGraphGenerator.java |   7 +-
 .../flink/runtime/jobmanager/JobManager.java    |   7 +-
 .../scheduler/CoLocationConstraint.java         |  46 ++-
 .../jobmanager/scheduler/CoLocationGroup.java   |  10 +-
 .../runtime/jobmanager/scheduler/Scheduler.java | 102 ++----
 .../jobmanager/scheduler/SharedSlot.java        |  93 +++---
 .../scheduler/SlotSharingGroupAssignment.java   | 258 ++++++++++-----
 .../runtime/jobmanager/scheduler/SubSlot.java   |  12 +-
 .../protocols/ExtendedManagementProtocol.java   |   4 +-
 .../runtime/jobgraph/JobManagerTestUtils.java   |   4 +-
 .../runtime/jobmanager/JobManagerITCase.java    |  14 +-
 .../ScheduleWithCoLocationHintTest.java         | 322 ++++++++++++++++++-
 .../scheduler/SchedulerTestUtils.java           |   1 +
 .../jobmanager/scheduler/SharedSlotsTest.java   |  32 +-
 .../runtime/testutils/ServerTestUtils.java      |   2 +-
 16 files changed, 655 insertions(+), 266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
index 2f31181..40d10b6 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/CliFrontendListCancelTest.java
@@ -183,7 +183,12 @@ public class CliFrontendListCancelTest {
 		}
 
 		@Override
-		public int getAvailableSlots() {
+		public int getTotalNumberOfRegisteredSlots() {
+			return 1;
+		}
+
+		@Override
+		public int getNumberOfSlotsAvailableToScheduler() throws IOException {
 			return 1;
 		}
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
index a3fef17..a224324 100644
--- a/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
+++ b/flink-compiler/src/main/java/org/apache/flink/compiler/plantranslate/NepheleJobGraphGenerator.java
@@ -493,11 +493,10 @@ public class NepheleJobGraphGenerator implements Visitor<PlanNode> {
 			
 			if (this.currentIteration != null) {
 				AbstractJobVertex head = this.iterations.get(this.currentIteration).getHeadTask();
-				if (head == null) {
-					throw new CompilerException("Found no iteration head task in the postVisit of translating a task inside an iteration");
+				// the head may still be null if we descend into the static parts first
+				if (head != null) {
+					targetVertex.setStrictlyCoLocatedWith(head);
 				}
-				
-				targetVertex.setStrictlyCoLocatedWith(head);
 			}
 			
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
index f79fecb..1c02127 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/JobManager.java
@@ -574,10 +574,15 @@ public class JobManager implements ExtendedManagementProtocol, InputSplitProvide
 	}
 
 	@Override
-	public int getAvailableSlots() {
+	public int getTotalNumberOfRegisteredSlots() {
 		return getInstanceManager().getTotalNumberOfSlots();
 	}
 	
+	@Override
+	public int getNumberOfSlotsAvailableToScheduler() {
+		return scheduler.getNumberOfAvailableSlots();
+	}
+	
 	/**
 	 * Starts the Jetty Infoserver for the Jobmanager
 	 * 

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
index 64f7ffc..89b644b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationConstraint.java
@@ -18,30 +18,52 @@
 
 package org.apache.flink.runtime.jobmanager.scheduler;
 
+import org.apache.flink.runtime.AbstractID;
 import org.apache.flink.runtime.instance.Instance;
 
+import com.google.common.base.Preconditions;
+
 public class CoLocationConstraint {
 	
-	private volatile Instance location;
+	private final CoLocationGroup group;
 	
+	private volatile SharedSlot sharedSlot;
 	
-	public void setLocation(Instance location) {
-		if (location == null) {
-			throw new IllegalArgumentException();
-		}
-		
-		if (this.location == null) {
-			this.location = location;
+	
+	CoLocationConstraint(CoLocationGroup group) {
+		Preconditions.checkNotNull(group);
+		this.group = group;
+	}
+	
+	
+	public SharedSlot getSharedSlot() {
+		return sharedSlot;
+	}
+	
+	public Instance getLocation() {
+		if (sharedSlot != null) {
+			return sharedSlot.getAllocatedSlot().getInstance();
 		} else {
-			throw new IllegalStateException("The constraint has already been assigned a location.");
+			throw new IllegalStateException("Not assigned");
 		}
 	}
 	
-	public Instance getLocation() {
-		return location;
+	public void setSharedSlot(SharedSlot sharedSlot) {
+		if (this.sharedSlot == sharedSlot) {
+			return;
+		}
+		else if (this.sharedSlot == null || this.sharedSlot.isDisposed()) {
+			this.sharedSlot = sharedSlot;
+		} else {
+			throw new IllegalStateException("Overriding shared slot that is still alive.");
+		}
 	}
 	
 	public boolean isUnassigned() {
-		return this.location == null;
+		return this.sharedSlot == null;
+	}
+	
+	public AbstractID getGroupId() {
+		return this.group.getId();
 	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java
index 2398334..84692a9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/CoLocationGroup.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.jobmanager.scheduler;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.flink.runtime.AbstractID;
 import org.apache.flink.runtime.jobgraph.AbstractJobVertex;
 
 import com.google.common.base.Preconditions;
@@ -29,6 +30,9 @@ public class CoLocationGroup implements java.io.Serializable {
 	
 	private static final long serialVersionUID = -2605819490401895297L;
 
+	// we use a job vertex ID, because the co location group acts as a unit inside which exclusive sharing of
+	// slots is used
+	private final AbstractID id = new AbstractID();
 	
 	private final List<AbstractJobVertex> vertices = new ArrayList<AbstractJobVertex>();
 	
@@ -80,8 +84,12 @@ public class CoLocationGroup implements java.io.Serializable {
 		if (num > constraints.size()) {
 			constraints.ensureCapacity(num);
 			for (int i = constraints.size(); i < num; i++) {
-				constraints.add(new CoLocationConstraint());
+				constraints.add(new CoLocationConstraint(this));
 			}
 		}
 	}
+	
+	public AbstractID getId() {
+		return id;
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
index b9f83fc..20e4d17 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/Scheduler.java
@@ -41,7 +41,7 @@ import org.apache.flink.util.ExceptionUtils;
  */
 public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 
-	private static final Logger LOG = LoggerFactory.getLogger(Scheduler.class);
+	static final Logger LOG = LoggerFactory.getLogger(Scheduler.class);
 	
 	
 	private final Object globalLock = new Object();
@@ -150,56 +150,6 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 		final ExecutionVertex vertex = task.getTaskToExecute().getVertex();
 	
 		synchronized (globalLock) {
-			
-//			// 1)  === If the task has a strict co-schedule hint, obey it ===
-//			
-//			CoLocationConstraint locationConstraint = task.getLocationConstraint();
-//			if (locationConstraint != null) {
-//				// location constraints can never be scheduled in a queued fashion
-//				if (queueIfNoResource) {
-//					throw new IllegalArgumentException("A task with a location constraint was scheduled in a queued fashion.");
-//				}
-//				
-//				// since we are inside the global lock scope, we can check, allocate, and assign
-//				// in one atomic action. however, slots may die and be deallocated
-//				
-//				// (a) is the constraint has not yet has a slot, get one
-//				if (locationConstraint.isUnassigned()) {
-//					// try and get a slot
-//					AllocatedSlot newSlot = getFreeSlotForTask(vertex);
-//					if (newSlot == null) {
-//						throw new NoResourceAvailableException();
-//					}
-//					SharedSlot sl = locationConstraint.swapInNewSlot(newSlot);
-//					SubSlot slot = sl.allocateSubSlot(vertex.getJobvertexId());
-//					
-//					updateLocalityCounters(newSlot.getLocality());
-//					return slot;
-//				}
-//				else {
-//					// try to get a subslot. returns null, if the location's slot has been released
-//					// in the meantime
-//					SubSlot slot = locationConstraint.allocateSubSlot(vertex.getJobvertexId());
-//					if (slot == null) {
-//						// get a new slot. at the same instance!!!
-//						Instance location = locationConstraint.getSlot().getAllocatedSlot().getInstance();
-//						AllocatedSlot newSlot;
-//						try {
-//							newSlot = location.allocateSlot(vertex.getJobId());
-//						} catch (InstanceDiedException e) {
-//							throw new NoResourceAvailableException("The instance of the required location died.");
-//						}
-//						if (newSlot == null) {
-//							throw new NoResourceAvailableException();
-//						}
-//						SharedSlot sharedSlot = locationConstraint.swapInNewSlot(newSlot);
-//						slot = sharedSlot.allocateSubSlot(vertex.getJobvertexId());
-//					}
-//					
-//					updateLocalityCounters(Locality.LOCAL);
-//					return slot;
-//				}
-//			}
 		
 			// 1)  === If the task has a slot sharing group, schedule with shared slots ===
 			
@@ -213,18 +163,17 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 				final SlotSharingGroupAssignment assignment = sharingUnit.getTaskAssignment();
 				final CoLocationConstraint constraint = task.getLocationConstraint();
 				
-				AllocatedSlot newSlot = null;
-				
-				// get a slot from the group. obey location constraints, if existing and assigned
-				AllocatedSlot slotFromGroup;
-				if (constraint == null || constraint.isUnassigned()) {
-					slotFromGroup = assignment.getSlotForTask(vertex.getJobvertexId(), vertex);
+				// get a slot from the group, if the group has one for us (and can fulfill the constraint)
+				SubSlot slotFromGroup;
+				if (constraint == null) {
+					slotFromGroup = assignment.getSlotForTask(vertex);
 				}
 				else {
-					// this returns null, if the constraint cannot be fulfilled
-					slotFromGroup = assignment.getSlotForTask(vertex.getJobvertexId(), constraint);
+					slotFromGroup = assignment.getSlotForTask(vertex, constraint);
 				}
 				
+				AllocatedSlot newSlot = null;
+				
 				// the following needs to make sure any allocated slot is released in case of an error
 				try {
 					
@@ -232,12 +181,6 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 					if (slotFromGroup != null) {
 						// local (or unconstrained in the current group)
 						if (slotFromGroup.getLocality() != Locality.NON_LOCAL) {
-							
-							// attach to the locality constraint
-							if (constraint != null && constraint.isUnassigned()) {
-								constraint.setLocation(slotFromGroup.getInstance());
-							}
-							
 							updateLocalityCounters(slotFromGroup.getLocality());
 							return slotFromGroup;
 						}
@@ -249,13 +192,19 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 					// get a new slot, since we could not place it into the group, or we could not place it locally
 					newSlot = getFreeSlotForTask(vertex, locations);
 					
-					AllocatedSlot toUse;
+					SubSlot toUse;
 					
 					if (newSlot == null) {
 						if (slotFromGroup == null) {
 							// both null
-							throw new NoResourceAvailableException();
+							if (constraint == null || constraint.isUnassigned()) {
+								throw new NoResourceAvailableException();
+							} else {
+								throw new NoResourceAvailableException("Could not allocate a slot on instance " + 
+											constraint.getLocation() + ", as required by the co-location constraint.");
+							}
 						} else {
+							// got a non-local from the group, and no new one
 							toUse = slotFromGroup;
 						}
 					}
@@ -265,17 +214,28 @@ public class Scheduler implements InstanceListener, SlotAvailablilityListener {
 							slotFromGroup.releaseSlot();
 						}
 						
-						toUse = sharingUnit.getTaskAssignment().addSlotWithTask(newSlot, task.getJobVertexId());
+						if (constraint == null) {
+							toUse = assignment.addNewSlotWithTask(newSlot, vertex);
+						} else {
+							toUse = assignment.addNewSlotWithTask(newSlot, vertex, constraint);
+						}
 					}
 					else {
-						// both are available and potentially usable
+						// both are available and usable. neither is local
 						newSlot.releaseSlot();
 						toUse = slotFromGroup;
 					}
 					
 					// assign to the co-location hint, if we have one and it is unassigned
-					if (constraint != null && constraint.isUnassigned()) {
-						constraint.setLocation(toUse.getInstance());
+					// if it was assigned before and the new one is not local, it is a fail
+					if (constraint != null) {
+						if (constraint.isUnassigned() || toUse.getLocality() == Locality.LOCAL) {
+							constraint.setSharedSlot(toUse.getSharedSlot());
+						} else {
+							// the fail
+							throw new NoResourceAvailableException("Could not allocate a slot on instance " + 
+									constraint.getLocation() + ", as required by the co-location constraint.");
+						}
 					}
 					
 					updateLocalityCounters(toUse.getLocality());

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
index 0f3687a..5d87e07 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlot.java
@@ -24,7 +24,13 @@ import java.util.Set;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 
-public class SharedSlot {
+/**
+ * 
+ * NOTE: This class does no synchronization by itself and its mutating
+ *       methods may only be called from within the synchronization scope of
+ *       it associated SlotSharingGroupAssignment.
+ */
+class SharedSlot {
 
 	private final AllocatedSlot allocatedSlot;
 	
@@ -48,77 +54,58 @@ public class SharedSlot {
 		this.subSlots = new HashSet<SubSlot>();
 	}
 	
-	public SharedSlot(AllocatedSlot allocatedSlot) {
-		if (allocatedSlot == null) {
-			throw new NullPointerException();
-		}
-		
-		this.allocatedSlot = allocatedSlot;
-		this.assignmentGroup = null;;
-		this.subSlots = new HashSet<SubSlot>();
-	}
-	
 	// --------------------------------------------------------------------------------------------
 	
-	public AllocatedSlot getAllocatedSlot() {
+	AllocatedSlot getAllocatedSlot() {
 		return this.allocatedSlot;
 	}
 	
-	public boolean isDisposed() {
+	boolean isDisposed() {
 		return disposed;
 	}
 	
-	public int getNumberOfAllocatedSubSlots() {
-		synchronized (this.subSlots) {
-			return this.subSlots.size();
-		}
+	int getNumberOfAllocatedSubSlots() {
+		return this.subSlots.size();
 	}
 	
-	public SubSlot allocateSubSlot(JobVertexID jid) {
-		synchronized (this.subSlots) {
-			if (isDisposed()) {
-				return null;
-			} else {
-				SubSlot ss = new SubSlot(this, subSlotNumber++, jid);
-				this.subSlots.add(ss);
-				return ss;
-			}
+	SubSlot allocateSubSlot(JobVertexID jid) {
+		if (disposed) {
+			return null;
+		} else {
+			SubSlot ss = new SubSlot(this, subSlotNumber++, jid);
+			this.subSlots.add(ss);
+			return ss;
 		}
 	}
 	
-	public void rease() {
-		synchronized (this.subSlots) {
-			disposed = true;
-			for (SubSlot ss : subSlots) {
-				ss.releaseSlot();
-			}
+	void returnAllocatedSlot(SubSlot slot) {
+		if (!slot.isReleased()) {
+			throw new IllegalArgumentException("SubSlot is not released.");
 		}
 		
-		allocatedSlot.releaseSlot();
+		this.assignmentGroup.releaseSubSlot(slot, this);
 	}
 	
-	void returnAllocatedSlot(SubSlot slot) {
-		boolean release;
-		
-		synchronized (this.subSlots) {
-			if (!this.subSlots.remove(slot)) {
-				throw new IllegalArgumentException("Wrong shared slot for subslot.");
-			}
-			
-			if (assignmentGroup != null) {
-				release = assignmentGroup.sharedSlotAvailableForJid(this, slot.getJobVertexId(), this.subSlots.isEmpty());
-			} else {
-				release = subSlots.isEmpty();
-			}
-			
-			if (release) {
-				disposed = true;
-			}
+	int releaseSlot(SubSlot slot) {
+		if (!this.subSlots.remove(slot)) {
+			throw new IllegalArgumentException("Wrong shared slot for subslot.");
 		}
-		
-		// do this call outside the lock, because releasing the allocated slot may go into further scheduler calls
-		if (release) {
+		return subSlots.size();
+	}
+	
+	void dispose() {
+		if (subSlots.isEmpty()) {
+			disposed = true;
 			this.allocatedSlot.releaseSlot();
+		} else {
+			throw new IllegalStateException("Cannot dispose while subslots are still alive.");
 		}
 	}
+	
+	// --------------------------------------------------------------------------------------------
+	
+	@Override
+	public String toString() {
+		return "Shared " + allocatedSlot.toString();
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
index 474fdbe..68d3888 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SlotSharingGroupAssignment.java
@@ -28,34 +28,53 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.flink.runtime.AbstractID;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
 import org.apache.flink.runtime.instance.AllocatedSlot;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.slf4j.Logger;
+
 
 public class SlotSharingGroupAssignment {
 	
+	private static final Logger LOG = Scheduler.LOG;
+	
+	private final Object lock = new Object();
+	
 	/** All slots currently allocated to this sharing group */
 	private final Set<SharedSlot> allSlots = new LinkedHashSet<SharedSlot>();
 	
 	/** The slots available per vertex type (jid), keyed by instance, to make them locatable */
-	private final Map<JobVertexID, Map<Instance, List<SharedSlot>>> availableSlotsPerJid = new LinkedHashMap<JobVertexID, Map<Instance, List<SharedSlot>>>();
+	private final Map<AbstractID, Map<Instance, List<SharedSlot>>> availableSlotsPerJid = new LinkedHashMap<AbstractID, Map<Instance, List<SharedSlot>>>();
 	
 	
 	// --------------------------------------------------------------------------------------------
 	
 	
-	public SubSlot addSlotWithTask(AllocatedSlot slot, JobVertexID jid) {
+	public SubSlot addNewSlotWithTask(AllocatedSlot slot, ExecutionVertex vertex) {
+		JobVertexID id = vertex.getJobvertexId();
+		return addNewSlotWithTask(slot, id, id);
+	}
+	
+	public SubSlot addNewSlotWithTask(AllocatedSlot slot, ExecutionVertex vertex, CoLocationConstraint constraint) {
+		AbstractID groupId = constraint.getGroupId();
+		return addNewSlotWithTask(slot, groupId, null);
+	}
+	
+	private SubSlot addNewSlotWithTask(AllocatedSlot slot, AbstractID groupId, JobVertexID vertexId) {
 		
 		final SharedSlot sharedSlot = new SharedSlot(slot, this);
 		final Instance location = slot.getInstance();
 		
-		synchronized (allSlots) {
+		synchronized (lock) {
 			// add to the total bookkeeping
 			allSlots.add(sharedSlot);
 			
 			// allocate us a sub slot to return
-			SubSlot subslot = sharedSlot.allocateSubSlot(jid);
+			SubSlot subslot = sharedSlot.allocateSubSlot(vertexId);
 			
 			// preserve the locality information
 			subslot.setLocality(slot.getLocality());
@@ -63,10 +82,9 @@ public class SlotSharingGroupAssignment {
 			boolean entryForNewJidExists = false;
 			
 			// let the other vertex types know about this one as well
-			
-			for (Map.Entry<JobVertexID, Map<Instance, List<SharedSlot>>> entry : availableSlotsPerJid.entrySet()) {
+			for (Map.Entry<AbstractID, Map<Instance, List<SharedSlot>>> entry : availableSlotsPerJid.entrySet()) {
 				
-				if (entry.getKey().equals(jid)) {
+				if (entry.getKey().equals(groupId)) {
 					entryForNewJidExists = true;
 					continue;
 				}
@@ -75,9 +93,9 @@ public class SlotSharingGroupAssignment {
 				putIntoMultiMap(available, location, sharedSlot);
 			}
 			
-			// make sure an empty entry exists for this jid, if no other entry exists
+			// make sure an empty entry exists for this group, if no other entry exists
 			if (!entryForNewJidExists) {
-				availableSlotsPerJid.put(jid, new LinkedHashMap<Instance, List<SharedSlot>>());
+				availableSlotsPerJid.put(groupId, new LinkedHashMap<Instance, List<SharedSlot>>());
 			}
 			
 			return subslot;
@@ -90,105 +108,97 @@ public class SlotSharingGroupAssignment {
 	 * slots if no local slot is available. The method returns null, when no slot is available for the
 	 * given JobVertexID at all.
 	 * 
-	 * @param jid
 	 * @param vertex
 	 * 
 	 * @return A task vertex for a task with the given JobVertexID, or null, if none is available.
 	 */
-	public AllocatedSlot getSlotForTask(JobVertexID jid, ExecutionVertex vertex) {
-		synchronized (allSlots) {
-			return getSlotForTaskInternal(jid, vertex.getPreferredLocations(), false);
-		}
-	}
-	
-	
-	public AllocatedSlot getSlotForTask(JobVertexID jid, CoLocationConstraint constraint) {
-		if (constraint.isUnassigned()) {
-			throw new IllegalArgumentException("CoLocationConstraint is unassigned");
+	public SubSlot getSlotForTask(ExecutionVertex vertex) {
+		synchronized (lock) {
+			Pair<SharedSlot, Locality> p = getSlotForTaskInternal(vertex.getJobvertexId(), vertex, vertex.getPreferredLocations(), false);
+			
+			if (p != null) {
+				SharedSlot ss = p.getLeft();
+				SubSlot slot = ss.allocateSubSlot(vertex.getJobvertexId());
+				slot.setLocality(p.getRight());
+				return slot;
+			}
+			else {
+				return null;
+			}
 		}
 		
-		synchronized (allSlots) {
-			return getSlotForTaskInternal(jid, Collections.singleton(constraint.getLocation()), true);
-		}
 	}
 	
-	
-	public boolean sharedSlotAvailableForJid(SharedSlot slot, JobVertexID jid, boolean lastSubSlot) {
-		if (slot == null || jid == null) {
-			throw new NullPointerException();
-		}
+	public SubSlot getSlotForTask(ExecutionVertex vertex, CoLocationConstraint constraint) {
 		
-		synchronized (allSlots) {
-			if (!allSlots.contains(slot)) {
-				throw new IllegalArgumentException("Slot was not associated with this SlotSharingGroup before.");
-			}
+		synchronized (lock) {
+			SharedSlot shared = constraint.getSharedSlot();
 			
-			if (lastSubSlot) {
-				// this was the last sub slot. unless there is something pending for this jid
-				// remove this from the availability list of all jids and 
-				// return that this one is good to release
-				allSlots.remove(slot);
-				
-				Instance location = slot.getAllocatedSlot().getInstance();
-				
-				for (Map.Entry<JobVertexID, Map<Instance, List<SharedSlot>>> mapEntry : availableSlotsPerJid.entrySet()) {
-					if (mapEntry.getKey().equals(jid)) {
-						continue;
-					}
+			if (shared != null && !shared.isDisposed()) {
+				// initialized and set
+				SubSlot subslot = shared.allocateSubSlot(null);
+				subslot.setLocality(Locality.LOCAL);
+				return subslot;
+			}
+			else if (shared == null) {
+				// not initialized, grab a new slot. preferred locations are defined by the vertex
+				// we only associate the slot with the constraint, if it was a local match
+				Pair<SharedSlot, Locality> p = getSlotForTaskInternal(constraint.getGroupId(), vertex, vertex.getPreferredLocations(), false);
+				if (p == null) {
+					return null;
+				} else {
+					shared = p.getLeft();
+					Locality l = p.getRight();
 					
-					Map<Instance, List<SharedSlot>> map = mapEntry.getValue();
-					List<SharedSlot> list = map.get(location);
-					if (list == null || !list.remove(slot)) {
-						throw new IllegalStateException("SharedSlot was not available to another vertex type that it was not allocated for before.");
-					}
-					if (list.isEmpty()) {
-						map.remove(location);
+					SubSlot sub = shared.allocateSubSlot(null);
+					sub.setLocality(l);
+					
+					if (l != Locality.NON_LOCAL) {
+						constraint.setSharedSlot(shared);
 					}
+					return sub;
 				}
-				
-				return true;
 			}
-			
-			Map<Instance, List<SharedSlot>> slotsForJid = availableSlotsPerJid.get(jid);
-			
-			// sanity check
-			if (slotsForJid == null) {
-				throw new IllegalStateException("Trying to return a slot for jid " + jid + 
-						" when available slots indicated that all slots were available.");
+			else {
+				// disposed. get a new slot on the same instance
+				Instance location = shared.getAllocatedSlot().getInstance();
+				Pair<SharedSlot, Locality> p = getSlotForTaskInternal(constraint.getGroupId(), vertex, Collections.singleton(location), true);
+				if (p == null) {
+					return null;
+				} else {
+					shared = p.getLeft();
+					constraint.setSharedSlot(shared);
+					SubSlot subslot = shared.allocateSubSlot(null);
+					subslot.setLocality(Locality.LOCAL);
+					return subslot;
+				}
 			}
-			
-			putIntoMultiMap(slotsForJid, slot.getAllocatedSlot().getInstance(), slot);
-			
-			// do not release, we are still depending on this shared slot
-			return false;
 		}
 	}
 	
-	
 	/**
 	 * NOTE: This method is not synchronized by itself, needs to be synchronized externally.
 	 * 
-	 * @param jid
 	 * @return An allocated sub slot, or {@code null}, if no slot is available.
 	 */
-	private AllocatedSlot getSlotForTaskInternal(JobVertexID jid, Iterable<Instance> preferredLocations, boolean localOnly) {
+	private Pair<SharedSlot, Locality> getSlotForTaskInternal(AbstractID groupId, ExecutionVertex vertex, Iterable<Instance> preferredLocations, boolean localOnly) {
 		if (allSlots.isEmpty()) {
 			return null;
 		}
 		
-		Map<Instance, List<SharedSlot>> slotsForJid = availableSlotsPerJid.get(jid);
+		Map<Instance, List<SharedSlot>> slotsForGroup = availableSlotsPerJid.get(groupId);
 		
-		// get the available slots for the vertex type (jid)
-		if (slotsForJid == null) {
-			// no task is yet scheduled for that jid, so all slots are available
-			slotsForJid = new LinkedHashMap<Instance, List<SharedSlot>>();
-			availableSlotsPerJid.put(jid, slotsForJid);
+		// get the available slots for the group
+		if (slotsForGroup == null) {
+			// no task is yet scheduled for that group, so all slots are available
+			slotsForGroup = new LinkedHashMap<Instance, List<SharedSlot>>();
+			availableSlotsPerJid.put(groupId, slotsForGroup);
 			
 			for (SharedSlot availableSlot : allSlots) {
-				putIntoMultiMap(slotsForJid, availableSlot.getAllocatedSlot().getInstance(), availableSlot);
+				putIntoMultiMap(slotsForGroup, availableSlot.getAllocatedSlot().getInstance(), availableSlot);
 			}
 		}
-		else if (slotsForJid.isEmpty()) {
+		else if (slotsForGroup.isEmpty()) {
 			return null;
 		}
 		
@@ -202,32 +212,102 @@ public class SlotSharingGroupAssignment {
 				// we return early anyways and skip the flag evaluation
 				didNotGetPreferred = true;
 				
-				SharedSlot slot = removeFromMultiMap(slotsForJid, location);
-				if (slot != null) {
-					SubSlot subslot = slot.allocateSubSlot(jid);
-					subslot.setLocality(Locality.LOCAL);
-					return subslot;
+				SharedSlot slot = removeFromMultiMap(slotsForGroup, location);
+				if (slot != null && !slot.isDisposed()) {
+					if (LOG.isDebugEnabled()) {
+						LOG.debug("Local assignment in shared group : " + vertex + " --> " + slot);
+					}
+					
+					return new ImmutablePair<SharedSlot, Locality>(slot, Locality.LOCAL);
 				}
 			}
 		}
 		
 		// if we want only local assignments, exit now with a "not found" result
 		if (didNotGetPreferred && localOnly) {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("No local assignment in shared possible for " + vertex);
+			}
 			return null;
 		}
 		
 		// schedule the task to any available location
-		SharedSlot slot = pollFromMultiMap(slotsForJid);
-		if (slot != null) {
-			SubSlot subslot = slot.allocateSubSlot(jid);
-			subslot.setLocality(didNotGetPreferred ? Locality.NON_LOCAL : Locality.UNCONSTRAINED);
-			return subslot;
+		SharedSlot slot = pollFromMultiMap(slotsForGroup);
+		if (slot != null && !slot.isDisposed()) {
+			if (LOG.isDebugEnabled()) {
+				LOG.debug((didNotGetPreferred ? "Non-local" : "Unconstrained") + " assignment in shared group : " + vertex + " --> " + slot);
+			}
+			
+			return new ImmutablePair<SharedSlot, Locality>(slot, didNotGetPreferred ? Locality.NON_LOCAL : Locality.UNCONSTRAINED);
 		}
 		else {
 			return null;
 		}
 	}
 	
+	
+	void releaseSubSlot(SubSlot subslot, SharedSlot sharedSlot) {
+		
+		AbstractID groupId = subslot.getGroupId();
+		
+		synchronized (lock) {
+
+			if (!allSlots.contains(sharedSlot)) {
+				throw new IllegalArgumentException("Slot was not associated with this SlotSharingGroup before.");
+			}
+			
+			int slotsRemaining = sharedSlot.releaseSlot(subslot);
+			
+			if (slotsRemaining == 0) {
+				// this was the last sub slot. remove this from the availability list 
+				// and trigger disposal
+				try {
+					allSlots.remove(sharedSlot);
+					
+					Instance location = sharedSlot.getAllocatedSlot().getInstance();
+
+					if (groupId != null) {
+						for (Map.Entry<AbstractID, Map<Instance, List<SharedSlot>>> mapEntry : availableSlotsPerJid.entrySet()) {
+							AbstractID id = mapEntry.getKey();
+							
+							// hack: we identify co location hint entries by the fact that they are keyed
+							//       by an abstract id, rather than a job vertex id
+							if (id.getClass() == AbstractID.class || id.equals(groupId)) {
+								continue;
+							}
+							
+							Map<Instance, List<SharedSlot>> map = mapEntry.getValue();
+							List<SharedSlot> list = map.get(location);
+							if (list == null || !list.remove(sharedSlot)) {
+								throw new IllegalStateException("Bug: SharedSlot was not available to another vertex type that it was not allocated for before.");
+							}
+							if (list.isEmpty()) {
+								map.remove(location);
+							}
+						}
+					}
+				} finally {
+					sharedSlot.dispose();
+				}
+			}
+			else if (groupId != null) {
+				// make the shared slot available to tasks within the group it available to
+				Map<Instance, List<SharedSlot>> slotsForJid = availableSlotsPerJid.get(groupId);
+				
+				// sanity check
+				if (slotsForJid == null) {
+					throw new IllegalStateException("Trying to return a slot for group " + groupId + 
+							" when available slots indicated that all slots were available.");
+				}
+				
+				putIntoMultiMap(slotsForJid, sharedSlot.getAllocatedSlot().getInstance(), sharedSlot);
+			}
+		}
+	}
+	
+	
+	
+	
 	// --------------------------------------------------------------------------------------------
 	//  State
 	// --------------------------------------------------------------------------------------------
@@ -237,7 +317,7 @@ public class SlotSharingGroupAssignment {
 	}
 	
 	public int getNumberOfAvailableSlotsForJid(JobVertexID jid) {
-		synchronized (allSlots) {
+		synchronized (lock) {
 			Map<Instance, List<SharedSlot>> available = availableSlotsPerJid.get(jid);
 			
 			if (available != null) {
@@ -255,6 +335,10 @@ public class SlotSharingGroupAssignment {
 			}
 		}
 	}
+
+	// --------------------------------------------------------------------------------------------
+	
+	
 	
 	// --------------------------------------------------------------------------------------------
 	//  Utilities

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
index ca2fb5e..7ff2990 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/scheduler/SubSlot.java
@@ -18,25 +18,25 @@
 
 package org.apache.flink.runtime.jobmanager.scheduler;
 
+import org.apache.flink.runtime.AbstractID;
 import org.apache.flink.runtime.instance.AllocatedSlot;
-import org.apache.flink.runtime.jobgraph.JobVertexID;
 
 public class SubSlot extends AllocatedSlot {
 
 	private final SharedSlot sharedSlot;
 	
-	private final JobVertexID jid;
+	private final AbstractID groupId;
 	
 	private final int subSlotNumber;
 	
 	
-	public SubSlot(SharedSlot sharedSlot, int subSlotNumber, JobVertexID jid) {
+	public SubSlot(SharedSlot sharedSlot, int subSlotNumber, AbstractID groupId) {
 		super(sharedSlot.getAllocatedSlot().getJobID(),
 				sharedSlot.getAllocatedSlot().getInstance(),
 				sharedSlot.getAllocatedSlot().getSlotNumber());
 		
 		this.sharedSlot = sharedSlot;
-		this.jid = jid;
+		this.groupId = groupId;
 		this.subSlotNumber = subSlotNumber;
 	}
 	
@@ -59,8 +59,8 @@ public class SubSlot extends AllocatedSlot {
 		return this.sharedSlot;
 	}
 	
-	public JobVertexID getJobVertexId() {
-		return jid;
+	public AbstractID getGroupId() {
+		return groupId;
 	}
 	
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
index 9e3e22e..909a595 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/protocols/ExtendedManagementProtocol.java
@@ -59,5 +59,7 @@ public interface ExtendedManagementProtocol extends JobManagementProtocol {
 	 * @return number of available slots
 	 * @throws IOException
 	 */
-	int getAvailableSlots() throws IOException;
+	int getTotalNumberOfRegisteredSlots() throws IOException;
+	
+	int getNumberOfSlotsAvailableToScheduler() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java
index 14a73e1..1d5471e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobManagerTestUtils.java
@@ -46,11 +46,11 @@ public class JobManagerTestUtils {
 		// max time is 5 seconds
 		long deadline = System.currentTimeMillis() + 5000;
 		
-		while (jm.getAvailableSlots() < numSlots && System.currentTimeMillis() < deadline) {
+		while (jm.getNumberOfSlotsAvailableToScheduler() < numSlots && System.currentTimeMillis() < deadline) {
 			Thread.sleep(10);
 		}
 		
-		assertEquals(numSlots, jm.getAvailableSlots());
+		assertEquals(numSlots, jm.getNumberOfSlotsAvailableToScheduler());
 		
 		return jm;
 	}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
index 70b3ad9..b07cc87 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/JobManagerITCase.java
@@ -68,7 +68,7 @@ public class JobManagerITCase {
 			
 			try {
 				
-				assertEquals(1, jm.getAvailableSlots());
+				assertEquals(1, jm.getTotalNumberOfRegisteredSlots());
 				
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -141,7 +141,7 @@ public class JobManagerITCase {
 			
 			try {
 				
-				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				assertEquals(NUM_TASKS, jm.getTotalNumberOfRegisteredSlots());
 				
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -535,7 +535,7 @@ public class JobManagerITCase {
 					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
 			
 			try {
-				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				assertEquals(NUM_TASKS, jm.getTotalNumberOfRegisteredSlots());
 				
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -599,7 +599,7 @@ public class JobManagerITCase {
 					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
 			
 			try {
-				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				assertEquals(NUM_TASKS, jm.getTotalNumberOfRegisteredSlots());
 				
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -727,7 +727,7 @@ public class JobManagerITCase {
 					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
 			
 			try {
-				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				assertEquals(NUM_TASKS, jm.getTotalNumberOfRegisteredSlots());
 				
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);
@@ -788,13 +788,13 @@ public class JobManagerITCase {
 			
 			final JobGraph jobGraph = new JobGraph("Pointwise Job", sender, receiver);
 			
-			final JobManager jm = startJobManager(NUM_TASKS);
+			final JobManager jm = startJobManager(2*NUM_TASKS);
 			
 			final GlobalBufferPool bp = ((LocalInstanceManager) jm.getInstanceManager())
 					.getTaskManagers()[0].getChannelManager().getGlobalBufferPool();
 			
 			try {
-				assertEquals(NUM_TASKS, jm.getAvailableSlots());
+				assertEquals(2*NUM_TASKS, jm.getNumberOfSlotsAvailableToScheduler());
 				
 				// we need to register the job at the library cache manager (with no libraries)
 				LibraryCacheManager.register(jobGraph.getJobID(), new String[0]);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
index 0efe10b..7ea4fd5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/ScheduleWithCoLocationHintTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.jobmanager.scheduler;
 
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getRandomInstance;
 import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertex;
+import static org.apache.flink.runtime.jobmanager.scheduler.SchedulerTestUtils.getTestVertexWithLocation;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -48,12 +49,13 @@ public class ScheduleWithCoLocationHintTest {
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
 			
-			CoLocationConstraint c1 = new CoLocationConstraint();
-			CoLocationConstraint c2 = new CoLocationConstraint();
-			CoLocationConstraint c3 = new CoLocationConstraint();
-			CoLocationConstraint c4 = new CoLocationConstraint();
-			CoLocationConstraint c5 = new CoLocationConstraint();
-			CoLocationConstraint c6 = new CoLocationConstraint();
+			CoLocationGroup ccg = new CoLocationGroup();
+			CoLocationConstraint c1 = new CoLocationConstraint(ccg);
+			CoLocationConstraint c2 = new CoLocationConstraint(ccg);
+			CoLocationConstraint c3 = new CoLocationConstraint(ccg);
+			CoLocationConstraint c4 = new CoLocationConstraint(ccg);
+			CoLocationConstraint c5 = new CoLocationConstraint(ccg);
+			CoLocationConstraint c6 = new CoLocationConstraint(ccg);
 			
 			// schedule 4 tasks from the first vertex group
 			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 6), sharingGroup, c1));
@@ -174,7 +176,7 @@ public class ScheduleWithCoLocationHintTest {
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
-			CoLocationConstraint c1 = new CoLocationConstraint();
+			CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup());
 			
 			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1));
 			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid2, 0, 1), sharingGroup, c1));
@@ -218,7 +220,7 @@ public class ScheduleWithCoLocationHintTest {
 			assertEquals(2, scheduler.getNumberOfAvailableSlots());
 			
 			SlotSharingGroup sharingGroup = new SlotSharingGroup();
-			CoLocationConstraint c1 = new CoLocationConstraint();
+			CoLocationConstraint c1 = new CoLocationConstraint(new CoLocationGroup());
 			
 			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertex(jid1, 0, 1), sharingGroup, c1));
 			s1.releaseSlot();
@@ -260,10 +262,12 @@ public class ScheduleWithCoLocationHintTest {
 			
 			assertEquals(4, scheduler.getNumberOfAvailableSlots());
 			
-			CoLocationConstraint clc1 = new CoLocationConstraint();
-			CoLocationConstraint clc2 = new CoLocationConstraint();
-			CoLocationConstraint clc3 = new CoLocationConstraint();
-			CoLocationConstraint clc4 = new CoLocationConstraint();
+			CoLocationGroup grp = new CoLocationGroup();
+			CoLocationConstraint clc1 = new CoLocationConstraint(grp);
+			CoLocationConstraint clc2 = new CoLocationConstraint(grp);
+			CoLocationConstraint clc3 = new CoLocationConstraint(grp);
+			CoLocationConstraint clc4 = new CoLocationConstraint(grp);
+			
 			SlotSharingGroup shareGroup = new SlotSharingGroup();
 
 			// first wave
@@ -303,4 +307,298 @@ public class ScheduleWithCoLocationHintTest {
 			fail(e.getMessage());
 		}
 	}
+
+	
+	@Test
+	public void testGetsNonLocalFromSharingGroupFirst() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			JobVertexID jid3 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(1);
+			Instance i2 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i1);
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
+			
+			CoLocationGroup ccg = new CoLocationGroup();
+			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
+			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
+
+			// schedule something into the shared group so that both instances are in the sharing group
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup));
+			
+			// schedule one locally to instance 1
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup, cc1));
+
+			// schedule with co location constraint (yet unassigned) and a preference for
+			// instance 1, but it can only get instance 2
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc2));
+			
+			// schedule something into the assigned co-location constraints and check that they override the
+			// other preferences
+			AllocatedSlot s5 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 0, 2, i2), sharingGroup, cc1));
+			AllocatedSlot s6 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid3, 1, 2, i1), sharingGroup, cc2));
+			
+			// check that each slot got three
+			assertEquals(3, ((SubSlot) s1).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(3, ((SubSlot) s2).getSharedSlot().getNumberOfAllocatedSubSlots());
+			
+			assertEquals(s1.getInstance(), s3.getInstance());
+			assertEquals(s2.getInstance(), s4.getInstance());
+			assertEquals(s1.getInstance(), s5.getInstance());
+			assertEquals(s2.getInstance(), s6.getInstance());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+			
+			assertEquals(5, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
+			
+			// release some slots, be sure that new available ones come up
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s3.releaseSlot();
+			s4.releaseSlot();
+			s5.releaseSlot();
+			s6.releaseSlot();
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSlotReleasedInBetween() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(1);
+			Instance i2 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i1);
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
+			
+			CoLocationGroup ccg = new CoLocationGroup();
+			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
+			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
+
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2));
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
+
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup, cc1));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc2));
+			
+			// still preserves the previous instance mapping)
+			assertEquals(i1, s3.getInstance());
+			assertEquals(i2, s4.getInstance());
+			
+			s3.releaseSlot();
+			s4.releaseSlot();
+
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			assertEquals(4, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSlotReleasedInBetweenAndNoNewLocal() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			JobVertexID jidx = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(1);
+			Instance i2 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i1);
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
+			
+			CoLocationGroup ccg = new CoLocationGroup();
+			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
+			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
+
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2));
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
+
+			AllocatedSlot sa = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 0, 2)));
+			AllocatedSlot sb = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jidx, 1, 2)));
+			
+			try {
+				scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i2), sharingGroup, cc1));
+				fail("should not be able to find a resource");
+			} catch (NoResourceAvailableException e) {
+				// good
+			} catch (Exception e) {
+				fail("wrong exception");
+			}
+			
+			sa.releaseSlot();
+			sb.releaseSlot();
+
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			assertEquals(2, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(2, scheduler.getNumberOfUnconstrainedAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testScheduleOutOfOrder() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(1);
+			Instance i2 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i1);
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
+			
+			CoLocationGroup ccg = new CoLocationGroup();
+			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
+			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
+
+			// schedule something from the second job vertex id before the first is filled,
+			// and give locality preferences that hint at using the same shared slot for both
+			// co location constraints (which we seek to prevent)
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup, cc2));
+
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup, cc1));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i1), sharingGroup, cc2));
+			
+			// check that each slot got three
+			assertEquals(2, ((SubSlot) s1).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s2).getSharedSlot().getNumberOfAllocatedSubSlots());
+			
+			assertEquals(s1.getInstance(), s3.getInstance());
+			assertEquals(s2.getInstance(), s4.getInstance());
+			
+			// check the scheduler's bookkeeping
+			assertEquals(0, scheduler.getNumberOfAvailableSlots());
+			
+			assertEquals(3, scheduler.getNumberOfLocalizedAssignments());
+			assertEquals(1, scheduler.getNumberOfNonLocalizedAssignments());
+			assertEquals(0, scheduler.getNumberOfUnconstrainedAssignments());
+			
+			// release some slots, be sure that new available ones come up
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s3.releaseSlot();
+			s4.releaseSlot();
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void nonColocationFollowsCoLocation() {
+		try {
+			JobVertexID jid1 = new JobVertexID();
+			JobVertexID jid2 = new JobVertexID();
+			
+			Scheduler scheduler = new Scheduler();
+			
+			Instance i1 = getRandomInstance(1);
+			Instance i2 = getRandomInstance(1);
+			
+			scheduler.newInstanceAvailable(i2);
+			scheduler.newInstanceAvailable(i1);
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			SlotSharingGroup sharingGroup = new SlotSharingGroup();
+			
+			CoLocationGroup ccg = new CoLocationGroup();
+			CoLocationConstraint cc1 = new CoLocationConstraint(ccg);
+			CoLocationConstraint cc2 = new CoLocationConstraint(ccg);
+
+			AllocatedSlot s1 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 0, 2, i1), sharingGroup, cc1));
+			AllocatedSlot s2 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid1, 1, 2, i2), sharingGroup, cc2));
+
+			AllocatedSlot s3 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 0, 2, i1), sharingGroup));
+			AllocatedSlot s4 = scheduler.scheduleImmediately(new ScheduledUnit(getTestVertexWithLocation(jid2, 1, 2, i1), sharingGroup));
+			
+			// check that each slot got three
+			assertEquals(2, ((SubSlot) s1).getSharedSlot().getNumberOfAllocatedSubSlots());
+			assertEquals(2, ((SubSlot) s2).getSharedSlot().getNumberOfAllocatedSubSlots());
+			
+			s1.releaseSlot();
+			s2.releaseSlot();
+			s3.releaseSlot();
+			s4.releaseSlot();
+			
+			assertEquals(2, scheduler.getNumberOfAvailableSlots());
+			
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfSlots());
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid1));
+			assertEquals(0, sharingGroup.getTaskAssignment().getNumberOfAvailableSlotsForJid(jid2));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
 }

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
index d2e7598..09a416e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SchedulerTestUtils.java
@@ -101,6 +101,7 @@ public class SchedulerTestUtils {
 		when(vertex.getParallelSubtaskIndex()).thenReturn(taskIndex);
 		when(vertex.getTotalNumberOfParallelSubtasks()).thenReturn(numTasks);
 		when(vertex.toString()).thenReturn("TEST-VERTEX");
+		when(vertex.getSimpleName()).thenReturn("TEST-VERTEX");
 		
 		Execution execution = mock(Execution.class);
 		when(execution.getVertex()).thenReturn(vertex);

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
index de22999..7859b33 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/scheduler/SharedSlotsTest.java
@@ -19,14 +19,13 @@
 package org.apache.flink.runtime.jobmanager.scheduler;
 
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.eq;
-
 import static org.junit.Assert.*;
 
 import org.junit.Test;
-
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 import org.apache.flink.runtime.instance.Instance;
 import org.apache.flink.runtime.jobgraph.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
@@ -37,7 +36,16 @@ public class SharedSlotsTest {
 	public void createAndDoNotRelease() {
 		try {
 			SlotSharingGroupAssignment assignment = mock(SlotSharingGroupAssignment.class);
-			when(assignment.sharedSlotAvailableForJid(any(SharedSlot.class), any(JobVertexID.class), any(boolean.class))).thenReturn(false);
+			doAnswer(new Answer<Void>() {
+				@Override
+				public Void answer(InvocationOnMock invocation) throws Throwable {
+					final SubSlot sub = (SubSlot) invocation.getArguments()[0];
+					final SharedSlot shared = (SharedSlot) invocation.getArguments()[1];
+					shared.releaseSlot(sub);
+					return null;
+				}
+				
+			}).when(assignment).releaseSubSlot(any(SubSlot.class), any(SharedSlot.class));
 			
 			Instance instance = SchedulerTestUtils.getRandomInstance(1);
 			
@@ -77,8 +85,18 @@ public class SharedSlotsTest {
 	public void createAndRelease() {
 		try {
 			SlotSharingGroupAssignment assignment = mock(SlotSharingGroupAssignment.class);
-			when(assignment.sharedSlotAvailableForJid(any(SharedSlot.class), any(JobVertexID.class), eq(false))).thenReturn(false);
-			when(assignment.sharedSlotAvailableForJid(any(SharedSlot.class), any(JobVertexID.class), eq(true))).thenReturn(true);
+			doAnswer(new Answer<Void>() {
+				@Override
+				public Void answer(InvocationOnMock invocation) throws Throwable {
+					final SubSlot sub = (SubSlot) invocation.getArguments()[0];
+					final SharedSlot shared = (SharedSlot) invocation.getArguments()[1];
+					if (shared.releaseSlot(sub) == 0) {
+						shared.dispose();
+					}
+					return null;
+				}
+				
+			}).when(assignment).releaseSubSlot(any(SubSlot.class), any(SharedSlot.class));
 			
 			Instance instance = SchedulerTestUtils.getRandomInstance(1);
 			

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/cdee8750/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ServerTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ServerTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ServerTestUtils.java
index 3b4f293..61ac104 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ServerTestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/ServerTestUtils.java
@@ -184,7 +184,7 @@ public final class ServerTestUtils {
 	public static void waitForJobManagerToBecomeReady(final ExtendedManagementProtocol jobManager) throws IOException,
 			InterruptedException {
 
-		while (jobManager.getAvailableSlots() == 0) {
+		while (jobManager.getTotalNumberOfRegisteredSlots() == 0) {
 			Thread.sleep(100);
 		}
 	}


[26/63] [abbrv] [FLINK-1094] Reworked, improved, and testes split assigners

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
new file mode 100644
index 0000000..5b76d53
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/DefaultSplitAssignerTest.java
@@ -0,0 +1,121 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.splitassigner;
+
+import static org.junit.Assert.*;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.core.io.GenericInputSplit;
+import org.apache.flink.core.io.InputSplit;
+import org.junit.Test;
+
+
+public class DefaultSplitAssignerTest {
+
+	@Test
+	public void testSerialSplitAssignment() {
+		try {
+			final int NUM_SPLITS = 50;
+			
+			Set<InputSplit> splits = new HashSet<InputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new GenericInputSplit(i, NUM_SPLITS));
+			}
+			
+			DefaultInputSplitAssigner ia = new DefaultInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit("")) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit(""));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignment() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			Set<InputSplit> splits = new HashSet<InputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new GenericInputSplit(i, NUM_SPLITS));
+			}
+			
+			final DefaultInputSplitAssigner ia = new DefaultInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					String host = "";
+					GenericInputSplit split;
+					while ((split = (GenericInputSplit) ia.getNextInputSplit(host)) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit(""));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/c32569ae/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
new file mode 100644
index 0000000..ddad0d3
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/splitassigner/LocatableSplitAssignerTest.java
@@ -0,0 +1,379 @@
+/***********************************************************************************************************************
+ *
+ * Copyright (C) 2010-2013 by the Stratosphere project (http://stratosphere.eu)
+ *
+ * Licensed 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.jobmanager.splitassigner;
+
+import static org.junit.Assert.*;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.LocatableInputSplit;
+
+import org.junit.Test;
+
+
+public class LocatableSplitAssignerTest {
+	
+	@Test
+	public void testSerialSplitAssignmentWithNullHost() {
+		try {
+			final int NUM_SPLITS = 50;
+			final String[][] hosts = new String[][] {
+					new String[] { "localhost" },
+					new String[0],
+					null
+			};
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i%3]));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit(null)) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit(""));
+			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
+			assertEquals(0, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialSplitAssignmentAllForSameHost() {
+		try {
+			final int NUM_SPLITS = 50;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, "testhost"));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit("testhost")) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit(""));
+			
+			assertEquals(0, ia.getNumberOfRemoteAssignments());
+			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialSplitAssignmentAllForRemoteHost() {
+		try {
+			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
+			final int NUM_SPLITS = 10 * hosts.length;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i % hosts.length]));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			while ((is = ia.getNextInputSplit("testhost")) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit("anotherHost"));
+			
+			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
+			assertEquals(0, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testSerialSplitAssignmentMixedLocalHost() {
+		try {
+			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
+			final int NUM_SPLITS = 10 * hosts.length;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i % hosts.length]));
+			}
+			
+			// get all available splits
+			LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			InputSplit is = null;
+			int i = 0;
+			while ((is = ia.getNextInputSplit(hosts[i++ % hosts.length])) != null) {
+				assertTrue(splits.remove(is));
+			}
+			
+			// check we had all
+			assertTrue(splits.isEmpty());
+			assertNull(ia.getNextInputSplit("anotherHost"));
+			
+			assertEquals(0, ia.getNumberOfRemoteAssignments());
+			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignmentNullHost() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			final String[][] hosts = new String[][] {
+					new String[] { "localhost" },
+					new String[0],
+					null
+			};
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i%3]));
+			}
+			
+			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					LocatableInputSplit split;
+					while ((split = ia.getNextInputSplit(null)) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit(""));
+			
+			assertEquals(NUM_SPLITS, ia.getNumberOfRemoteAssignments());
+			assertEquals(0, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignmentForSingleHost() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, "testhost"));
+			}
+			
+			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					LocatableInputSplit split;
+					while ((split = ia.getNextInputSplit("testhost")) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit("testhost"));
+			
+			assertEquals(0, ia.getNumberOfRemoteAssignments());
+			assertEquals(NUM_SPLITS, ia.getNumberOfLocalAssignments());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentSplitAssignmentForMultipleHosts() {
+		try {
+			final int NUM_THREADS = 10;
+			final int NUM_SPLITS = 500;
+			final int SUM_OF_IDS = (NUM_SPLITS-1) * (NUM_SPLITS) / 2;
+			
+			final String[] hosts = { "host1", "host1", "host1", "host2", "host2", "host3" };
+			
+			// load some splits
+			Set<LocatableInputSplit> splits = new HashSet<LocatableInputSplit>();
+			for (int i = 0; i < NUM_SPLITS; i++) {
+				splits.add(new LocatableInputSplit(i, hosts[i%hosts.length]));
+			}
+			
+			final LocatableInputSplitAssigner ia = new LocatableInputSplitAssigner(splits);
+			
+			final AtomicInteger splitsRetrieved = new AtomicInteger(0);
+			final AtomicInteger sumOfIds = new AtomicInteger(0);
+			
+			Runnable retriever = new Runnable() {
+				
+				@Override
+				public void run() {
+					final String threadHost = hosts[(int) (Math.random() * hosts.length)];
+					
+					LocatableInputSplit split;
+					while ((split = ia.getNextInputSplit(threadHost)) != null) {
+						splitsRetrieved.incrementAndGet();
+						sumOfIds.addAndGet(split.getSplitNumber());
+					}
+				}
+			};
+			
+			// create the threads
+			Thread[] threads = new Thread[NUM_THREADS];
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i] = new Thread(retriever);
+				threads[i].setDaemon(true);
+			}
+			
+			// launch concurrently
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].start();
+			}
+			
+			// sync
+			for (int i = 0; i < NUM_THREADS; i++) {
+				threads[i].join(5000);
+			}
+			
+			// verify
+			for (int i = 0; i < NUM_THREADS; i++) {
+				if (threads[i].isAlive()) {
+					fail("The concurrency test case is erroneous, the thread did not respond in time.");
+				}
+			}
+			
+			assertEquals(NUM_SPLITS, splitsRetrieved.get());
+			assertEquals(SUM_OF_IDS, sumOfIds.get());
+			
+			// nothing left
+			assertNull(ia.getNextInputSplit("testhost"));
+			
+			// at least one fraction of hosts needs be local, no matter how bad the thread races
+			assertTrue(ia.getNumberOfLocalAssignments() >= NUM_SPLITS / hosts.length);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}


[61/63] [abbrv] git commit: Adjust the web frontend display of the jobs Tolerate concurrent scheduling attempts

Posted by se...@apache.org.
Adjust the web frontend display of the jobs
Tolerate concurrent scheduling attempts


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/9803657a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/9803657a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/9803657a

Branch: refs/heads/master
Commit: 9803657a93f3a32618796b8a5204749e459fb5e9
Parents: a00ef7a
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 17 19:23:26 2014 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Sat Sep 20 20:21:02 2014 +0200

----------------------------------------------------------------------
 .../web-docs-infoserver/js/analyzer.js          | 27 ++++++++------------
 .../web-docs-infoserver/js/taskmanager.js       |  5 ++--
 .../flink/runtime/executiongraph/Execution.java | 10 +++-----
 .../runtime/executiongraph/ExecutionGraph.java  |  1 +
 .../jobmanager/archive/MemoryArchivist.java     |  4 +--
 .../jobmanager/web/JobmanagerInfoServlet.java   | 15 +++++------
 .../runtime/jobmanager/web/JsonFactory.java     |  4 +--
 .../ExecutionVertexCancelTest.java              |  5 ++--
 8 files changed, 31 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/resources/web-docs-infoserver/js/analyzer.js
----------------------------------------------------------------------
diff --git a/flink-runtime/resources/web-docs-infoserver/js/analyzer.js b/flink-runtime/resources/web-docs-infoserver/js/analyzer.js
index e96f420..f93db1c 100644
--- a/flink-runtime/resources/web-docs-infoserver/js/analyzer.js
+++ b/flink-runtime/resources/web-docs-infoserver/js/analyzer.js
@@ -230,18 +230,18 @@ function analyzeGroupvertexTime(json) {
 
 		data.addRows([
 						[
-							new Date(json.verticetimes[vertex.vertexid].READY),
-							new Date(json.verticetimes[vertex.vertexid].STARTING),
-							"ready",
+							new Date(json.verticetimes[vertex.vertexid].SCHEDULED),
+							new Date(json.verticetimes[vertex.vertexid].DEPLOYING),
+							"scheduled",
 							vertex.vertexinstancename+ "_" + cnt,
-							"ready" 
+							"scheduled" 
 						],
 						[
-							new Date(json.verticetimes[vertex.vertexid].STARTING),
+							new Date(json.verticetimes[vertex.vertexid].DEPLOYING),
 							new Date(json.verticetimes[vertex.vertexid].RUNNING),
-							"starting",
+							"deploying",
 							vertex.vertexinstancename+ "_"+ cnt,
-							"starting" 
+							"deploying" 
 						] 
 					]);
 
@@ -249,17 +249,10 @@ function analyzeGroupvertexTime(json) {
 			data.addRows([
 							[
 								new Date(json.verticetimes[vertex.vertexid].RUNNING),
-								new Date(json.verticetimes[vertex.vertexid].FINISHING),
-								" running",
-								vertex.vertexinstancename + "_" + cnt,
-								"running" 
-							],
-							[
-								new Date(json.verticetimes[vertex.vertexid].FINISHING),
 								new Date(json.verticetimes[vertex.vertexid].FINISHED),
-								"finishing",
+								"running",
 								vertex.vertexinstancename + "_" + cnt,
-								"finishing" 
+								"running" 
 							] 
 						]);
 
@@ -310,4 +303,4 @@ function getSelectedRow(timeline) {
 		}
 	}
 	return row;
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/resources/web-docs-infoserver/js/taskmanager.js
----------------------------------------------------------------------
diff --git a/flink-runtime/resources/web-docs-infoserver/js/taskmanager.js b/flink-runtime/resources/web-docs-infoserver/js/taskmanager.js
index 2bfbed2..7d27d4e 100644
--- a/flink-runtime/resources/web-docs-infoserver/js/taskmanager.js
+++ b/flink-runtime/resources/web-docs-infoserver/js/taskmanager.js
@@ -28,11 +28,11 @@ function loadTaskmanagers(json) {
 	$("#taskmanagerTable").empty();
 	var table = "<table class=\"table table-bordered table-hover table-striped\">";
 	table += "<tr><th>Node</th><th>Ipc Port</th><th>Data Port</th><th>Seconds since last Heartbeat</th>" +
-			"<th>Number of Slots</th><th>Available Slots</th><th>CPU Cores</th><th>Physical Memory (mb)</th><th>TaskManager Heapsize (mb)</th></tr>";
+			"<th>Number of Slots</th><th>Available Slots</th><th>CPU Cores</th><th>Physical Memory (mb)</th><th>TaskManager Heapsize (mb)</th><th>Managed Memory (mb)</th></tr>";
 	for (var i = 0; i < json.taskmanagers.length; i++) {
 		var tm = json.taskmanagers[i]
 		table += "<tr><td>"+tm.inetAdress+"</td><td>"+tm.ipcPort+"</td><td>"+tm.dataPort+"</td><td>"+tm.timeSinceLastHeartbeat+"</td>" +
-				"<td>"+tm.slotsNumber+"</td><td>"+tm.freeSlots+"</td><td>"+tm.cpuCores+"</td><td>"+tm.physicalMemory+"</td><td>"+tm.freeMemory+"</td></tr>";
+				"<td>"+tm.slotsNumber+"</td><td>"+tm.freeSlots+"</td><td>"+tm.cpuCores+"</td><td>"+tm.physicalMemory+"</td><td>"+tm.freeMemory+"</td><<td>"+tm.managedMemory+"</td></tr>";
 	}
 	table += "</table>";
 	$("#taskmanagerTable").append(table);
@@ -47,3 +47,4 @@ function pollTaskmanagers() {
 		pollTaskmanagers();
 	}, 10000);
 }
+

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index b002d8d..37394bd 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -171,7 +171,7 @@ public class Execution {
 		if (locationConstraint != null && sharingGroup == null) {
 			throw new RuntimeException("Trying to schedule with co-location constraint but without slot sharing allowed.");
 		}
-				
+		
 		if (transitionState(CREATED, SCHEDULED)) {
 			
 			ScheduledUnit toSchedule = locationConstraint == null ?
@@ -213,13 +213,9 @@ public class Execution {
 				}
 			}
 		}
-		else if (this.state == CANCELED) {
-			// this can occur very rarely through heavy races. if the task was canceled, we do not
-			// schedule it
-			return;
-		}
 		else {
-			throw new IllegalStateException("The vertex must be in CREATED state to be scheduled.");
+			// call race, already deployed
+			return;
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
index 3ac3386..0c11170 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraph.java
@@ -139,6 +139,7 @@ public class ExecutionGraph {
 		this.executionListeners = new CopyOnWriteArrayList<ExecutionListener>();
 		
 		this.stateTimestamps = new long[JobStatus.values().length];
+		this.stateTimestamps[JobStatus.CREATED.ordinal()] = System.currentTimeMillis();
 	}
 
 	// --------------------------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java
index 2579b75..76d799f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/archive/MemoryArchivist.java
@@ -103,8 +103,8 @@ public class MemoryArchivist implements ArchiveListener {
 	public RecentJobEvent getJob(JobID jobId) {
 		synchronized (lock) {
 			return oldJobs.get(jobId);
-		}}
-	
+		}
+	}
 	
 	@Override
 	public List<AbstractEvent> getEvents(JobID jobID) {

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
index f4c67df..9c8ae27 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JobmanagerInfoServlet.java
@@ -33,13 +33,13 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
 import org.apache.flink.api.common.accumulators.AccumulatorHelper;
 import org.apache.flink.runtime.event.job.AbstractEvent;
 import org.apache.flink.runtime.event.job.ExecutionStateChangeEvent;
 import org.apache.flink.runtime.event.job.JobEvent;
 import org.apache.flink.runtime.event.job.RecentJobEvent;
 import org.apache.flink.runtime.execution.ExecutionState;
+import org.apache.flink.runtime.executiongraph.Execution;
 import org.apache.flink.runtime.executiongraph.ExecutionGraph;
 import org.apache.flink.runtime.executiongraph.ExecutionJobVertex;
 import org.apache.flink.runtime.executiongraph.ExecutionVertex;
@@ -89,7 +89,7 @@ public class JobmanagerInfoServlet extends HttpServlet {
 				writeJsonForArchivedJobGroupvertex(resp.getWriter(), jobmanager.getArchive().getJob(JobID.fromHexString(jobId)), JobVertexID.fromHexString(groupvertexId));
 			}
 			else if("taskmanagers".equals(req.getParameter("get"))) {
-				resp.getWriter().write("{\"taskmanagers\": " + jobmanager.getNumberOfTaskManagers() +"}");
+				resp.getWriter().write("{\"taskmanagers\": " + jobmanager.getNumberOfTaskManagers() +", \"slots\": "+jobmanager.getTotalNumberOfRegisteredSlots()+"}");
 			}
 			else if("cancel".equals(req.getParameter("get"))) {
 				String jobId = req.getParameter("job");
@@ -464,28 +464,27 @@ public class JobmanagerInfoServlet extends HttpServlet {
 			boolean first = true;
 			for (ExecutionJobVertex groupVertex : graph.getAllVertices().values()) {
 				
-				int num = 0;
 				for (ExecutionVertex vertex : groupVertex.getTaskVertices()) {
 					
+					Execution exec = vertex.getCurrentExecutionAttempt();
+					
 					if(first) {
 						first = false;
 					} else {
 						wrt.write(","); }
 					
-					wrt.write("\""+jobVertex.getJobVertex()+"-"+num +"\": {");
-					wrt.write("\"vertexid\": \"" + vertex.getJobvertexId() + "\",");
+					wrt.write("\""+exec.getAttemptId() +"\": {");
+					wrt.write("\"vertexid\": \"" + exec.getAttemptId() + "\",");
 					wrt.write("\"vertexname\": \"" + vertex + "\",");
 					wrt.write("\"CREATED\": "+ vertex.getStateTimestamp(ExecutionState.CREATED) + ",");
 					wrt.write("\"SCHEDULED\": "+ vertex.getStateTimestamp(ExecutionState.SCHEDULED) + ",");
-					wrt.write("\"STARTING\": "+ vertex.getStateTimestamp(ExecutionState.DEPLOYING) + ",");
+					wrt.write("\"DEPLOYING\": "+ vertex.getStateTimestamp(ExecutionState.DEPLOYING) + ",");
 					wrt.write("\"RUNNING\": "+ vertex.getStateTimestamp(ExecutionState.RUNNING) + ",");
 					wrt.write("\"FINISHED\": "+ vertex.getStateTimestamp(ExecutionState.FINISHED) + ",");
 					wrt.write("\"CANCELING\": "+ vertex.getStateTimestamp(ExecutionState.CANCELING) + ",");
 					wrt.write("\"CANCELED\": "+ vertex.getStateTimestamp(ExecutionState.CANCELED) + ",");
 					wrt.write("\"FAILED\": "+ vertex.getStateTimestamp(ExecutionState.FAILED) + "");
 					wrt.write("}");
-					
-					num++;
 				}
 				
 			}

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
index 745a9f8..6ac8613 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/web/JsonFactory.java
@@ -35,7 +35,7 @@ public class JsonFactory {
 	public static String toJson(ExecutionVertex vertex) {
 		StringBuilder json = new StringBuilder("");
 		json.append("{");
-		json.append("\"vertexid\": \"" + vertex.getJobvertexId() + "\",");
+		json.append("\"vertexid\": \"" + vertex.getCurrentExecutionAttempt().getAttemptId() + "\",");
 		json.append("\"vertexname\": \"" + StringUtils.escapeHtml(vertex.getSimpleName()) + "\",");
 		json.append("\"vertexstatus\": \"" + vertex.getExecutionState() + "\",");
 		
@@ -66,7 +66,7 @@ public class JsonFactory {
 		
 		ExecutionVertex[] vertices = jobVertex.getTaskVertices();
 		
-		for(int j = 0; j < vertices.length; j++) {
+		for (int j = 0; j < vertices.length; j++) {
 			ExecutionVertex vertex = vertices[j];
 			
 			json.append(toJson(vertex));

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/9803657a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
index a351209..223f8ec 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexCancelTest.java
@@ -516,9 +516,10 @@ public class ExecutionVertexCancelTest {
 				
 				Scheduler scheduler = mock(Scheduler.class);
 				vertex.scheduleForExecution(scheduler, false);
-				fail("Method should throw an exception");
 			}
-			catch (IllegalStateException e) {}
+			catch (Exception e) {
+				fail("should not throw an exception");
+			}
 			
 			
 			// deploying while in canceling state is illegal (should immediately go to canceled)